From 385fd848302765c9026bc3db6a8c5a00b5ff098d Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 10 Sep 2018 16:18:10 -0700 Subject: [PATCH 01/87] created seekablestream classes --- .../SeekableStreamDataSourceMetadata.java | 79 ++ .../SeekableStreamIOConfig.java | 120 ++ .../SeekableStreamIndexTask.java | 163 +++ .../SeekableStreamIndexTaskClient.java | 270 +++++ .../SeekableStreamIndexTaskClientFactory.java | 56 + .../SeekableStreamPartitions.java | 79 ++ .../SeekableStreamTuningConfig.java | 348 ++++++ .../SeekableStream/common/Record.java | 46 + .../SeekableStream/common/RecordSupplier.java | 33 + .../common/SequenceNumberPlus.java | 29 + .../common/StreamPartition.java | 64 ++ .../supervisor/SeekableStreamSupervisor.java | 1005 +++++++++++++++++ .../SeekableStreamSupervisorIOConfig.java | 139 +++ ...SeekableStreamSupervisorReportPayload.java | 144 +++ .../SeekableStreamSupervisorSpec.java | 184 +++ .../SeekableStreamSupervisorTuningConfig.java | 163 +++ .../supervisor/TaskReportData.java | 120 ++ 17 files changed, 3042 insertions(+) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamDataSourceMetadata.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIOConfig.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTask.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClient.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClientFactory.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamPartitions.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamTuningConfig.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/Record.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/RecordSupplier.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/SequenceNumberPlus.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/StreamPartition.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisor.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorIOConfig.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorReportPayload.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorSpec.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorTuningConfig.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/TaskReportData.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamDataSourceMetadata.java new file mode 100644 index 000000000000..4f5875eddce2 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamDataSourceMetadata.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.SeekableStream; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.overlord.DataSourceMetadata; + +import java.util.Objects; + +abstract public class SeekableStreamDataSourceMetadata implements DataSourceMetadata +{ + private final SeekableStreamPartitions seekableStreamPartitions; + + @JsonCreator + public SeekableStreamDataSourceMetadata( + @JsonProperty("partitions") SeekableStreamPartitions seekableStreamPartitions + ) + { + this.seekableStreamPartitions = seekableStreamPartitions; + } + + @JsonProperty("partitions") + public SeekableStreamPartitions getSeekableStreamPartitions() + { + return seekableStreamPartitions; + } + + @Override + public boolean isValidStart() + { + return true; + } + + @Override + public boolean matches(DataSourceMetadata other) + { + if (getClass() != other.getClass()) { + return false; + } + + return plus(other).equals(other.plus(this)); + } + + @Override + abstract public DataSourceMetadata plus(DataSourceMetadata other); + + @Override + abstract public DataSourceMetadata minus(DataSourceMetadata other); + + @Override + abstract public boolean equals(Object o); + + @Override + public int hashCode() + { + return Objects.hash(seekableStreamPartitions); + } + + @Override + abstract public String toString(); +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIOConfig.java new file mode 100644 index 000000000000..861bb8ae87bb --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIOConfig.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.SeekableStream; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import org.apache.druid.segment.indexing.IOConfig; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; + +abstract public class SeekableStreamIOConfig implements IOConfig +{ + private static final boolean DEFAULT_USE_TRANSACTION = true; + private static final boolean DEFAULT_SKIP_OFFSET_GAPS = false; + + @Nullable + private final Integer taskGroupId; + private final String baseSequenceName; + private final SeekableStreamPartitions startPartitions; + private final SeekableStreamPartitions endPartitions; + private final boolean useTransaction; + private final Optional minimumMessageTime; + private final Optional maximumMessageTime; + + @JsonCreator + public SeekableStreamIOConfig( + @JsonProperty("taskGroupId") @Nullable Integer taskGroupId, // can be null for backward compabitility + @JsonProperty("baseSequenceName") String baseSequenceName, + @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, + @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, + @JsonProperty("useTransaction") Boolean useTransaction, + @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, + @JsonProperty("maximumMessageTime") DateTime maximumMessageTime + ) + { + this.taskGroupId = taskGroupId; + this.baseSequenceName = Preconditions.checkNotNull(baseSequenceName, "baseSequenceName"); + this.startPartitions = Preconditions.checkNotNull(startPartitions, "startPartitions"); + this.endPartitions = Preconditions.checkNotNull(endPartitions, "endPartitions"); + this.useTransaction = useTransaction != null ? useTransaction : DEFAULT_USE_TRANSACTION; + this.minimumMessageTime = Optional.fromNullable(minimumMessageTime); + this.maximumMessageTime = Optional.fromNullable(maximumMessageTime); + + Preconditions.checkArgument( + startPartitions.getId().equals(endPartitions.getId()), + "start id and end id must match" + ); + + Preconditions.checkArgument( + startPartitions.getPartitionSequenceMap().keySet().equals(endPartitions.getPartitionSequenceMap().keySet()), + "start partition set and end partition set must match" + ); + } + + @Nullable + @JsonProperty + public Integer getTaskGroupId() + { + return taskGroupId; + } + + @JsonProperty + public String getBaseSequenceName() + { + return baseSequenceName; + } + + @JsonProperty + public SeekableStreamPartitions getStartPartitions() + { + return startPartitions; + } + + @JsonProperty + public SeekableStreamPartitions getEndPartitions() + { + return endPartitions; + } + + @JsonProperty + public boolean isUseTransaction() + { + return useTransaction; + } + + @JsonProperty + public Optional getMaximumMessageTime() + { + return maximumMessageTime; + } + + @JsonProperty + public Optional getMinimumMessageTime() + { + return minimumMessageTime; + } + + @Override + abstract public String toString(); +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTask.java new file mode 100644 index 000000000000..554bc706a9f6 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTask.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.SeekableStream; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.AbstractTask; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.realtime.firehose.ChatHandler; +import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.utils.CircularBuffer; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Random; + +abstract public class SeekableStreamIndexTask extends AbstractTask implements ChatHandler +{ + public enum Status + { + NOT_STARTED, + STARTING, + READING, + PAUSED, + PUBLISHING + } + + private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTask.class); + private static final String TYPE = "index_seekable_stream"; + private static final Random RANDOM = new Random(); + + private final DataSchema dataSchema; + private final InputRowParser parser; + private final SeekableStreamTuningConfig tuningConfig; + private final SeekableStreamIOConfig ioConfig; + private final Optional chatHandlerProvider; + + @JsonCreator + public SeekableStreamIndexTask( + @JsonProperty("id") String id, + @JsonProperty("resource") TaskResource taskResource, + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("tuningConfig") SeekableStreamTuningConfig tuningConfig, + @JsonProperty("ioConfig") SeekableStreamIOConfig ioConfig, + @JsonProperty("context") Map context, + @JacksonInject ChatHandlerProvider chatHandlerProvider, + @JacksonInject AuthorizerMapper authorizerMapper, + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + ) + { + super( + id == null ? makeTaskId(dataSchema.getDataSource(), RANDOM.nextInt()) : id, + StringUtils.format("%s_%s", TYPE, dataSchema.getDataSource()), + taskResource, + dataSchema.getDataSource(), + context + ); + + this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); + this.parser = Preconditions.checkNotNull((InputRowParser) dataSchema.getParser(), "parser"); + this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); + this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); + this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); + final CircularBuffer savedParseExceptions; + if (tuningConfig.getMaxSavedParseExceptions() > 0) { + savedParseExceptions = new CircularBuffer<>(tuningConfig.getMaxSavedParseExceptions()); + } else { + savedParseExceptions = null; + } + } + + private static String makeTaskId(String dataSource, int randomBits) + { + final StringBuilder suffix = new StringBuilder(8); + for (int i = 0; i < Integer.BYTES * 2; ++i) { + suffix.append((char) ('a' + ((randomBits >>> (i * 4)) & 0x0F))); + } + return Joiner.on("_").join(TYPE, dataSource, suffix); + } + + @Override + public int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY); + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) + { + return true; + } + + @JsonProperty + public DataSchema getDataSchema() + { + return dataSchema; + } + + @JsonProperty + public SeekableStreamTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @JsonProperty("ioConfig") + public SeekableStreamIOConfig getIOConfig() + { + return ioConfig; + } + + + @Override + abstract public TaskStatus run(final TaskToolbox toolbox); + + @Override + abstract public boolean canRestore(); + + @Override + abstract public void stopGracefully(); + + @Override + abstract public QueryRunner getQueryRunner(Query query); + +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClient.java new file mode 100644 index 000000000000..9f8dbc5425b9 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClient.java @@ -0,0 +1,270 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.SeekableStream; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.indexing.common.IndexTaskClient; +import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.response.FullResponseHolder; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.joda.time.DateTime; +import org.joda.time.Duration; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.TreeMap; + +//TODO: Need to refactor implemented methods +abstract public class SeekableStreamIndexTaskClient extends IndexTaskClient +{ + private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTaskClient.class); + + public SeekableStreamIndexTaskClient( + HttpClient httpClient, + ObjectMapper jsonMapper, + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ) + { + super(httpClient, jsonMapper, taskInfoProvider, httpTimeout, dataSource, numThreads, numRetries); + } + + public boolean stop(final String id, final boolean publish) + { + log.debug("Stop task[%s] publish[%s]", id, publish); + + try { + final FullResponseHolder response = submitRequestWithEmptyContent( + id, HttpMethod.POST, "stop", publish ? "publish=true" : null, true + ); + return isSuccess(response); + } + catch (NoTaskLocationException e) { + return false; + } + catch (TaskNotRunnableException e) { + log.info("Task [%s] couldn't be stopped because it is no longer running", id); + return true; + } + catch (Exception e) { + log.warn(e, "Exception while stopping task [%s]", id); + return false; + } + } + + public boolean resume(final String id) + { + log.debug("Resume task[%s]", id); + + try { + final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.POST, "resume", null, true); + return isSuccess(response); + } + catch (NoTaskLocationException | IOException e) { + log.warn(e, "Exception while stopping task [%s]", id); + return false; + } + } + + + abstract public Map pause(final String id); + + + abstract public SeekableStreamIndexTask.Status getStatus(final String id); + + + @Nullable + public DateTime getStartTime(final String id) + { + log.debug("GetStartTime task[%s]", id); + + try { + final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "time/start", null, true); + return response.getContent() == null || response.getContent().isEmpty() + ? null + : deserialize(response.getContent(), DateTime.class); + } + catch (NoTaskLocationException e) { + return null; + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + public Map getMovingAverages(final String id) + { + log.debug("GetMovingAverages task[%s]", id); + + try { + final FullResponseHolder response = submitRequestWithEmptyContent( + id, + HttpMethod.GET, + "rowStats", + null, + true + ); + return response.getContent() == null || response.getContent().isEmpty() + ? Collections.emptyMap() + : deserialize(response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT); + } + catch (NoTaskLocationException e) { + return Collections.emptyMap(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + public Map getCurrentOffsets(final String id, final boolean retry) + { + log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry); + + try { + final FullResponseHolder response = submitRequestWithEmptyContent( + id, + HttpMethod.GET, + "offsets/current", + null, + retry + ); + return deserialize(response.getContent(), new TypeReference>() + { + }); + } + catch (NoTaskLocationException e) { + return ImmutableMap.of(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + public TreeMap> getCheckpoints(final String id, final boolean retry) + { + log.debug("GetCheckpoints task[%s] retry[%s]", id, retry); + try { + final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "checkpoints", null, retry); + return deserialize( + response.getContent(), + new TypeReference>>() + { + } + ); + } + catch (NoTaskLocationException e) { + return new TreeMap<>(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + public ListenableFuture>> getCheckpointsAsync( + final String id, + final boolean retry + ) + { + return doAsync(() -> getCheckpoints(id, retry)); + } + + public Map getEndOffsets(final String id) + { + log.debug("GetEndOffsets task[%s]", id); + + try { + final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "offsets/end", null, true); + return deserialize(response.getContent(), new TypeReference>() + { + }); + } + catch (NoTaskLocationException e) { + return ImmutableMap.of(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + public boolean setEndOffsets( + final String id, + final Map endOffsets, + final boolean finalize + ) throws IOException + { + log.debug("SetEndOffsets task[%s] endOffsets[%s] finalize[%s]", id, endOffsets, finalize); + + try { + final FullResponseHolder response = submitJsonRequest( + id, + HttpMethod.POST, + "offsets/end", + StringUtils.format("finish=%s", finalize), + serialize(endOffsets), + true + ); + return isSuccess(response); + } + catch (NoTaskLocationException e) { + return false; + } + } + + public ListenableFuture stopAsync(final String id, final boolean publish) + { + return doAsync(() -> stop(id, publish)); + } + + public ListenableFuture getStartTimeAsync(final String id) + { + return doAsync(() -> getStartTime(id)); + } + + + public ListenableFuture> pauseAsync(final String id) + { + return doAsync(() -> pause(id)); + } + + public ListenableFuture setEndOffsetsAsync( + final String id, + final Map endOffsets, + final boolean finalize + ) + { + return doAsync(() -> setEndOffsets(id, endOffsets, finalize)); + } + + abstract public ListenableFuture getStatusAsync(final String id); + +} + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClientFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClientFactory.java new file mode 100644 index 000000000000..4ec028558233 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClientFactory.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.SeekableStream; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.java.util.http.client.HttpClient; +import org.joda.time.Duration; + +abstract public class SeekableStreamIndexTaskClientFactory + implements IndexTaskClientFactory +{ + private HttpClient httpClient; + private ObjectMapper mapper; + + @Inject + public SeekableStreamIndexTaskClientFactory( + @EscalatedGlobal HttpClient httpClient, + @Json ObjectMapper mapper + ) + { + this.httpClient = httpClient; + this.mapper = mapper; + } + + @Override + abstract public SeekableStreamIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ); + +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamPartitions.java new file mode 100644 index 000000000000..06331ecfb4f3 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamPartitions.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.SeekableStream; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableMap; + +import java.util.Map; +import java.util.Objects; + +abstract public class SeekableStreamPartitions +{ + private final String id; + private final Map partitionOffsetMap; + + @JsonCreator + public SeekableStreamPartitions( + @JsonProperty("id") final String id, + @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap + ) + { + this.id = id; + this.partitionOffsetMap = ImmutableMap.copyOf(partitionOffsetMap); + } + + @JsonProperty + public String getId() + { + return id; + } + + @JsonProperty + public Map getPartitionSequenceMap() + { + return partitionOffsetMap; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SeekableStreamPartitions that = (SeekableStreamPartitions) o; + return Objects.equals(id, that.id) && + Objects.equals(partitionOffsetMap, that.partitionOffsetMap); + } + + @Override + public int hashCode() + { + return Objects.hash(id, partitionOffsetMap); + } + + @Override + abstract public String toString(); + +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamTuningConfig.java new file mode 100644 index 000000000000..6a52448af5f9 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamTuningConfig.java @@ -0,0 +1,348 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.SeekableStream; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.indexing.RealtimeTuningConfig; +import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.Objects; + +public class SeekableStreamTuningConfig implements TuningConfig, AppenderatorConfig +{ + private static final int DEFAULT_MAX_ROWS_PER_SEGMENT = 5_000_000; + private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false; + + private final int maxRowsInMemory; + private final long maxBytesInMemory; + private final int maxRowsPerSegment; + private final Period intermediatePersistPeriod; + private final File basePersistDirectory; + @Deprecated + private final int maxPendingPersists; + private final IndexSpec indexSpec; + private final boolean reportParseExceptions; + private final long handoffConditionTimeout; + private final boolean resetOffsetAutomatically; + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + private final Period intermediateHandoffPeriod; + + private final boolean logParseExceptions; + private final int maxParseExceptions; + private final int maxSavedParseExceptions; + + @JsonCreator + public SeekableStreamTuningConfig( + @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, + @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, + @JsonProperty("intermediatePersistPeriod") @Nullable Period intermediatePersistPeriod, + @JsonProperty("basePersistDirectory") @Nullable File basePersistDirectory, + @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, + @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, + // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. + @JsonProperty("buildV9Directly") @Nullable Boolean buildV9Directly, + @Deprecated @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, + @JsonProperty("handoffConditionTimeout") @Nullable Long handoffConditionTimeout, + @JsonProperty("resetOffsetAutomatically") @Nullable Boolean resetOffsetAutomatically, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions + ) + { + // Cannot be a static because default basePersistDirectory is unique per-instance + final RealtimeTuningConfig defaults = RealtimeTuningConfig.makeDefaultTuningConfig(basePersistDirectory); + + this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; + this.maxRowsPerSegment = maxRowsPerSegment == null ? DEFAULT_MAX_ROWS_PER_SEGMENT : maxRowsPerSegment; + // initializing this to 0, it will be lazily initialized to a value + // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) + this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; + this.intermediatePersistPeriod = intermediatePersistPeriod == null + ? defaults.getIntermediatePersistPeriod() + : intermediatePersistPeriod; + this.basePersistDirectory = defaults.getBasePersistDirectory(); + this.maxPendingPersists = 0; + this.indexSpec = indexSpec == null ? defaults.getIndexSpec() : indexSpec; + this.reportParseExceptions = reportParseExceptions == null + ? defaults.isReportParseExceptions() + : reportParseExceptions; + this.handoffConditionTimeout = handoffConditionTimeout == null + ? defaults.getHandoffConditionTimeout() + : handoffConditionTimeout; + this.resetOffsetAutomatically = resetOffsetAutomatically == null + ? DEFAULT_RESET_OFFSET_AUTOMATICALLY + : resetOffsetAutomatically; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + this.intermediateHandoffPeriod = intermediateHandoffPeriod == null + ? new Period().withDays(Integer.MAX_VALUE) + : intermediateHandoffPeriod; + + if (this.reportParseExceptions) { + this.maxParseExceptions = 0; + this.maxSavedParseExceptions = maxSavedParseExceptions == null ? 0 : Math.min(1, maxSavedParseExceptions); + } else { + this.maxParseExceptions = maxParseExceptions == null + ? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS + : maxParseExceptions; + this.maxSavedParseExceptions = maxSavedParseExceptions == null + ? TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS + : maxSavedParseExceptions; + } + this.logParseExceptions = logParseExceptions == null + ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS + : logParseExceptions; + } + + public static SeekableStreamTuningConfig copyOf(SeekableStreamTuningConfig config) + { + return new SeekableStreamTuningConfig( + config.maxRowsInMemory, + config.maxBytesInMemory, + config.maxRowsPerSegment, + config.intermediatePersistPeriod, + config.basePersistDirectory, + config.maxPendingPersists, + config.indexSpec, + true, + config.reportParseExceptions, + config.handoffConditionTimeout, + config.resetOffsetAutomatically, + config.segmentWriteOutMediumFactory, + config.intermediateHandoffPeriod, + config.logParseExceptions, + config.maxParseExceptions, + config.maxSavedParseExceptions + ); + } + + @Override + @JsonProperty + public int getMaxRowsInMemory() + { + return maxRowsInMemory; + } + + @Override + @JsonProperty + public long getMaxBytesInMemory() + { + return maxBytesInMemory; + } + + @JsonProperty + public int getMaxRowsPerSegment() + { + return maxRowsPerSegment; + } + + @Override + @JsonProperty + public Period getIntermediatePersistPeriod() + { + return intermediatePersistPeriod; + } + + @Override + @JsonProperty + public File getBasePersistDirectory() + { + return basePersistDirectory; + } + + @Override + @JsonProperty + @Deprecated + public int getMaxPendingPersists() + { + return maxPendingPersists; + } + + @Override + @JsonProperty + public IndexSpec getIndexSpec() + { + return indexSpec; + } + + /** + * Always returns true, doesn't affect the version being built. + */ + @Deprecated + @JsonProperty + public boolean getBuildV9Directly() + { + return true; + } + + @Override + @JsonProperty + public boolean isReportParseExceptions() + { + return reportParseExceptions; + } + + @JsonProperty + public long getHandoffConditionTimeout() + { + return handoffConditionTimeout; + } + + @JsonProperty + public boolean isResetOffsetAutomatically() + { + return resetOffsetAutomatically; + } + + @Override + @JsonProperty + @Nullable + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + return segmentWriteOutMediumFactory; + } + + @JsonProperty + public Period getIntermediateHandoffPeriod() + { + return intermediateHandoffPeriod; + } + + @JsonProperty + public boolean isLogParseExceptions() + { + return logParseExceptions; + } + + @JsonProperty + public int getMaxParseExceptions() + { + return maxParseExceptions; + } + + @JsonProperty + public int getMaxSavedParseExceptions() + { + return maxSavedParseExceptions; + } + + public SeekableStreamTuningConfig withBasePersistDirectory(File dir) + { + return new SeekableStreamTuningConfig( + maxRowsInMemory, + maxBytesInMemory, + maxRowsPerSegment, + intermediatePersistPeriod, + dir, + maxPendingPersists, + indexSpec, + true, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + segmentWriteOutMediumFactory, + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions + ); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SeekableStreamTuningConfig that = (SeekableStreamTuningConfig) o; + return maxRowsInMemory == that.maxRowsInMemory && + maxRowsPerSegment == that.maxRowsPerSegment && + maxBytesInMemory == that.maxBytesInMemory && + maxPendingPersists == that.maxPendingPersists && + reportParseExceptions == that.reportParseExceptions && + handoffConditionTimeout == that.handoffConditionTimeout && + resetOffsetAutomatically == that.resetOffsetAutomatically && + Objects.equals(intermediatePersistPeriod, that.intermediatePersistPeriod) && + Objects.equals(basePersistDirectory, that.basePersistDirectory) && + Objects.equals(indexSpec, that.indexSpec) && + Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory) && + Objects.equals(intermediateHandoffPeriod, that.intermediateHandoffPeriod) && + logParseExceptions == that.logParseExceptions && + maxParseExceptions == that.maxParseExceptions && + maxSavedParseExceptions == that.maxSavedParseExceptions; + } + + @Override + public int hashCode() + { + return Objects.hash( + maxRowsInMemory, + maxRowsPerSegment, + maxBytesInMemory, + intermediatePersistPeriod, + basePersistDirectory, + maxPendingPersists, + indexSpec, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + segmentWriteOutMediumFactory, + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions + ); + } + + @Override + public String toString() + { + return "SeekableTuningConfig{" + + "maxRowsInMemory=" + maxRowsInMemory + + ", maxRowsPerSegment=" + maxRowsPerSegment + + ", maxBytesInMemory=" + maxBytesInMemory + + ", intermediatePersistPeriod=" + intermediatePersistPeriod + + ", basePersistDirectory=" + basePersistDirectory + + ", maxPendingPersists=" + maxPendingPersists + + ", indexSpec=" + indexSpec + + ", reportParseExceptions=" + reportParseExceptions + + ", handoffConditionTimeout=" + handoffConditionTimeout + + ", resetOffsetAutomatically=" + resetOffsetAutomatically + + ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + + ", intermediateHandoffPeriod=" + intermediateHandoffPeriod + + ", logParseExceptions=" + logParseExceptions + + ", maxParseExceptions=" + maxParseExceptions + + ", maxSavedParseExceptions=" + maxSavedParseExceptions + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/Record.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/Record.java new file mode 100644 index 000000000000..770614f3ef5c --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/Record.java @@ -0,0 +1,46 @@ +package org.apache.druid.indexing.SeekableStream.common; + +import java.util.List; + +public class Record +{ + public static final String END_OF_SHARD_MARKER = "EOS"; + + private final String streamName; + private final PartitionType partitionId; + private final SequenceType sequenceNumber; + private final List data; + + public Record(String streamName, PartitionType partitionId, SequenceType sequenceNumber, List data) + { + this.streamName = streamName; + this.partitionId = partitionId; + this.sequenceNumber = sequenceNumber; + this.data = data; + } + + public String getStreamName() + { + return streamName; + } + + public PartitionType getPartitionId() + { + return partitionId; + } + + public SequenceType getSequenceNumber() + { + return sequenceNumber; + } + + public List getData() + { + return data; + } + + public StreamPartition getStreamPartition() + { + return StreamPartition.of(streamName, partitionId); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/RecordSupplier.java new file mode 100644 index 000000000000..6e886a4ae895 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/RecordSupplier.java @@ -0,0 +1,33 @@ +package org.apache.druid.indexing.SeekableStream.common; + +import java.io.Closeable; +import java.util.Set; +import java.util.concurrent.TimeoutException; + +/** + * The RecordSupplier interface is a wrapper for the incoming seekable data stream + * (i.e. Kafka consumer) + */ +public interface RecordSupplier extends Closeable +{ + void assign(Set> partitions); + + void seek(StreamPartition partition, SequenceType sequenceNumber); + + void seekAfter(StreamPartition partition, SequenceType sequenceNumber); + + void seekToEarliest(StreamPartition partition); + + void seekToLatest(StreamPartition partition); + + Record poll(long timeout); + + SequenceType getLatestSequenceNumber(StreamPartition partition) throws TimeoutException; + + SequenceType getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException; + + Set getPartitionIds(String streamName); + + @Override + void close(); +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/SequenceNumberPlus.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/SequenceNumberPlus.java new file mode 100644 index 000000000000..cc9789edda7d --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/SequenceNumberPlus.java @@ -0,0 +1,29 @@ +package org.apache.druid.indexing.SeekableStream.common; + + +public class SequenceNumberPlus +{ + private final String sequenceNumber; + private final boolean exclusive; + + private SequenceNumberPlus(String sequenceNumber, boolean exclusive) + { + this.sequenceNumber = sequenceNumber; + this.exclusive = exclusive; + } + + public String get() + { + return sequenceNumber; + } + + public boolean isExclusive() + { + return exclusive; + } + + public static SequenceNumberPlus of(String sequenceNumber, boolean exclusive) + { + return new SequenceNumberPlus(sequenceNumber, exclusive); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/StreamPartition.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/StreamPartition.java new file mode 100644 index 000000000000..585fd539e7f8 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/StreamPartition.java @@ -0,0 +1,64 @@ +package org.apache.druid.indexing.SeekableStream.common; + +public class StreamPartition +{ + private final String streamName; + private final PartitionType partitionId; + + public StreamPartition(String streamName, PartitionType partitionId) + { + this.streamName = streamName; + this.partitionId = partitionId; + } + + public String getStreamName() + { + return streamName; + } + + public PartitionType getPartitionId() + { + return partitionId; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + StreamPartition that = (StreamPartition) o; + + if (streamName != null ? !streamName.equals(that.streamName) : that.streamName != null) { + return false; + } + return !(partitionId != null ? !partitionId.equals(that.partitionId) : that.partitionId != null); + } + + @Override + public int hashCode() + { + int result = streamName != null ? streamName.hashCode() : 0; + result = 31 * result + (partitionId != null ? partitionId.hashCode() : 0); + return result; + } + + @Override + public String toString() + { + return "StreamPartition{" + + "streamName='" + streamName + '\'' + + ", partitionId='" + partitionId + '\'' + + '}'; + } + + public static StreamPartition of(String streamName, PartitionType partitionId) + { + return new StreamPartition<>(streamName, partitionId); + } +} + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisor.java new file mode 100644 index 000000000000..17fd0af58b33 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisor.java @@ -0,0 +1,1005 @@ +package org.apache.druid.indexing.SeekableStream.supervisor; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.apache.commons.codec.digest.DigestUtils; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.SeekableStream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask; +import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTaskClient; +import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTaskClientFactory; +import org.apache.druid.indexing.SeekableStream.SeekableStreamTuningConfig; +import org.apache.druid.indexing.SeekableStream.common.RecordSupplier; +import org.apache.druid.indexing.common.IndexTaskClient; +import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskQueue; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.Supervisor; +import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +abstract public class SeekableStreamSupervisor implements Supervisor +{ + //---------------------------------------GOOD--------------------------------------------------------- + private static final EmittingLogger log = new EmittingLogger(SeekableStreamSupervisor.class); + private static final Random RANDOM = new Random(); + private static final long MAX_RUN_FREQUENCY_MILLIS = 1000; + private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120; + //---------------------------------------GOOD--------------------------------------------------------- + + + public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED"; + + private class TaskGroup + { + final int groupId; + + final ImmutableMap partitionOffsets; + final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); + final Optional minimumMessageTime; + final Optional maximumMessageTime; + final Set exclusiveStartSequenceNumberPartitions; + final TreeMap> sequenceOffsets = new TreeMap<>(); + final String baseSequenceName; + DateTime completionTimeout; + + public TaskGroup( + int groupId, + ImmutableMap partitionOffsets, + Optional minimumMessageTime, + Optional maximumMessageTime, + Set exclusiveStartSequenceNumberPartitions + ) + { + this.groupId = groupId; + this.partitionOffsets = partitionOffsets; + this.minimumMessageTime = minimumMessageTime; + this.maximumMessageTime = maximumMessageTime; + this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions != null + ? exclusiveStartSequenceNumberPartitions + : new HashSet<>(); + this.baseSequenceName = generateSequenceName(partitionOffsets, minimumMessageTime, maximumMessageTime); + } + + int addNewCheckpoint(Map checkpoint) + { + sequenceOffsets.put(sequenceOffsets.lastKey() + 1, checkpoint); + return sequenceOffsets.lastKey(); + } + + public Set taskIds() + { + return tasks.keySet(); + } + } + + private class TaskData + { + volatile TaskStatus status; + volatile DateTime startTime; + volatile Map currentOffsets = new HashMap<>(); + + @Override + public String toString() + { + return "TaskData{" + + "status=" + status + + ", startTime=" + startTime + + ", currentOffsets=" + currentOffsets + + '}'; + } + } + + //TODO: doc + private final ConcurrentHashMap taskGroups = new ConcurrentHashMap<>(); + private final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); + private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); + + private final TaskStorage taskStorage; + private final TaskMaster taskMaster; + private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; + private final SeekableStreamIndexTaskClient taskClient; + private final ObjectMapper sortingMapper; + private final SeekableStreamSupervisorSpec spec; + private final ServiceEmitter emitter; + private final DruidMonitorSchedulerConfig monitorSchedulerConfig; + private final String dataSource; + private final SeekableStreamSupervisorIOConfig ioConfig; + private final SeekableStreamSupervisorTuningConfig tuningConfig; + private final SeekableStreamTuningConfig taskTuningConfig; + private final String supervisorId; + private final TaskInfoProvider taskInfoProvider; + private final long futureTimeoutInSeconds; // how long to wait for async operations to complete + private final RowIngestionMetersFactory rowIngestionMetersFactory; + + private final ExecutorService exec; + private final ScheduledExecutorService scheduledExec; + private final ScheduledExecutorService reportingExec; + private final ListeningExecutorService workerExec; + private final BlockingQueue notices = new LinkedBlockingDeque<>(); + private final Object stopLock = new Object(); + private final Object stateChangeLock = new Object(); + private final Object consumerLock = new Object(); + + private boolean listenerRegistered = false; + private long lastRunTime; + + private volatile DateTime firstRunTime; + private volatile DateTime earlyPublishTime = null; + private volatile RecordSupplier recordSupplier; + private volatile boolean started = false; + private volatile boolean stopped = false; + + private final T2 NOT_SET; + + //-------------------------------GOOD---------------------------------------- + + /** + * Notice is used to queue tasks that are internal to the supervisor + */ + private interface Notice + { + void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException; + } + + private class RunNotice implements Notice + { + @Override + public void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException + { + long nowTime = System.currentTimeMillis(); + if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { + return; + } + lastRunTime = nowTime; + + runInternal(); + } + } + + private class GracefulShutdownNotice extends ShutdownNotice + { + @Override + public void handle() throws InterruptedException, ExecutionException, TimeoutException + { + gracefulShutdownInternal(); + super.handle(); + } + } + + private class ShutdownNotice implements Notice + { + @Override + public void handle() throws InterruptedException, ExecutionException, TimeoutException + { + recordSupplier.close(); + + synchronized (stopLock) { + stopped = true; + stopLock.notifyAll(); + } + } + } + + private class ResetNotice implements Notice + { + final DataSourceMetadata dataSourceMetadata; + + ResetNotice(DataSourceMetadata dataSourceMetadata) + { + this.dataSourceMetadata = dataSourceMetadata; + } + + @Override + public void handle() + { + resetInternal(dataSourceMetadata); + } + } + + //---------------------------GOOD----------------------------------------- + public SeekableStreamSupervisor( + final String supervisorId, + final TaskStorage taskStorage, + final TaskMaster taskMaster, + final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + final SeekableStreamIndexTaskClientFactory taskClientFactory, + final ObjectMapper mapper, + final SeekableStreamSupervisorSpec spec, + final RowIngestionMetersFactory rowIngestionMetersFactory, + final T2 NOT_SET + ) + { + this.taskStorage = taskStorage; + this.taskMaster = taskMaster; + this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; + this.sortingMapper = mapper.copy().configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); + this.spec = spec; + this.emitter = spec.getEmitter(); + this.monitorSchedulerConfig = spec.getMonitorSchedulerConfig(); + this.rowIngestionMetersFactory = rowIngestionMetersFactory; + this.NOT_SET = NOT_SET; + + this.dataSource = spec.getDataSchema().getDataSource(); + this.ioConfig = spec.getIoConfig(); + this.tuningConfig = spec.getTuningConfig(); + this.taskTuningConfig = SeekableStreamTuningConfig.copyOf(this.tuningConfig); + this.supervisorId = supervisorId; + this.exec = Execs.singleThreaded(supervisorId); + this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d"); + this.reportingExec = Execs.scheduledSingleThreaded(supervisorId + "-Reporting-%d"); + + int workerThreads = (this.tuningConfig.getWorkerThreads() != null + ? this.tuningConfig.getWorkerThreads() + : Math.min(10, this.ioConfig.getTaskCount())); + + this.workerExec = MoreExecutors.listeningDecorator(Execs.multiThreaded(workerThreads, supervisorId + "-Worker-%d")); + log.info("Created worker pool with [%d] threads for dataSource [%s]", workerThreads, this.dataSource); + + this.taskInfoProvider = new TaskInfoProvider() + { + @Override + public TaskLocation getTaskLocation(final String id) + { + Preconditions.checkNotNull(id, "id"); + Optional taskRunner = taskMaster.getTaskRunner(); + if (taskRunner.isPresent()) { + Optional item = Iterables.tryFind( + taskRunner.get().getRunningTasks(), + (Predicate) taskRunnerWorkItem -> id.equals(taskRunnerWorkItem.getTaskId()) + ); + + if (item.isPresent()) { + return item.get().getLocation(); + } + } else { + log.error("Failed to get task runner because I'm not the leader!"); + } + + return TaskLocation.unknown(); + } + + @Override + public Optional getTaskStatus(String id) + { + return taskStorage.getStatus(id); + } + }; + + this.futureTimeoutInSeconds = Math.max( + MINIMUM_FUTURE_TIMEOUT_IN_SECONDS, + tuningConfig.getChatRetries() * (tuningConfig.getHttpTimeout().getStandardSeconds() + + IndexTaskClient.MAX_RETRY_WAIT_SECONDS) + ); + + + int chatThreads = (this.tuningConfig.getChatThreads() != null + ? this.tuningConfig.getChatThreads() + : Math.min(10, this.ioConfig.getTaskCount() * this.ioConfig.getReplicas())); + this.taskClient = taskClientFactory.build( + taskInfoProvider, + dataSource, + chatThreads, + this.tuningConfig.getHttpTimeout(), + this.tuningConfig.getChatRetries() + ); + log.info( + "Created taskClient with dataSource[%s] chatThreads[%d] httpTimeout[%s] chatRetries[%d]", + dataSource, + chatThreads, + this.tuningConfig.getHttpTimeout(), + this.tuningConfig.getChatRetries() + ); + } + + //-------------------------------GOOD---------------------------------------- + @Override + public void start() + { + synchronized (stateChangeLock) { + Preconditions.checkState(!started, "already started"); + Preconditions.checkState(!exec.isShutdown(), "already stopped"); + + try { + recordSupplier = setupRecordSupplier(); + + exec.submit( + () -> { + try { + while (!Thread.currentThread().isInterrupted()) { + final Notice notice = notices.take(); + + try { + notice.handle(); + } + catch (Throwable e) { + log.makeAlert(e, "[%s] failed to handle notice", supervisorId) + .addData("noticeClass", notice.getClass().getSimpleName()) + .emit(); + } + } + } + catch (InterruptedException e) { + log.info("[%s] interrupted, exiting", supervisorId); + } + } + ); + firstRunTime = DateTimes.nowUtc().plus(ioConfig.getStartDelay()); + scheduledExec.scheduleAtFixedRate( + () -> notices.add(new RunNotice()), + ioConfig.getStartDelay().getMillis(), + Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS), + TimeUnit.MILLISECONDS + ); + + + //TODO: this logic is different from kafka and kinesis, need to look into + scheduleReporting(); + + started = true; + log.info( + "Started [%s], first run in [%s], with spec: [%s]", + supervisorId, + ioConfig.getStartDelay(), + spec.toString() + ); + + } + catch (Exception e) { + if (recordSupplier != null) { + recordSupplier.close(); + } + log.makeAlert(e, "Exception starting [%s]", supervisorId) + .emit(); + throw Throwables.propagate(e); + } + } + } + + + //---------------------------GOOOD------------------------------ + @Override + public void stop(boolean stopGracefully) + { + synchronized (stateChangeLock) { + Preconditions.checkState(started, "not started"); + + log.info("Beginning shutdown of [%s]", supervisorId); + + try { + scheduledExec.shutdownNow(); // stop recurring executions + reportingExec.shutdownNow(); + recordSupplier.close(); + + Optional taskRunner = taskMaster.getTaskRunner(); + if (taskRunner.isPresent()) { + taskRunner.get().unregisterListener(supervisorId); + } + + // Stopping gracefully will synchronize the end offsets of the tasks and signal them to publish, and will block + // until the tasks have acknowledged or timed out. We want this behavior when we're explicitly shut down through + // the API, but if we shut down for other reasons (e.g. we lose leadership) we want to just stop and leave the + // tasks as they are. + synchronized (stopLock) { + if (stopGracefully) { + log.info("Posting GracefulShutdownNotice, signalling managed tasks to complete and publish"); + notices.add(new GracefulShutdownNotice()); + } else { + log.info("Posting ShutdownNotice"); + notices.add(new ShutdownNotice()); + } + + long shutdownTimeoutMillis = tuningConfig.getShutdownTimeout().getMillis(); + long endTime = System.currentTimeMillis() + shutdownTimeoutMillis; + while (!stopped) { + long sleepTime = endTime - System.currentTimeMillis(); + if (sleepTime <= 0) { + log.info("Timed out while waiting for shutdown (timeout [%,dms])", shutdownTimeoutMillis); + stopped = true; + break; + } + stopLock.wait(sleepTime); + } + } + log.info("Shutdown notice handled"); + + taskClient.close(); + workerExec.shutdownNow(); + exec.shutdownNow(); + started = false; + + log.info("[%s] has stopped", supervisorId); + } + catch (Exception e) { + log.makeAlert(e, "Exception stopping [%s]", supervisorId) + .emit(); + } + } + } + + + //-------------------------------GOOD---------------------------------------- + @Override + public SupervisorReport getStatus() + { + return generateReport(true); + } + + //-------------------------------GOOD---------------------------------------- + @Override + public void reset(DataSourceMetadata dataSourceMetadata) + { + log.info("Posting ResetNotice"); + notices.add(new ResetNotice(dataSourceMetadata)); + } + + //TODO: checkpoints + @Override + abstract public void checkpoint( + @Nullable Integer taskGroupId, + String baseSequenceName, + DataSourceMetadata previousCheckPoint, + DataSourceMetadata currentCheckPoint + ); + + //-------------------------------GOOD---------------------------------------- + @VisibleForTesting + void runInternal() + throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException + { + possiblyRegisterListener(); + updatePartitionDataFromStream(); + discoverTasks(); + updateTaskStatus(); + checkTaskDuration(); + checkPendingCompletionTasks(); + checkCurrentTaskState(); + createNewTasks(); + + if (log.isDebugEnabled()) { + log.debug(generateReport(true).toString()); + } else { + log.info(generateReport(false).toString()); + } + } + + //-------------------------------GOOD---------------------------------------- + private void possiblyRegisterListener() + { + if (listenerRegistered) { + return; + } + + Optional taskRunner = taskMaster.getTaskRunner(); + if (taskRunner.isPresent()) { + taskRunner.get().registerListener( + new TaskRunnerListener() + { + @Override + public String getListenerId() + { + return supervisorId; + } + + @Override + public void locationChanged(final String taskId, final TaskLocation newLocation) + { + // do nothing + } + + @Override + public void statusChanged(String taskId, TaskStatus status) + { + notices.add(new RunNotice()); + } + }, MoreExecutors.sameThreadExecutor() + ); + + listenerRegistered = true; + } + } + + //-------------------------------GOOD---------------------------------------- + @VisibleForTesting + void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException + { + for (TaskGroup taskGroup : taskGroups.values()) { + for (Map.Entry entry : + taskGroup.tasks.entrySet()) { + if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) { + killTask(entry.getKey()); + } else { + entry.getValue().startTime = new DateTime(0); + } + } + } + + checkTaskDuration(); + } + + //-------------------------------GOOD---------------------------------------- + @VisibleForTesting + void resetInternal(DataSourceMetadata dataSourceMetadata) + { + if (dataSourceMetadata == null) { + // Reset everything + boolean result = indexerMetadataStorageCoordinator.deleteDataSourceMetadata(dataSource); + log.info("Reset dataSource[%s] - dataSource metadata entry deleted? [%s]", dataSource, result); + taskGroups.values().forEach(this::killTasksInGroup); + taskGroups.clear(); + partitionGroups.clear(); + } else { + + checkSourceMetadataInstanceMatch(dataSourceMetadata); + + // Reset only the partitions in dataSourceMetadata if it has not been reset yet + final SeekableStreamDataSourceMetadata resetMetadata = (SeekableStreamDataSourceMetadata) dataSourceMetadata; + + if (resetMetadata.getSeekableStreamPartitions().getId().equals(ioConfig.getId())) { + // metadata can be null + final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); + checkSourceMetadataInstanceMatch(metadata); + + final SeekableStreamDataSourceMetadata currentMetadata = (SeekableStreamDataSourceMetadata) metadata; + + // defend against consecutive reset requests from replicas + // as well as the case where the metadata store do not have an entry for the reset partitions + boolean doReset = false; + for (Map.Entry resetPartitionOffset : resetMetadata.getSeekableStreamPartitions() + .getPartitionSequenceMap() + .entrySet()) { + final T2 partitionOffsetInMetadataStore = currentMetadata == null + ? null + : currentMetadata.getSeekableStreamPartitions() + .getPartitionSequenceMap() + .get(resetPartitionOffset.getKey()); + final TaskGroup partitionTaskGroup = taskGroups.get( + getTaskGroupIdForPartition(resetPartitionOffset.getKey()) + ); + final boolean isSameOffset = partitionTaskGroup != null + && partitionTaskGroup.partitionOffsets.get(resetPartitionOffset.getKey()) + .equals(resetPartitionOffset.getValue()); + if (partitionOffsetInMetadataStore != null || isSameOffset) { + doReset = true; + break; + } + } + + if (!doReset) { + log.info("Ignoring duplicate reset request [%s]", dataSourceMetadata); + return; + } + + boolean metadataUpdateSuccess = false; + if (currentMetadata == null) { + metadataUpdateSuccess = true; + } else { + final DataSourceMetadata newMetadata = currentMetadata.minus(resetMetadata); + try { + metadataUpdateSuccess = indexerMetadataStorageCoordinator.resetDataSourceMetadata(dataSource, newMetadata); + } + catch (IOException e) { + log.error("Resetting DataSourceMetadata failed [%s]", e.getMessage()); + Throwables.propagate(e); + } + } + if (metadataUpdateSuccess) { + resetMetadata.getSeekableStreamPartitions().getPartitionSequenceMap().keySet().forEach(partition -> { + final int groupId = getTaskGroupIdForPartition(partition); + killTaskGroupForPartitions(ImmutableSet.of(partition)); + taskGroups.remove(groupId); + partitionGroups.get(groupId).replaceAll((partitionId, offset) -> NOT_SET); + }); + } else { + throw new ISE("Unable to reset metadata"); + } + } else { + log.warn( + "Reset metadata topic [%s] and supervisor's stream name [%s] do not match", + resetMetadata.getSeekableStreamPartitions().getId(), + ioConfig.getId() + ); + } + } + + + } + + + private void killTask(final String id) + { + Optional taskQueue = taskMaster.getTaskQueue(); + if (taskQueue.isPresent()) { + taskQueue.get().shutdown(id); + } else { + log.error("Failed to get task queue because I'm not the leader!"); + } + } + + private void killTasksInGroup(TaskGroup taskGroup) + { + if (taskGroup != null) { + for (String taskId : taskGroup.tasks.keySet()) { + log.info("Killing task [%s] in the task group", taskId); + killTask(taskId); + } + } + } + + private void killTaskGroupForPartitions(Set partitions) + { + for (T1 partition : partitions) { + killTasksInGroup(taskGroups.get(getTaskGroupIdForPartition(partition))); + } + } + + private boolean isTaskInPendingCompletionGroups(String taskId) + { + for (List taskGroups : pendingCompletionTaskGroups.values()) { + for (TaskGroup taskGroup : taskGroups) { + if (taskGroup.tasks.containsKey(taskId)) { + return true; + } + } + } + return false; + } + + abstract protected int getTaskGroupIdForPartition(T1 partition); + + abstract protected void checkSourceMetadataInstanceMatch(DataSourceMetadata metadata) throws IAE; + + // TODO: may want to put more logic in the base class + abstract protected SupervisorReport generateReport(boolean includeOffsets); + + abstract protected void updatePartitionDataFromStream(); + + private void discoverTasks() throws ExecutionException, InterruptedException, TimeoutException + { + int taskCount = 0; + List futureTaskIds = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + List tasks = taskStorage.getActiveTasks(); + final Map taskGroupsToVerify = new HashMap<>(); + + for (Task task : tasks) { + if (!isTaskInstanceOfThis(task) || !dataSource.equals(task.getDataSource())) { + continue; + } + + taskCount++; + final SeekableStreamIndexTask seekableStreamIndexTask = (SeekableStreamIndexTask) task; + final String taskId = task.getId(); + + // Determine which task group this task belongs to based on one of the partitions handled by this task. If we + // later determine that this task is actively reading, we will make sure that it matches our current partition + // allocation (getTaskGroupIdForPartition(partition) should return the same value for every partition being read + // by this task) and kill it if it is not compatible. If the task is instead found to be in the publishing + // state, we will permit it to complete even if it doesn't match our current partition allocation to support + // seamless schema migration. + + Iterator it = seekableStreamIndexTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceMap() + .keySet() + .iterator(); + final Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null); + + if (taskGroupId != null) { + // check to see if we already know about this task, either in [taskGroups] or in [pendingCompletionTaskGroups] + // and if not add it to taskGroups or pendingCompletionTaskGroups (if status = PUBLISHING) + TaskGroup taskGroup = taskGroups.get(taskGroupId); + + if (!isTaskInPendingCompletionGroups(taskId) && (taskGroup == null || !taskGroup.tasks.containsKey(taskId))) { + futureTaskIds.add(taskId); + futures.add( + Futures.transform( + taskClient.getStatusAsync(taskId), new Function() + { + @Override + public Boolean apply(SeekableStreamIndexTask.Status status) + { + try { + log.debug("Task [%s], status [%s]", taskId, status); + if (status == SeekableStreamIndexTask.Status.PUBLISHING) { + seekableStreamIndexTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceMap() + .keySet() + .forEach( + partition -> addDiscoveredTaskToPendingCompletionTaskGroups( + getTaskGroupIdForPartition(partition), + taskId, + seekableStreamIndexTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceMap() + ) + ); + + // update partitionGroups with the publishing task's offsets (if they are greater than what is + // existing) so that the next tasks will start reading from where this task left off + Map publishingTaskEndOffsets = taskClient.getEndOffsets(taskId); + + for (Map.Entry entry : publishingTaskEndOffsets.entrySet()) { + T1 partition = entry.getKey(); + T2 offset = entry.getValue(); + ConcurrentHashMap partitionOffsets = partitionGroups.get( + getTaskGroupIdForPartition(partition) + ); + + boolean succeeded; + do { + succeeded = true; + T2 previousOffset = partitionOffsets.putIfAbsent(partition, offset); + if (previousOffset != null && (previousOffset.compareTo(offset)) < 0) { + succeeded = partitionOffsets.replace(partition, previousOffset, offset); + } + } while (!succeeded); + } + } else { + for (T1 partition : seekableStreamIndexTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceMap() + .keySet()) { + if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { + log.warn( + "Stopping task [%s] which does not match the expected partition allocation", + taskId + ); + try { + stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + catch (InterruptedException | ExecutionException | TimeoutException e) { + log.warn(e, "Exception while stopping task"); + } + return false; + } + } + // make sure the task's io and tuning configs match with the supervisor config + // if it is current then only create corresponding taskGroup if it does not exist + if (!isTaskCurrent(taskGroupId, taskId)) { + log.info( + "Stopping task [%s] which does not match the expected parameters and ingestion spec", + taskId + ); + try { + stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + catch (InterruptedException | ExecutionException | TimeoutException e) { + log.warn(e, "Exception while stopping task"); + } + return false; + } else { + final TaskGroup taskGroup = taskGroups.computeIfAbsent( + taskGroupId, + k -> { + log.info("Creating a new task group for taskGroupId[%d]", taskGroupId); + return new TaskGroup( + taskGroupId, + ImmutableMap.copyOf( + seekableStreamIndexTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceMap() + ), + seekableStreamIndexTask.getIOConfig().getMinimumMessageTime(), + seekableStreamIndexTask.getIOConfig().getMaximumMessageTime(), + null //TODO: fix + ); + } + ); + taskGroupsToVerify.put(taskGroupId, taskGroup); + final TaskData prevTaskGroup = taskGroup.tasks.putIfAbsent(taskId, new TaskData()); + if (prevTaskGroup != null) { + throw new ISE( + "WTH? a taskGroup[%s] already exists for new task[%s]", + prevTaskGroup, + taskId + ); + } + } + } + return true; + } + catch (Throwable t) { + log.error(t, "Something bad while discovering task [%s]", taskId); + return null; + } + } + + }, workerExec + ) + ); + } + } + } + + } + + private void addDiscoveredTaskToPendingCompletionTaskGroups( + int groupId, + String taskId, + Map startingPartitions + ) + { + final CopyOnWriteArrayList taskGroupList = pendingCompletionTaskGroups.computeIfAbsent( + groupId, + k -> new CopyOnWriteArrayList<>() + ); + for (TaskGroup taskGroup : taskGroupList) { + if (taskGroup.partitionOffsets.equals(startingPartitions)) { + if (taskGroup.tasks.putIfAbsent(taskId, new TaskData()) == null) { + log.info("Added discovered task [%s] to existing pending task group [%s]", taskId, groupId); + } + return; + } + } + + log.info("Creating new pending completion task group [%s] for discovered task [%s]", groupId, taskId); + + // reading the minimumMessageTime & maximumMessageTime from the publishing task and setting it here is not necessary as this task cannot + // change to a state where it will read any more events + TaskGroup newTaskGroup = new TaskGroup( + groupId, + ImmutableMap.copyOf(startingPartitions), + Optional.absent(), + Optional.absent(), + null + ); + + newTaskGroup.tasks.put(taskId, new TaskData()); + newTaskGroup.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout()); + + taskGroupList.add(newTaskGroup); + } + + private ListenableFuture stopTask(final String id, final boolean publish) + { + return Futures.transform( + taskClient.stopAsync(id, publish), new Function() + { + @Nullable + @Override + public Void apply(@Nullable Boolean result) + { + if (result == null || !result) { + log.info("Task [%s] failed to stop in a timely manner, killing task", id); + killTask(id); + } + return null; + } + } + ); + } + + private boolean isTaskCurrent(int taskGroupId, String taskId) + { + Optional taskOptional = taskStorage.getTask(taskId); + if (!taskOptional.isPresent() || !(isTaskInstanceOfThis(taskOptional.get()))) { + return false; + } + + String taskSequenceName = ((SeekableStreamIndexTask) taskOptional.get()).getIOConfig().getBaseSequenceName(); + if (taskGroups.get(taskGroupId) != null) { + return Preconditions + .checkNotNull(taskGroups.get(taskGroupId), "null taskGroup for taskId[%s]", taskGroupId) + .baseSequenceName + .equals(taskSequenceName); + } else { + return generateSequenceName( + ((SeekableStreamIndexTask) taskOptional.get()).getIOConfig() + .getStartPartitions() + .getPartitionSequenceMap(), + ((SeekableStreamIndexTask) taskOptional.get()).getIOConfig().getMinimumMessageTime(), + ((SeekableStreamIndexTask) taskOptional.get()).getIOConfig().getMaximumMessageTime() + ).equals(taskSequenceName); + } + } + + String generateSequenceName( + Map startPartitions, + Optional minimumMessageTime, + Optional maximumMessageTime + ) + { + StringBuilder sb = new StringBuilder(); + + for (Map.Entry entry : startPartitions.entrySet()) { + sb.append(StringUtils.format("+%s(%s)", entry.getKey().toString(), entry.getValue().toString())); + } + String partitionOffsetStr = sb.toString().substring(1); + + String minMsgTimeStr = (minimumMessageTime.isPresent() ? String.valueOf(minimumMessageTime.get().getMillis()) : ""); + String maxMsgTimeStr = (maximumMessageTime.isPresent() ? String.valueOf(maximumMessageTime.get().getMillis()) : ""); + + String dataSchema, tuningConfig; + try { + dataSchema = sortingMapper.writeValueAsString(spec.getDataSchema()); + tuningConfig = sortingMapper.writeValueAsString(taskTuningConfig); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } + + String hashCode = DigestUtils.sha1Hex(dataSchema + + tuningConfig + + partitionOffsetStr + + minMsgTimeStr + + maxMsgTimeStr) + .substring(0, 15); + + return Joiner.on("_").join("index_seekable_streaming", dataSource, hashCode); + } + + abstract protected void updateTaskStatus() throws ExecutionException, InterruptedException, TimeoutException; + + abstract protected void checkTaskDuration() throws ExecutionException, InterruptedException, TimeoutException; + + abstract protected void checkPendingCompletionTasks() + throws ExecutionException, InterruptedException, TimeoutException; + + abstract protected boolean isTaskInstanceOfThis(Task task); + + abstract protected void checkCurrentTaskState() throws ExecutionException, InterruptedException, TimeoutException; + + abstract protected void createNewTasks() throws JsonProcessingException; + + abstract protected RecordSupplier setupRecordSupplier(); + + abstract protected SeekableStreamSupervisorReportPayload createSeekableStreamSupervisorReportPayload(); + + abstract protected void scheduleReporting(); + +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorIOConfig.java new file mode 100644 index 000000000000..e425d5be9475 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorIOConfig.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.SeekableStream.supervisor; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import org.joda.time.Duration; +import org.joda.time.Period; + +abstract public class SeekableStreamSupervisorIOConfig +{ + private final String id; + private final Integer replicas; + private final Integer taskCount; + private final Duration taskDuration; + private final Duration startDelay; + private final Duration period; + private final boolean useEarliestOffset; + private final Duration completionTimeout; + private final Optional lateMessageRejectionPeriod; + private final Optional earlyMessageRejectionPeriod; + + @JsonCreator + public SeekableStreamSupervisorIOConfig( + @JsonProperty("id") String id, + @JsonProperty("replicas") Integer replicas, + @JsonProperty("taskCount") Integer taskCount, + @JsonProperty("taskDuration") Period taskDuration, + @JsonProperty("startDelay") Period startDelay, + @JsonProperty("period") Period period, + @JsonProperty("useEarliestOffset") Boolean useEarliestOffset, + @JsonProperty("completionTimeout") Period completionTimeout, + @JsonProperty("lateMessageRejectionPeriod") Period lateMessageRejectionPeriod, + @JsonProperty("earlyMessageRejectionPeriod") Period earlyMessageRejectionPeriod + ) + { + this.id = Preconditions.checkNotNull(id, "id cannot be null"); + this.replicas = replicas != null ? replicas : 1; + this.taskCount = taskCount != null ? taskCount : 1; + this.taskDuration = defaultDuration(taskDuration, "PT1H"); + this.startDelay = defaultDuration(startDelay, "PT5S"); + this.period = defaultDuration(period, "PT30S"); + this.useEarliestOffset = useEarliestOffset != null ? useEarliestOffset : false; + this.completionTimeout = defaultDuration(completionTimeout, "PT30M"); + this.lateMessageRejectionPeriod = lateMessageRejectionPeriod == null + ? Optional.absent() + : Optional.of(lateMessageRejectionPeriod.toStandardDuration()); + this.earlyMessageRejectionPeriod = earlyMessageRejectionPeriod == null + ? Optional.absent() + : Optional.of(earlyMessageRejectionPeriod.toStandardDuration()); + } + + @JsonProperty + public String getId() + { + return id; + } + + @JsonProperty + public Integer getReplicas() + { + return replicas; + } + + @JsonProperty + public Integer getTaskCount() + { + return taskCount; + } + + @JsonProperty + public Duration getTaskDuration() + { + return taskDuration; + } + + @JsonProperty + public Duration getStartDelay() + { + return startDelay; + } + + @JsonProperty + public Duration getPeriod() + { + return period; + } + + @JsonProperty + public boolean isUseEarliestOffset() + { + return useEarliestOffset; + } + + @JsonProperty + public Duration getCompletionTimeout() + { + return completionTimeout; + } + + @JsonProperty + public Optional getEarlyMessageRejectionPeriod() + { + return earlyMessageRejectionPeriod; + } + + @JsonProperty + public Optional getLateMessageRejectionPeriod() + { + return lateMessageRejectionPeriod; + } + + @Override + abstract public String toString(); + + private static Duration defaultDuration(final Period period, final String theDefault) + { + return (period == null ? new Period(theDefault) : period).toStandardDuration(); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorReportPayload.java new file mode 100644 index 000000000000..05ee3026049a --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.SeekableStream.supervisor; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.IAE; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +abstract public class SeekableStreamSupervisorReportPayload +{ + private final String dataSource; + private final int partitions; + private final int replicas; + private final long durationSeconds; + private final List activeTasks; + private final List publishingTasks; + private final Map latestOffsets; + private final Map minimumLag; + private final Long aggregateLag; + private final DateTime offsetsLastUpdated; + + public SeekableStreamSupervisorReportPayload( + String dataSource, + int partitions, + int replicas, + long durationSeconds, + @Nullable Map latestOffsets, + @Nullable Map minimumLag, + @Nullable Long aggregateLag, + @Nullable DateTime offsetsLastUpdated + ) + { + this.dataSource = dataSource; + this.partitions = partitions; + this.replicas = replicas; + this.durationSeconds = durationSeconds; + this.activeTasks = new ArrayList<>(); + this.publishingTasks = new ArrayList<>(); + this.latestOffsets = latestOffsets; + this.minimumLag = minimumLag; + this.aggregateLag = aggregateLag; + this.offsetsLastUpdated = offsetsLastUpdated; + } + + public void addTask(TaskReportData data) + { + if (data.getType().equals(TaskReportData.TaskType.ACTIVE)) { + activeTasks.add(data); + } else if (data.getType().equals(TaskReportData.TaskType.PUBLISHING)) { + publishingTasks.add(data); + } else { + throw new IAE("Unknown task type [%s]", data.getType().name()); + } + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public int getPartitions() + { + return partitions; + } + + @JsonProperty + public int getReplicas() + { + return replicas; + } + + @JsonProperty + public long getDurationSeconds() + { + return durationSeconds; + } + + @JsonProperty + public List getActiveTasks() + { + return activeTasks; + } + + @JsonProperty + public List getPublishingTasks() + { + return publishingTasks; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Map getLatestOffsets() + { + return latestOffsets; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Map getMinimumLag() + { + return minimumLag; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Long getAggregateLag() + { + return aggregateLag; + } + + @JsonProperty + public DateTime getOffsetsLastUpdated() + { + return offsetsLastUpdated; + } + + @Override + abstract public String toString(); +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorSpec.java new file mode 100644 index 000000000000..830d03d175aa --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorSpec.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.SeekableStream.supervisor; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.Supervisor; +import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; + +import java.util.List; +import java.util.Map; + +public class SeekableStreamSupervisorSpec implements SupervisorSpec +{ + private final DataSchema dataSchema; + private final SeekableStreamSupervisorTuningConfig tuningConfig; + private final SeekableStreamSupervisorIOConfig ioConfig; + private final Map context; + + private final TaskStorage taskStorage; + private final TaskMaster taskMaster; + private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; + private final IndexTaskClientFactory indexTaskClientFactory; + private final ObjectMapper mapper; + private final ServiceEmitter emitter; + private final DruidMonitorSchedulerConfig monitorSchedulerConfig; + private final RowIngestionMetersFactory rowIngestionMetersFactory; + + @JsonCreator + public SeekableStreamSupervisorSpec( + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("tuningConfig") SeekableStreamSupervisorTuningConfig tuningConfig, + @JsonProperty("ioConfig") SeekableStreamSupervisorIOConfig ioConfig, + @JsonProperty("context") Map context, + @JacksonInject TaskStorage taskStorage, + @JacksonInject TaskMaster taskMaster, + @JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + @JacksonInject IndexTaskClientFactory indexTaskClientFactory, + @JacksonInject @Json ObjectMapper mapper, + @JacksonInject ServiceEmitter emitter, + @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + ) + { + this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); + this.tuningConfig = tuningConfig != null + ? tuningConfig + : new SeekableStreamSupervisorTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); + this.context = context; + + this.taskStorage = taskStorage; + this.taskMaster = taskMaster; + this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; + this.indexTaskClientFactory = indexTaskClientFactory; + this.mapper = mapper; + this.emitter = emitter; + this.monitorSchedulerConfig = monitorSchedulerConfig; + this.rowIngestionMetersFactory = rowIngestionMetersFactory; + } + + @JsonProperty + public DataSchema getDataSchema() + { + return dataSchema; + } + + @JsonProperty + public SeekableStreamSupervisorTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @JsonProperty + public SeekableStreamSupervisorIOConfig getIoConfig() + { + return ioConfig; + } + + @JsonProperty + public Map getContext() + { + return context; + } + + public ServiceEmitter getEmitter() + { + return emitter; + } + + @Override + public String getId() + { + return dataSchema.getDataSource(); + } + + public DruidMonitorSchedulerConfig getMonitorSchedulerConfig() + { + return monitorSchedulerConfig; + } + + @Override + public Supervisor createSupervisor() + { + return new SeekableStreamSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + indexTaskClientFactory, + mapper, + this, + rowIngestionMetersFactory + ); + } + + @Override + public List getDataSources() + { + return ImmutableList.of(getDataSchema().getDataSource()); + } + + @Override + public String toString() + { + return "SeekableStreamSupervisorSpec{" + + "dataSchema=" + dataSchema + + ", tuningConfig=" + tuningConfig + + ", ioConfig=" + ioConfig + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorTuningConfig.java new file mode 100644 index 000000000000..591a1bbf2c83 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorTuningConfig.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.SeekableStream.supervisor; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.SeekableStream.SeekableStreamTuningConfig; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.indexing.TuningConfigs; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.joda.time.Duration; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import java.io.File; + +public class SeekableStreamSupervisorTuningConfig extends SeekableStreamTuningConfig +{ + private final Integer workerThreads; + private final Integer chatThreads; + private final Long chatRetries; + private final Duration httpTimeout; + private final Duration shutdownTimeout; + private final Duration offsetFetchPeriod; + + public SeekableStreamSupervisorTuningConfig( + @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, + @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, + @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, + @JsonProperty("basePersistDirectory") File basePersistDirectory, + @JsonProperty("maxPendingPersists") Integer maxPendingPersists, + @JsonProperty("indexSpec") IndexSpec indexSpec, + // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. + @JsonProperty("buildV9Directly") Boolean buildV9Directly, + @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, + @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, + @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("workerThreads") Integer workerThreads, + @JsonProperty("chatThreads") Integer chatThreads, + @JsonProperty("chatRetries") Long chatRetries, + @JsonProperty("httpTimeout") Period httpTimeout, + @JsonProperty("shutdownTimeout") Period shutdownTimeout, + @JsonProperty("offsetFetchPeriod") Period offsetFetchPeriod, + @JsonProperty("intermediateHandoffPeriod") Period intermediateHandoffPeriod, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions + ) + { + super( + maxRowsInMemory, + maxBytesInMemory, + maxRowsPerSegment, + intermediatePersistPeriod, + basePersistDirectory, + maxPendingPersists, + indexSpec, + true, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + segmentWriteOutMediumFactory, + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions + ); + + this.workerThreads = workerThreads; + this.chatThreads = chatThreads; + this.chatRetries = (chatRetries != null ? chatRetries : 8); + this.httpTimeout = defaultDuration(httpTimeout, "PT10S"); + this.shutdownTimeout = defaultDuration(shutdownTimeout, "PT80S"); + this.offsetFetchPeriod = defaultDuration(offsetFetchPeriod, "PT30S"); + } + + @JsonProperty + public Integer getWorkerThreads() + { + return workerThreads; + } + + @JsonProperty + public Integer getChatThreads() + { + return chatThreads; + } + + @JsonProperty + public Long getChatRetries() + { + return chatRetries; + } + + @JsonProperty + public Duration getHttpTimeout() + { + return httpTimeout; + } + + @JsonProperty + public Duration getShutdownTimeout() + { + return shutdownTimeout; + } + + @JsonProperty + public Duration getOffsetFetchPeriod() + { + return offsetFetchPeriod; + } + + @Override + public String toString() + { + return "SeekableStreamSupervisorTuningConfig{" + + "maxRowsInMemory=" + getMaxRowsInMemory() + + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + + ", maxBytesInMemory=" + TuningConfigs.getMaxBytesInMemoryOrDefault(getMaxBytesInMemory()) + + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + + ", basePersistDirectory=" + getBasePersistDirectory() + + ", maxPendingPersists=" + getMaxPendingPersists() + + ", indexSpec=" + getIndexSpec() + + ", reportParseExceptions=" + isReportParseExceptions() + + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + + ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() + + ", workerThreads=" + workerThreads + + ", chatThreads=" + chatThreads + + ", chatRetries=" + chatRetries + + ", httpTimeout=" + httpTimeout + + ", shutdownTimeout=" + shutdownTimeout + + ", offsetFetchPeriod=" + offsetFetchPeriod + + ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() + + ", logParseExceptions=" + isLogParseExceptions() + + ", maxParseExceptions=" + getMaxParseExceptions() + + ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + + '}'; + } + + private static Duration defaultDuration(final Period period, final String theDefault) + { + return (period == null ? new Period(theDefault) : period).toStandardDuration(); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/TaskReportData.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/TaskReportData.java new file mode 100644 index 000000000000..8c0e84c2f8ee --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/TaskReportData.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.SeekableStream.supervisor; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.util.Map; + +public class TaskReportData +{ + private final String id; + private final Map startingOffsets; + private final DateTime startTime; + private final Long remainingSeconds; + private final TaskType type; + private final Map currentOffsets; + private final Map lag; + + public TaskReportData( + String id, + @Nullable Map startingOffsets, + @Nullable Map currentOffsets, + @Nullable DateTime startTime, + Long remainingSeconds, + TaskType type, + @Nullable Map lag + ) + { + this.id = id; + this.startingOffsets = startingOffsets; + this.currentOffsets = currentOffsets; + this.startTime = startTime; + this.remainingSeconds = remainingSeconds; + this.type = type; + this.lag = lag; + } + + @JsonProperty + public String getId() + { + return id; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Map getStartingOffsets() + { + return startingOffsets; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Map getCurrentOffsets() + { + return currentOffsets; + } + + @JsonProperty + public DateTime getStartTime() + { + return startTime; + } + + @JsonProperty + public Long getRemainingSeconds() + { + return remainingSeconds; + } + + @JsonProperty + public TaskType getType() + { + return type; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Map getLag() + { + return lag; + } + + @Override + public String toString() + { + return "{" + + "id='" + id + '\'' + + (startingOffsets != null ? ", startingOffsets=" + startingOffsets : "") + + (currentOffsets != null ? ", currentOffsets=" + currentOffsets : "") + + ", startTime=" + startTime + + ", remainingSeconds=" + remainingSeconds + + (lag != null ? ", lag=" + lag : "") + + '}'; + } + + public enum TaskType + { + ACTIVE, PUBLISHING, UNKNOWN + } +} From 1c99483a97e2b69b0d2b6a19458a8857534105b0 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 12 Sep 2018 13:07:28 -0700 Subject: [PATCH 02/87] created seekablestreamsupervisor class --- .../SeekableStreamIOConfig.java | 1 - .../SeekableStreamIndexTask.java | 14 +- .../SeekableStreamIndexTaskClient.java | 99 ++- .../SeekableStreamIndexTaskClientFactory.java | 15 +- .../SeekableStreamIndexTaskRunner.java | 64 ++ .../SeekableStreamPartitions.java | 2 - .../SeekableStreamTuningConfig.java | 68 +- .../supervisor/SeekableStreamSupervisor.java | 735 +++++++++++++++++- .../SeekableStreamSupervisorIOConfig.java | 34 +- ...SeekableStreamSupervisorReportPayload.java | 36 +- .../SeekableStreamSupervisorSpec.java | 71 +- .../SeekableStreamSupervisorTuningConfig.java | 129 +-- ...java => SeekableStreamTaskReportData.java} | 49 +- 13 files changed, 975 insertions(+), 342 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskRunner.java rename indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/{TaskReportData.java => SeekableStreamTaskReportData.java} (67%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIOConfig.java index 861bb8ae87bb..48586b9b3d8e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIOConfig.java @@ -31,7 +31,6 @@ abstract public class SeekableStreamIOConfig implements IOConfig { private static final boolean DEFAULT_USE_TRANSACTION = true; - private static final boolean DEFAULT_SKIP_OFFSET_GAPS = false; @Nullable private final Integer taskGroupId; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTask.java index 554bc706a9f6..d69a8a0bd18c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTask.java @@ -47,6 +47,7 @@ import java.util.Map; import java.util.Random; +//TODO: need more refactoring for run() abstract public class SeekableStreamIndexTask extends AbstractTask implements ChatHandler { public enum Status @@ -59,14 +60,15 @@ public enum Status } private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTask.class); - private static final String TYPE = "index_seekable_stream"; private static final Random RANDOM = new Random(); + private static final String TYPE = "index_seekable_stream"; + - private final DataSchema dataSchema; - private final InputRowParser parser; - private final SeekableStreamTuningConfig tuningConfig; - private final SeekableStreamIOConfig ioConfig; - private final Optional chatHandlerProvider; + protected final DataSchema dataSchema; + protected final InputRowParser parser; + protected final SeekableStreamTuningConfig tuningConfig; + protected final SeekableStreamIOConfig ioConfig; + protected final Optional chatHandlerProvider; @JsonCreator public SeekableStreamIndexTask( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClient.java index 9f8dbc5425b9..ccb4f3e2037a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClient.java @@ -25,12 +25,14 @@ import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.indexing.common.IndexTaskClient; import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.response.FullResponseHolder; import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -40,7 +42,6 @@ import java.util.Map; import java.util.TreeMap; -//TODO: Need to refactor implemented methods abstract public class SeekableStreamIndexTaskClient extends IndexTaskClient { private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTaskClient.class); @@ -96,10 +97,73 @@ public boolean resume(final String id) } - abstract public Map pause(final String id); + public Map pause(final String id) + { + log.debug("Pause task[%s]", id); + + try { + final FullResponseHolder response = submitRequestWithEmptyContent( + id, + HttpMethod.POST, + "pause", + null, + true + ); + + if (response.getStatus().equals(HttpResponseStatus.OK)) { + log.info("Task [%s] paused successfully", id); + return deserialize(response.getContent(), new TypeReference>() + { + }); + } + + while (true) { + if (getStatus(id) == SeekableStreamIndexTask.Status.PAUSED) { + return getCurrentOffsets(id, true); + } + + final Duration delay = newRetryPolicy().getAndIncrementRetryDelay(); + if (delay == null) { + log.error("Task [%s] failed to pause, aborting", id); + throw new ISE("Task [%s] failed to pause, aborting", id); + } else { + final long sleepTime = delay.getMillis(); + log.info( + "Still waiting for task [%s] to pause; will try again in [%s]", + id, + new Duration(sleepTime).toString() + ); + Thread.sleep(sleepTime); + } + } + } + catch (NoTaskLocationException e) { + log.error("Exception [%s] while pausing Task [%s]", e.getMessage(), id); + return ImmutableMap.of(); + } + catch (IOException | InterruptedException e) { + throw new RuntimeException( + StringUtils.format("Exception [%s] while pausing Task [%s]", e.getMessage(), id), + e + ); + } + } + public SeekableStreamIndexTask.Status getStatus(final String id) + { + log.debug("GetStatus task[%s]", id); - abstract public SeekableStreamIndexTask.Status getStatus(final String id); + try { + final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "status", null, true); + return deserialize(response.getContent(), SeekableStreamIndexTask.Status.class); + } + catch (NoTaskLocationException e) { + return SeekableStreamIndexTask.Status.NOT_STARTED; + } + catch (IOException e) { + throw new RuntimeException(e); + } + } @Nullable @@ -244,6 +308,13 @@ public ListenableFuture stopAsync(final String id, final boolean publis return doAsync(() -> stop(id, publish)); } + + public ListenableFuture resumeAsync(final String id) + { + return doAsync(() -> resume(id)); + } + + public ListenableFuture getStartTimeAsync(final String id) { return doAsync(() -> getStartTime(id)); @@ -264,7 +335,27 @@ public ListenableFuture setEndOffsetsAsync( return doAsync(() -> setEndOffsets(id, endOffsets, finalize)); } - abstract public ListenableFuture getStatusAsync(final String id); + public ListenableFuture> getCurrentOffsetsAsync(final String id, final boolean retry) + { + return doAsync(() -> getCurrentOffsets(id, retry)); + } + + public ListenableFuture> getEndOffsetsAsync(final String id) + { + return doAsync(() -> getEndOffsets(id)); + } + + + public ListenableFuture> getMovingAveragesAsync(final String id) + { + return doAsync(() -> getMovingAverages(id)); + } + + + public ListenableFuture getStatusAsync(final String id) + { + return doAsync(() -> getStatus(id)); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClientFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClientFactory.java index 4ec028558233..313c4de80d6a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClientFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClientFactory.java @@ -28,8 +28,8 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; -abstract public class SeekableStreamIndexTaskClientFactory - implements IndexTaskClientFactory +abstract public class SeekableStreamIndexTaskClientFactory + implements IndexTaskClientFactory { private HttpClient httpClient; private ObjectMapper mapper; @@ -45,7 +45,7 @@ public SeekableStreamIndexTaskClientFactory( } @Override - abstract public SeekableStreamIndexTaskClient build( + abstract public T build( TaskInfoProvider taskInfoProvider, String dataSource, int numThreads, @@ -53,4 +53,13 @@ abstract public SeekableStreamIndexTaskClient build( long numRetries ); + protected HttpClient getHttpClient() + { + return httpClient; + } + + protected ObjectMapper getMapper() + { + return mapper; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskRunner.java new file mode 100644 index 000000000000..621af0d8819e --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskRunner.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.druid.indexing.SeekableStream; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.stats.RowIngestionMeters; +import org.apache.druid.segment.realtime.appenderator.Appenderator; +import org.apache.druid.segment.realtime.firehose.ChatHandler; + +import javax.ws.rs.core.Response; +import java.util.Map; + +public interface SeekableStreamIndexTaskRunner extends ChatHandler +{ + Appenderator getAppenderator(); + + TaskStatus run(TaskToolbox toolbox); + + void stopGracefully(); + + // The below methods are mostly for unit testing. + + @VisibleForTesting + RowIngestionMeters getRowIngestionMeters(); + + @VisibleForTesting + SeekableStreamIndexTask.Status getStatus(); + + @VisibleForTesting + Map getCurrentOffsets(); + + @VisibleForTesting + Map getEndOffsets(); + + @VisibleForTesting + Response setEndOffsets( + Map offsets, + boolean finish // this field is only for internal purposes, shouldn't be usually set by users + ) throws InterruptedException; + + @VisibleForTesting + Response pause() throws InterruptedException; + + @VisibleForTesting + void resume() throws InterruptedException; +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamPartitions.java index 06331ecfb4f3..6821b6fdbcdd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamPartitions.java @@ -41,13 +41,11 @@ public SeekableStreamPartitions( this.partitionOffsetMap = ImmutableMap.copyOf(partitionOffsetMap); } - @JsonProperty public String getId() { return id; } - @JsonProperty public Map getPartitionSequenceMap() { return partitionOffsetMap; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamTuningConfig.java index 6a52448af5f9..0012ad9d9884 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamTuningConfig.java @@ -32,7 +32,7 @@ import java.io.File; import java.util.Objects; -public class SeekableStreamTuningConfig implements TuningConfig, AppenderatorConfig +abstract public class SeekableStreamTuningConfig implements TuningConfig, AppenderatorConfig { private static final int DEFAULT_MAX_ROWS_PER_SEGMENT = 5_000_000; private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false; @@ -121,27 +121,7 @@ public SeekableStreamTuningConfig( : logParseExceptions; } - public static SeekableStreamTuningConfig copyOf(SeekableStreamTuningConfig config) - { - return new SeekableStreamTuningConfig( - config.maxRowsInMemory, - config.maxBytesInMemory, - config.maxRowsPerSegment, - config.intermediatePersistPeriod, - config.basePersistDirectory, - config.maxPendingPersists, - config.indexSpec, - true, - config.reportParseExceptions, - config.handoffConditionTimeout, - config.resetOffsetAutomatically, - config.segmentWriteOutMediumFactory, - config.intermediateHandoffPeriod, - config.logParseExceptions, - config.maxParseExceptions, - config.maxSavedParseExceptions - ); - } + abstract public SeekableStreamTuningConfig copyOf(); @Override @JsonProperty @@ -253,27 +233,7 @@ public int getMaxSavedParseExceptions() return maxSavedParseExceptions; } - public SeekableStreamTuningConfig withBasePersistDirectory(File dir) - { - return new SeekableStreamTuningConfig( - maxRowsInMemory, - maxBytesInMemory, - maxRowsPerSegment, - intermediatePersistPeriod, - dir, - maxPendingPersists, - indexSpec, - true, - reportParseExceptions, - handoffConditionTimeout, - resetOffsetAutomatically, - segmentWriteOutMediumFactory, - intermediateHandoffPeriod, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions - ); - } + abstract public SeekableStreamTuningConfig withBasePersistDirectory(File dir); @Override public boolean equals(Object o) @@ -325,24 +285,6 @@ public int hashCode() } @Override - public String toString() - { - return "SeekableTuningConfig{" + - "maxRowsInMemory=" + maxRowsInMemory + - ", maxRowsPerSegment=" + maxRowsPerSegment + - ", maxBytesInMemory=" + maxBytesInMemory + - ", intermediatePersistPeriod=" + intermediatePersistPeriod + - ", basePersistDirectory=" + basePersistDirectory + - ", maxPendingPersists=" + maxPendingPersists + - ", indexSpec=" + indexSpec + - ", reportParseExceptions=" + reportParseExceptions + - ", handoffConditionTimeout=" + handoffConditionTimeout + - ", resetOffsetAutomatically=" + resetOffsetAutomatically + - ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + - ", intermediateHandoffPeriod=" + intermediateHandoffPeriod + - ", logParseExceptions=" + logParseExceptions + - ", maxParseExceptions=" + maxParseExceptions + - ", maxSavedParseExceptions=" + maxSavedParseExceptions + - '}'; - } + abstract public String toString(); + } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisor.java index 17fd0af58b33..3e6b2bbb38de 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisor.java @@ -10,6 +10,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; @@ -44,6 +45,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -53,6 +55,8 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -60,6 +64,7 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.SortedMap; import java.util.TreeMap; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; @@ -70,7 +75,14 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; - +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +//TODO: rename offset -> sequence +//TODO: prune 'kafka' and 'kinesis' +//TODO: inheritance +//TODO: make classes abstract +//TODO: resolve warnings abstract public class SeekableStreamSupervisor implements Supervisor { //---------------------------------------GOOD--------------------------------------------------------- @@ -78,6 +90,7 @@ abstract public class SeekableStreamSupervisor tasks = new ConcurrentHashMap<>(); final Optional minimumMessageTime; final Optional maximumMessageTime; - final Set exclusiveStartSequenceNumberPartitions; - final TreeMap> sequenceOffsets = new TreeMap<>(); + final Set exclusiveStartSequenceNumberPartitions; //TODO: exclusiveSequence + final TreeMap> sequenceOffsets = new TreeMap<>(); final String baseSequenceName; DateTime completionTimeout; @@ -114,7 +127,7 @@ public TaskGroup( this.baseSequenceName = generateSequenceName(partitionOffsets, minimumMessageTime, maximumMessageTime); } - int addNewCheckpoint(Map checkpoint) + int addNewCheckpoint(Map checkpoint) { sequenceOffsets.put(sequenceOffsets.lastKey() + 1, checkpoint); return sequenceOffsets.lastKey(); @@ -172,7 +185,7 @@ public String toString() private final BlockingQueue notices = new LinkedBlockingDeque<>(); private final Object stopLock = new Object(); private final Object stateChangeLock = new Object(); - private final Object consumerLock = new Object(); + private final Object consumerLock = new Object(); //TODO: prob not needed private boolean listenerRegistered = false; private long lastRunTime; @@ -276,7 +289,7 @@ public SeekableStreamSupervisor( this.dataSource = spec.getDataSchema().getDataSource(); this.ioConfig = spec.getIoConfig(); this.tuningConfig = spec.getTuningConfig(); - this.taskTuningConfig = SeekableStreamTuningConfig.copyOf(this.tuningConfig); + this.taskTuningConfig = this.tuningConfig.copyOf(); this.supervisorId = supervisorId; this.exec = Execs.singleThreaded(supervisorId); this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d"); @@ -379,7 +392,7 @@ public void start() ); firstRunTime = DateTimes.nowUtc().plus(ioConfig.getStartDelay()); scheduledExec.scheduleAtFixedRate( - () -> notices.add(new RunNotice()), + buildRunTask(), ioConfig.getStartDelay().getMillis(), Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS), TimeUnit.MILLISECONDS @@ -698,14 +711,6 @@ private boolean isTaskInPendingCompletionGroups(String taskId) return false; } - abstract protected int getTaskGroupIdForPartition(T1 partition); - - abstract protected void checkSourceMetadataInstanceMatch(DataSourceMetadata metadata) throws IAE; - - // TODO: may want to put more logic in the base class - abstract protected SupervisorReport generateReport(boolean includeOffsets); - - abstract protected void updatePartitionDataFromStream(); private void discoverTasks() throws ExecutionException, InterruptedException, TimeoutException { @@ -866,6 +871,187 @@ public Boolean apply(SeekableStreamIndexTask.Status status) } } + + List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + for (int i = 0; i < results.size(); i++) { + if (results.get(i) == null) { + String taskId = futureTaskIds.get(i); + log.warn("Task [%s] failed to return status, killing task", taskId); + killTask(taskId); + } + } + log.debug("Found [%d] Kafka indexing tasks for dataSource [%s]", taskCount, dataSource); + + // make sure the checkpoints are consistent with each other and with the metadata store + verifyAndMergeCheckpoints(taskGroupsToVerify.values()); + + } + + private void verifyAndMergeCheckpoints(final Collection taskGroupsToVerify) + { + final List> futures = new ArrayList<>(); + for (TaskGroup taskGroup : taskGroupsToVerify) { + futures.add(workerExec.submit(() -> { + verifyAndMergeCheckpoints(taskGroup); + return true; + })); + } + try { + Futures.allAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + catch (InterruptedException | ExecutionException | TimeoutException e) { + throw new RuntimeException(e); + } + } + + //TODO: prob wanna refactor this + + /** + * This method does two things - + * 1. Makes sure the checkpoints information in the taskGroup is consistent with that of the tasks, if not kill + * inconsistent tasks. + * 2. truncates the checkpoints in the taskGroup corresponding to which segments have been published, so that any newly + * created tasks for the taskGroup start indexing from after the latest published offsets. + */ + private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) + { + final int groupId = taskGroup.groupId; + final List>>> taskSequences = new ArrayList<>(); + final List>>> futures = new ArrayList<>(); + final List taskIds = new ArrayList<>(); + + for (String taskId : taskGroup.taskIds()) { + final ListenableFuture>> checkpointsFuture = taskClient.getCheckpointsAsync( + taskId, + true + ); + futures.add(checkpointsFuture); + taskIds.add(taskId); + } + + try { + List>> futuresResult = + Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + for (int i = 0; i < futuresResult.size(); i++) { + final TreeMap> checkpoints = futuresResult.get(i); + final String taskId = taskIds.get(i); + if (checkpoints == null) { + try { + futures.get(i).get(1, TimeUnit.NANOSECONDS); + } + catch (Exception e) { + log.error(e, "Problem while getting checkpoints for task [%s], killing the task", taskId); + killTask(taskId); + taskGroup.tasks.remove(taskId); + } + } else if (checkpoints.isEmpty()) { + log.warn("Ignoring task [%s], as probably it is not started running yet", taskId); + } else { + taskSequences.add(new Pair<>(taskId, checkpoints)); + } + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + + final SeekableStreamDataSourceMetadata latestDataSourceMetadata = (SeekableStreamDataSourceMetadata) indexerMetadataStorageCoordinator + .getDataSourceMetadata(dataSource); + final boolean hasValidOffsetsFromDb = latestDataSourceMetadata != null && + latestDataSourceMetadata.getSeekableStreamPartitions() != null && + ioConfig.getId().equals( + latestDataSourceMetadata.getSeekableStreamPartitions().getId() + ); + final Map latestOffsetsFromDb; + if (hasValidOffsetsFromDb) { + latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamPartitions().getPartitionSequenceMap(); + } else { + latestOffsetsFromDb = null; + } + + // order tasks of this taskGroup by the latest sequenceId + taskSequences.sort((o1, o2) -> o2.rhs.firstKey().compareTo(o1.rhs.firstKey())); + + final Set tasksToKill = new HashSet<>(); + final AtomicInteger earliestConsistentSequenceId = new AtomicInteger(-1); + int taskIndex = 0; + + while (taskIndex < taskSequences.size()) { + TreeMap> taskCheckpoints = taskSequences.get(taskIndex).rhs; + String taskId = taskSequences.get(taskIndex).lhs; + if (earliestConsistentSequenceId.get() == -1) { + // find the first replica task with earliest sequenceId consistent with datasource metadata in the metadata + // store + if (taskCheckpoints.entrySet().stream().anyMatch( + sequenceCheckpoint -> sequenceCheckpoint.getValue().entrySet().stream().allMatch( + partitionOffset -> partitionOffset.getValue().compareTo(latestOffsetsFromDb == null ? + partitionOffset.getValue() : + latestOffsetsFromDb.getOrDefault( + partitionOffset + .getKey(), + partitionOffset + .getValue() + )) == 0) + && earliestConsistentSequenceId.compareAndSet(-1, sequenceCheckpoint.getKey())) || ( + pendingCompletionTaskGroups.getOrDefault(groupId, EMPTY_LIST).size() > 0 + && earliestConsistentSequenceId.compareAndSet(-1, taskCheckpoints.firstKey()))) { + final SortedMap> latestCheckpoints = new TreeMap<>( + taskCheckpoints.tailMap(earliestConsistentSequenceId.get()) + ); + log.info("Setting taskGroup sequences to [%s] for group [%d]", latestCheckpoints, groupId); + taskGroup.sequenceOffsets.clear(); + taskGroup.sequenceOffsets.putAll(latestCheckpoints); + } else { + log.debug( + "Adding task [%s] to kill list, checkpoints[%s], latestoffsets from DB [%s]", + taskId, + taskCheckpoints, + latestOffsetsFromDb + ); + tasksToKill.add(taskId); + } + } else { + // check consistency with taskGroup sequences + if (taskCheckpoints.get(taskGroup.sequenceOffsets.firstKey()) == null + || !(taskCheckpoints.get(taskGroup.sequenceOffsets.firstKey()) + .equals(taskGroup.sequenceOffsets.firstEntry().getValue())) + || taskCheckpoints.tailMap(taskGroup.sequenceOffsets.firstKey()).size() + != taskGroup.sequenceOffsets.size()) { + log.debug( + "Adding task [%s] to kill list, checkpoints[%s], taskgroup checkpoints [%s]", + taskId, + taskCheckpoints, + taskGroup.sequenceOffsets + ); + tasksToKill.add(taskId); + } + } + taskIndex++; + } + + if ((tasksToKill.size() > 0 && tasksToKill.size() == taskGroup.tasks.size()) || + (taskGroup.tasks.size() == 0 && pendingCompletionTaskGroups.getOrDefault(groupId, EMPTY_LIST).size() == 0)) { + // 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 + log.warn("Clearing task group [%d] information as no valid tasks left the group", groupId); + taskGroups.remove(groupId); + partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET); + } + + taskSequences.stream().filter(taskIdSequences -> tasksToKill.contains(taskIdSequences.lhs)).forEach( + sequenceCheckpoint -> { + log.warn( + "Killing task [%s], as its checkpoints [%s] are not consistent with group checkpoints[%s] or latest " + + "persisted offsets in metadata store [%s]", + sequenceCheckpoint.lhs, + sequenceCheckpoint.rhs, + taskGroup.sequenceOffsets, + latestOffsetsFromDb + ); + killTask(sequenceCheckpoint.lhs); + taskGroup.tasks.remove(sequenceCheckpoint.lhs); + } + ); } private void addDiscoveredTaskToPendingCompletionTaskGroups( @@ -983,18 +1169,514 @@ String generateSequenceName( return Joiner.on("_").join("index_seekable_streaming", dataSource, hashCode); } - abstract protected void updateTaskStatus() throws ExecutionException, InterruptedException, TimeoutException; + // TODO: refactor this with recordSupplier + abstract protected void updatePartitionDataFromStream(); - abstract protected void checkTaskDuration() throws ExecutionException, InterruptedException, TimeoutException; + private void updateTaskStatus() throws ExecutionException, InterruptedException, TimeoutException + { + final List> futures = Lists.newArrayList(); + final List futureTaskIds = Lists.newArrayList(); - abstract protected void checkPendingCompletionTasks() - throws ExecutionException, InterruptedException, TimeoutException; + // update status (and startTime if unknown) of current tasks in taskGroups + for (TaskGroup group : taskGroups.values()) { + for (Map.Entry entry : group.tasks.entrySet()) { + final String taskId = entry.getKey(); + final TaskData taskData = entry.getValue(); - abstract protected boolean isTaskInstanceOfThis(Task task); + if (taskData.startTime == null) { + futureTaskIds.add(taskId); + futures.add( + Futures.transform( + taskClient.getStartTimeAsync(taskId), new Function() + { + @Nullable + @Override + public Boolean apply(@Nullable DateTime startTime) + { + if (startTime == null) { + return false; + } + + taskData.startTime = startTime; + long millisRemaining = ioConfig.getTaskDuration().getMillis() - + (System.currentTimeMillis() - taskData.startTime.getMillis()); + if (millisRemaining > 0) { + scheduledExec.schedule( + buildRunTask(), + millisRemaining + MAX_RUN_FREQUENCY_MILLIS, + TimeUnit.MILLISECONDS + ); + } + + return true; + } + }, workerExec + ) + ); + } - abstract protected void checkCurrentTaskState() throws ExecutionException, InterruptedException, TimeoutException; + taskData.status = taskStorage.getStatus(taskId).get(); + } + } - abstract protected void createNewTasks() throws JsonProcessingException; + // update status of pending completion tasks in pendingCompletionTaskGroups + for (List taskGroups : pendingCompletionTaskGroups.values()) { + for (TaskGroup group : taskGroups) { + for (Map.Entry entry : group.tasks.entrySet()) { + entry.getValue().status = taskStorage.getStatus(entry.getKey()).get(); + } + } + } + + List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + for (int i = 0; i < results.size(); i++) { + // false means the task hasn't started running yet and that's okay; null means it should be running but the HTTP + // request threw an exception so kill the task + if (results.get(i) == null) { + String taskId = futureTaskIds.get(i); + log.warn("Task [%s] failed to return start time, killing task", taskId); + killTask(taskId); + } + } + } + + private Runnable buildRunTask() + { + return () -> notices.add(new RunNotice()); + } + + private void checkTaskDuration() throws ExecutionException, InterruptedException, TimeoutException + { + final List>> futures = Lists.newArrayList(); + final List futureGroupIds = Lists.newArrayList(); + + for (Map.Entry entry : taskGroups.entrySet()) { + Integer groupId = entry.getKey(); + TaskGroup group = entry.getValue(); + + // find the longest running task from this group + DateTime earliestTaskStart = DateTimes.nowUtc(); + for (TaskData taskData : group.tasks.values()) { + // startTime can be null if kafkaSupervisor is stopped gracefully before processing any runNotice + if (taskData.startTime != null && earliestTaskStart.isAfter(taskData.startTime)) { + earliestTaskStart = taskData.startTime; + } + } + + // TODO: early publish time + + // if this task has run longer than the configured duration, signal all tasks in the group to persist + if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { + log.info("Task group [%d] has run for [%s]", groupId, ioConfig.getTaskDuration()); + futureGroupIds.add(groupId); + futures.add(checkpointTaskGroup(group, true)); + } + } + + List> results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + for (int j = 0; j < results.size(); j++) { + Integer groupId = futureGroupIds.get(j); + TaskGroup group = taskGroups.get(groupId); + Map endOffsets = results.get(j); + + if (endOffsets != null) { + // set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion + group.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout()); + pendingCompletionTaskGroups.computeIfAbsent(groupId, k -> new CopyOnWriteArrayList<>()).add(group); + + // set endOffsets as the next startOffsets + for (Map.Entry entry : endOffsets.entrySet()) { + partitionGroups.get(groupId).put(entry.getKey(), entry.getValue()); + } + } else { + log.warn( + "All tasks in group [%s] failed to transition to publishing state, killing tasks [%s]", + groupId, + group.taskIds() + ); + for (String id : group.taskIds()) { + killTask(id); + } + // clear partitionGroups, so that latest offsets from db is used as start offsets not the stale ones + // if tasks did some successful incremental handoffs + partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET); + } + + // remove this task group from the list of current task groups now that it has been handled + taskGroups.remove(groupId); + } + } + + private ListenableFuture> checkpointTaskGroup(final TaskGroup taskGroup, final boolean finalize) + { + if (finalize) { + // 1) Check if any task completed (in which case we're done) and kill unassigned tasks + Iterator> i = taskGroup.tasks.entrySet().iterator(); + while (i.hasNext()) { + Map.Entry taskEntry = i.next(); + String taskId = taskEntry.getKey(); + TaskData task = taskEntry.getValue(); + + // task.status can be null if kafkaSupervisor is stopped gracefully before processing any runNotice. + if (task.status != null) { + if (task.status.isSuccess()) { + // If any task in this group has already completed, stop the rest of the tasks in the group and return. + // This will cause us to create a new set of tasks next cycle that will start from the offsets in + // 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) + return Futures.transform( + stopTasksInGroup(taskGroup), + new Function>() + { + @Nullable + @Override + public Map apply(@Nullable Object input) + { + return null; + } + } + ); + } + + if (task.status.isRunnable()) { + if (taskInfoProvider.getTaskLocation(taskId).equals(TaskLocation.unknown())) { + log.info("Killing task [%s] which hasn't been assigned to a worker", taskId); + killTask(taskId); + i.remove(); + } + } + } + } + } + + // 2) Pause running tasks + final List>> pauseFutures = Lists.newArrayList(); + final List pauseTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); + for (final String taskId : pauseTaskIds) { + pauseFutures.add(taskClient.pauseAsync(taskId)); + } + + return Futures.transform( + Futures.successfulAsList(pauseFutures), new Function>, Map>() + { + @Nullable + @Override + public Map apply(List> input) + { + // 3) Build a map of the highest offset read by any task in the group for each partition + final Map endOffsets = new HashMap<>(); + for (int i = 0; i < input.size(); i++) { + Map result = input.get(i); + + if (result == null || result.isEmpty()) { // kill tasks that didn't return a value + String taskId = pauseTaskIds.get(i); + log.warn("Task [%s] failed to respond to [pause] in a timely manner, killing task", taskId); + killTask(taskId); + taskGroup.tasks.remove(taskId); + + } else { // otherwise build a map of the highest offsets seen + for (Map.Entry offset : result.entrySet()) { + if (!endOffsets.containsKey(offset.getKey()) + || endOffsets.get(offset.getKey()).compareTo(offset.getValue()) < 0) { + endOffsets.put(offset.getKey(), offset.getValue()); + } + } + } + } + + // 4) Set the end offsets for each task to the values from step 3 and resume the tasks. All the tasks should + // finish reading and start publishing within a short period, depending on how in sync the tasks were. + final List> setEndOffsetFutures = Lists.newArrayList(); + final List setEndOffsetTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); + + if (setEndOffsetTaskIds.isEmpty()) { + log.info("All tasks in taskGroup [%d] have failed, tasks will be re-created", taskGroup.groupId); + return null; + } + + try { + + if (endOffsets.equals(taskGroup.sequenceOffsets.lastEntry().getValue())) { + log.warn( + "Checkpoint [%s] is same as the start offsets [%s] of latest sequence for the task group [%d]", + endOffsets, + taskGroup.sequenceOffsets.lastEntry().getValue(), + taskGroup.groupId + ); + } + + log.info( + "Setting endOffsets for tasks in taskGroup [%d] to %s and resuming", + taskGroup.groupId, + endOffsets + ); + for (final String taskId : setEndOffsetTaskIds) { + setEndOffsetFutures.add(taskClient.setEndOffsetsAsync(taskId, endOffsets, finalize)); + } + + List results = Futures.successfulAsList(setEndOffsetFutures) + .get(futureTimeoutInSeconds, TimeUnit.SECONDS); + for (int i = 0; i < results.size(); i++) { + if (results.get(i) == null || !results.get(i)) { + String taskId = setEndOffsetTaskIds.get(i); + log.warn("Task [%s] failed to respond to [set end offsets] in a timely manner, killing task", taskId); + killTask(taskId); + taskGroup.tasks.remove(taskId); + } + } + } + catch (Exception e) { + log.error("Something bad happened [%s]", e.getMessage()); + Throwables.propagate(e); + } + + if (taskGroup.tasks.isEmpty()) { + log.info("All tasks in taskGroup [%d] have failed, tasks will be re-created", taskGroup.groupId); + return null; + } + + return endOffsets; + } + }, workerExec + ); + } + + private ListenableFuture stopTasksInGroup(@Nullable TaskGroup taskGroup) + { + if (taskGroup == null) { + return Futures.immediateFuture(null); + } + + final List> futures = Lists.newArrayList(); + for (Map.Entry entry : taskGroup.tasks.entrySet()) { + final String taskId = entry.getKey(); + final TaskData taskData = entry.getValue(); + if (taskData.status == null) { + killTask(taskId); + } else if (!taskData.status.isComplete()) { + futures.add(stopTask(taskId, false)); + } + } + + return Futures.successfulAsList(futures); + } + + private void checkPendingCompletionTasks() + throws ExecutionException, InterruptedException, TimeoutException + { + List> futures = Lists.newArrayList(); + + for (Map.Entry> pendingGroupList : pendingCompletionTaskGroups.entrySet()) { + + boolean stopTasksInTaskGroup = false; + Integer groupId = pendingGroupList.getKey(); + CopyOnWriteArrayList taskGroupList = pendingGroupList.getValue(); + List toRemove = Lists.newArrayList(); + + for (TaskGroup group : taskGroupList) { + boolean foundSuccess = false, entireTaskGroupFailed = false; + + if (stopTasksInTaskGroup) { + // One of the earlier groups that was handling the same partition set timed out before the segments were + // published so stop any additional groups handling the same partition set that are pending completion. + futures.add(stopTasksInGroup(group)); + toRemove.add(group); + continue; + } + + Iterator> iTask = group.tasks.entrySet().iterator(); + while (iTask.hasNext()) { + final Map.Entry entry = iTask.next(); + final String taskId = entry.getKey(); + final TaskData taskData = entry.getValue(); + + Preconditions.checkNotNull(taskData.status, "WTH? task[%s] has a null status", taskId); + + if (taskData.status.isFailure()) { + iTask.remove(); // remove failed task + if (group.tasks.isEmpty()) { + // if all tasks in the group have failed, just nuke all task groups with this partition set and restart + entireTaskGroupFailed = true; + break; + } + } + + if (taskData.status.isSuccess()) { + // If one of the pending completion tasks was successful, stop the rest of the tasks in the group as + // we no longer need them to publish their segment. + log.info("Task [%s] completed successfully, stopping tasks %s", taskId, group.taskIds()); + futures.add(stopTasksInGroup(group)); + foundSuccess = true; + toRemove.add(group); // remove the TaskGroup from the list of pending completion task groups + break; // skip iterating the rest of the tasks in this group as they've all been stopped now + } + } + + if ((!foundSuccess && group.completionTimeout.isBeforeNow()) || entireTaskGroupFailed) { + if (entireTaskGroupFailed) { + log.warn("All tasks in group [%d] failed to publish, killing all tasks for these partitions", groupId); + } else { + log.makeAlert( + "No task in [%s] for taskGroup [%d] succeeded before the completion timeout elapsed [%s]!", + group.taskIds(), + groupId, + ioConfig.getCompletionTimeout() + ).emit(); + } + + // 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); + // kill all the tasks in this pending completion group + killTasksInGroup(group); + // set a flag so the other pending completion groups for this set of partitions will also stop + stopTasksInTaskGroup = true; + + // kill all the tasks in the currently reading task group and remove the bad task group + killTasksInGroup(taskGroups.remove(groupId)); + toRemove.add(group); + } + } + + taskGroupList.removeAll(toRemove); + } + + // wait for all task shutdowns to complete before returning + Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + + private void checkCurrentTaskState() throws ExecutionException, InterruptedException, TimeoutException + { + List> futures = Lists.newArrayList(); + Iterator> iTaskGroups = taskGroups.entrySet().iterator(); + while (iTaskGroups.hasNext()) { + Map.Entry taskGroupEntry = iTaskGroups.next(); + Integer groupId = taskGroupEntry.getKey(); + TaskGroup taskGroup = taskGroupEntry.getValue(); + + // Iterate the list of known tasks in this group and: + // 1) Kill any tasks which are not "current" (have the partitions, starting offsets, and minimumMessageTime + // & maximumMessageTime (if applicable) in [taskGroups]) + // 2) Remove any tasks that have failed from the list + // 3) If any task completed successfully, stop all the tasks in this group and move to the next group + + log.debug("Task group [%d] pre-pruning: %s", groupId, taskGroup.taskIds()); + + Iterator> iTasks = taskGroup.tasks.entrySet().iterator(); + while (iTasks.hasNext()) { + Map.Entry task = iTasks.next(); + String taskId = task.getKey(); + TaskData taskData = task.getValue(); + + // stop and remove bad tasks from the task group + if (!isTaskCurrent(groupId, taskId)) { + log.info("Stopping task [%s] which does not match the expected offset range and ingestion spec", taskId); + futures.add(stopTask(taskId, false)); + iTasks.remove(); + continue; + } + + Preconditions.checkNotNull(taskData.status, "WTH? task[%s] has a null status", taskId); + + // remove failed tasks + if (taskData.status.isFailure()) { + iTasks.remove(); + continue; + } + + // check for successful tasks, and if we find one, stop all tasks in the group and remove the group so it can + // be recreated with the next set of offsets + if (taskData.status.isSuccess()) { + futures.add(stopTasksInGroup(taskGroup)); + iTaskGroups.remove(); + break; + } + } + log.debug("Task group [%d] post-pruning: %s", groupId, taskGroup.taskIds()); + } + + // wait for all task shutdowns to complete before returning + Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + + private void createNewTasks() throws JsonProcessingException + { + // update the checkpoints in the taskGroup to latest ones so that new tasks do not read what is already published + verifyAndMergeCheckpoints( + taskGroups.values() + .stream() + .filter(taskGroup -> taskGroup.tasks.size() < ioConfig.getReplicas()) + .collect(Collectors.toList()) + ); + + // check that there is a current task group for each group of partitions in [partitionGroups] + for (Integer groupId : partitionGroups.keySet()) { + if (!taskGroups.containsKey(groupId)) { + log.info("Creating new task group [%d] for partitions %s", groupId, partitionGroups.get(groupId).keySet()); + + Optional minimumMessageTime = (ioConfig.getLateMessageRejectionPeriod().isPresent() ? Optional.of( + DateTimes.nowUtc().minus(ioConfig.getLateMessageRejectionPeriod().get()) + ) : Optional.absent()); + + Optional maximumMessageTime = (ioConfig.getEarlyMessageRejectionPeriod().isPresent() ? Optional.of( + DateTimes.nowUtc().plus(ioConfig.getTaskDuration()).plus(ioConfig.getEarlyMessageRejectionPeriod().get()) + ) : Optional.absent()); + + try { + taskGroups.put( + groupId, + new TaskGroup( + groupId, + generateStartingSequence(groupId), + minimumMessageTime, + maximumMessageTime, + null //TODO: exclusive sequence + ) + ); + } + catch (TimeoutException e) { + log.warn( + e, + "Timeout while fetching sequence numbers - if you are reading from the latest sequence number, you need to write events to the stream before the sequence number can be determined" + ); + } + } + } + + // iterate through all the current task groups and make sure each one has the desired number of replica tasks + boolean createdTask = false; + for (Map.Entry entry : taskGroups.entrySet()) { + TaskGroup taskGroup = entry.getValue(); + Integer groupId = entry.getKey(); + + //TODO: kinesis +// if (taskGroup.partitionOffsets == null || taskGroup.partitionOffsets +// .values().stream().allMatch(x -> x == null || Record.END_OF_SHARD_MARKER.equals(x))) { +// log.debug("Nothing to read in any partition for taskGroup [%d], skipping task creation", groupId); +// continue; +// } + + if (ioConfig.getReplicas() > taskGroup.tasks.size()) { + log.info( + "Number of tasks [%d] does not match configured numReplicas [%d] in task group [%d], creating more tasks", + taskGroup.tasks.size(), ioConfig.getReplicas(), groupId + ); + createTasksForGroup(groupId, ioConfig.getReplicas() - taskGroup.tasks.size()); + createdTask = true; + } + } + + if (createdTask && firstRunTime.isBeforeNow()) { + // Schedule a run event after a short delay to update our internal data structures with the new tasks that were + // just created. This is mainly for the benefit of the status API in situations where the run period is lengthy. + scheduledExec.schedule(buildRunTask(), 5000, TimeUnit.MILLISECONDS); + } + + } + + abstract protected void createTasksForGroup(int group, int replicas) throws JsonProcessingException; + + abstract protected ImmutableMap generateStartingSequence(int groupId) + throws TimeoutException; abstract protected RecordSupplier setupRecordSupplier(); @@ -1002,4 +1684,13 @@ abstract protected void checkPendingCompletionTasks() abstract protected void scheduleReporting(); + abstract protected int getTaskGroupIdForPartition(T1 partition); + + abstract protected void checkSourceMetadataInstanceMatch(DataSourceMetadata metadata) throws IAE; + + // TODO: may want to put more logic in the base class + abstract protected SupervisorReport generateReport(boolean includeOffsets); + + abstract protected boolean isTaskInstanceOfThis(Task task); + } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorIOConfig.java index e425d5be9475..d17135520bdd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorIOConfig.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.SeekableStream.supervisor; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.base.Preconditions; @@ -34,23 +33,22 @@ abstract public class SeekableStreamSupervisorIOConfig private final Duration taskDuration; private final Duration startDelay; private final Duration period; - private final boolean useEarliestOffset; + private final boolean useEarliestSequenceNumber; private final Duration completionTimeout; private final Optional lateMessageRejectionPeriod; private final Optional earlyMessageRejectionPeriod; - @JsonCreator public SeekableStreamSupervisorIOConfig( - @JsonProperty("id") String id, - @JsonProperty("replicas") Integer replicas, - @JsonProperty("taskCount") Integer taskCount, - @JsonProperty("taskDuration") Period taskDuration, - @JsonProperty("startDelay") Period startDelay, - @JsonProperty("period") Period period, - @JsonProperty("useEarliestOffset") Boolean useEarliestOffset, - @JsonProperty("completionTimeout") Period completionTimeout, - @JsonProperty("lateMessageRejectionPeriod") Period lateMessageRejectionPeriod, - @JsonProperty("earlyMessageRejectionPeriod") Period earlyMessageRejectionPeriod + String id, + Integer replicas, + Integer taskCount, + Period taskDuration, + Period startDelay, + Period period, + Boolean useEarliestSequenceNumber, + Period completionTimeout, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod ) { this.id = Preconditions.checkNotNull(id, "id cannot be null"); @@ -59,7 +57,7 @@ public SeekableStreamSupervisorIOConfig( this.taskDuration = defaultDuration(taskDuration, "PT1H"); this.startDelay = defaultDuration(startDelay, "PT5S"); this.period = defaultDuration(period, "PT30S"); - this.useEarliestOffset = useEarliestOffset != null ? useEarliestOffset : false; + this.useEarliestSequenceNumber = useEarliestSequenceNumber != null ? useEarliestSequenceNumber : false; this.completionTimeout = defaultDuration(completionTimeout, "PT30M"); this.lateMessageRejectionPeriod = lateMessageRejectionPeriod == null ? Optional.absent() @@ -69,7 +67,7 @@ public SeekableStreamSupervisorIOConfig( : Optional.of(earlyMessageRejectionPeriod.toStandardDuration()); } - @JsonProperty + public String getId() { return id; @@ -105,10 +103,10 @@ public Duration getPeriod() return period; } - @JsonProperty - public boolean isUseEarliestOffset() + + public boolean isUseEarliestSequenceNumber() { - return useEarliestOffset; + return useEarliestSequenceNumber; } @JsonProperty diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorReportPayload.java index 05ee3026049a..7864676c5e1a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorReportPayload.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -29,31 +29,34 @@ import java.util.List; import java.util.Map; -abstract public class SeekableStreamSupervisorReportPayload +abstract public class SeekableStreamSupervisorReportPayload { private final String dataSource; + private final String id; private final int partitions; private final int replicas; private final long durationSeconds; - private final List activeTasks; - private final List publishingTasks; - private final Map latestOffsets; - private final Map minimumLag; + private final List activeTasks; + private final List publishingTasks; + private final Map latestOffsets; + private final Map minimumLag; private final Long aggregateLag; private final DateTime offsetsLastUpdated; public SeekableStreamSupervisorReportPayload( String dataSource, + String id, int partitions, int replicas, long durationSeconds, - @Nullable Map latestOffsets, - @Nullable Map minimumLag, + @Nullable Map latestOffsets, + @Nullable Map minimumLag, @Nullable Long aggregateLag, @Nullable DateTime offsetsLastUpdated ) { this.dataSource = dataSource; + this.id = id; this.partitions = partitions; this.replicas = replicas; this.durationSeconds = durationSeconds; @@ -65,11 +68,11 @@ public SeekableStreamSupervisorReportPayload( this.offsetsLastUpdated = offsetsLastUpdated; } - public void addTask(TaskReportData data) + public void addTask(SeekableStreamTaskReportData data) { - if (data.getType().equals(TaskReportData.TaskType.ACTIVE)) { + if (data.getType().equals(SeekableStreamTaskReportData.TaskType.ACTIVE)) { activeTasks.add(data); - } else if (data.getType().equals(TaskReportData.TaskType.PUBLISHING)) { + } else if (data.getType().equals(SeekableStreamTaskReportData.TaskType.PUBLISHING)) { publishingTasks.add(data); } else { throw new IAE("Unknown task type [%s]", data.getType().name()); @@ -82,6 +85,11 @@ public String getDataSource() return dataSource; } + protected String getId() + { + return id; + } + @JsonProperty public int getPartitions() { @@ -101,27 +109,27 @@ public long getDurationSeconds() } @JsonProperty - public List getActiveTasks() + public List getActiveTasks() { return activeTasks; } @JsonProperty - public List getPublishingTasks() + public List getPublishingTasks() { return publishingTasks; } @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getLatestOffsets() + public Map getLatestOffsets() { return latestOffsets; } @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getMinimumLag() + public Map getMinimumLag() { return minimumLag; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorSpec.java index 830d03d175aa..bcd09044a54f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorSpec.java @@ -26,8 +26,8 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import org.apache.druid.guice.annotations.Json; +import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; @@ -40,21 +40,21 @@ import java.util.List; import java.util.Map; -public class SeekableStreamSupervisorSpec implements SupervisorSpec +abstract public class SeekableStreamSupervisorSpec implements SupervisorSpec { private final DataSchema dataSchema; private final SeekableStreamSupervisorTuningConfig tuningConfig; private final SeekableStreamSupervisorIOConfig ioConfig; private final Map context; - - private final TaskStorage taskStorage; - private final TaskMaster taskMaster; - private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; - private final IndexTaskClientFactory indexTaskClientFactory; - private final ObjectMapper mapper; private final ServiceEmitter emitter; private final DruidMonitorSchedulerConfig monitorSchedulerConfig; - private final RowIngestionMetersFactory rowIngestionMetersFactory; + + protected final TaskStorage taskStorage; + protected final TaskMaster taskMaster; + protected final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; + protected final SeekableStreamIndexTaskClientFactory indexTaskClientFactory; + protected final ObjectMapper mapper; + protected final RowIngestionMetersFactory rowIngestionMetersFactory; @JsonCreator public SeekableStreamSupervisorSpec( @@ -65,7 +65,7 @@ public SeekableStreamSupervisorSpec( @JacksonInject TaskStorage taskStorage, @JacksonInject TaskMaster taskMaster, @JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, - @JacksonInject IndexTaskClientFactory indexTaskClientFactory, + @JacksonInject SeekableStreamIndexTaskClientFactory indexTaskClientFactory, @JacksonInject @Json ObjectMapper mapper, @JacksonInject ServiceEmitter emitter, @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, @@ -73,32 +73,7 @@ public SeekableStreamSupervisorSpec( ) { this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); - this.tuningConfig = tuningConfig != null - ? tuningConfig - : new SeekableStreamSupervisorTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + this.tuningConfig = tuningConfig; // null check done in concrete class this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); this.context = context; @@ -153,18 +128,7 @@ public DruidMonitorSchedulerConfig getMonitorSchedulerConfig() } @Override - public Supervisor createSupervisor() - { - return new SeekableStreamSupervisor( - taskStorage, - taskMaster, - indexerMetadataStorageCoordinator, - indexTaskClientFactory, - mapper, - this, - rowIngestionMetersFactory - ); - } + abstract public Supervisor createSupervisor(); @Override public List getDataSources() @@ -173,12 +137,7 @@ public List getDataSources() } @Override - public String toString() - { - return "SeekableStreamSupervisorSpec{" + - "dataSchema=" + dataSchema + - ", tuningConfig=" + tuningConfig + - ", ioConfig=" + ioConfig + - '}'; - } + abstract public String toString(); + + } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorTuningConfig.java index 591a1bbf2c83..a370d44b0388 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorTuningConfig.java @@ -21,142 +21,33 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.SeekableStream.SeekableStreamTuningConfig; -import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.indexing.TuningConfigs; -import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Duration; import org.joda.time.Period; -import javax.annotation.Nullable; -import java.io.File; - -public class SeekableStreamSupervisorTuningConfig extends SeekableStreamTuningConfig +public interface SeekableStreamSupervisorTuningConfig { - private final Integer workerThreads; - private final Integer chatThreads; - private final Long chatRetries; - private final Duration httpTimeout; - private final Duration shutdownTimeout; - private final Duration offsetFetchPeriod; - - public SeekableStreamSupervisorTuningConfig( - @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, - @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, - @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, - @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, - @JsonProperty("basePersistDirectory") File basePersistDirectory, - @JsonProperty("maxPendingPersists") Integer maxPendingPersists, - @JsonProperty("indexSpec") IndexSpec indexSpec, - // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. - @JsonProperty("buildV9Directly") Boolean buildV9Directly, - @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, - @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, - @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, - @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - @JsonProperty("workerThreads") Integer workerThreads, - @JsonProperty("chatThreads") Integer chatThreads, - @JsonProperty("chatRetries") Long chatRetries, - @JsonProperty("httpTimeout") Period httpTimeout, - @JsonProperty("shutdownTimeout") Period shutdownTimeout, - @JsonProperty("offsetFetchPeriod") Period offsetFetchPeriod, - @JsonProperty("intermediateHandoffPeriod") Period intermediateHandoffPeriod, - @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, - @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, - @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions - ) - { - super( - maxRowsInMemory, - maxBytesInMemory, - maxRowsPerSegment, - intermediatePersistPeriod, - basePersistDirectory, - maxPendingPersists, - indexSpec, - true, - reportParseExceptions, - handoffConditionTimeout, - resetOffsetAutomatically, - segmentWriteOutMediumFactory, - intermediateHandoffPeriod, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions - ); - - this.workerThreads = workerThreads; - this.chatThreads = chatThreads; - this.chatRetries = (chatRetries != null ? chatRetries : 8); - this.httpTimeout = defaultDuration(httpTimeout, "PT10S"); - this.shutdownTimeout = defaultDuration(shutdownTimeout, "PT80S"); - this.offsetFetchPeriod = defaultDuration(offsetFetchPeriod, "PT30S"); - } @JsonProperty - public Integer getWorkerThreads() - { - return workerThreads; - } - - @JsonProperty - public Integer getChatThreads() - { - return chatThreads; - } + Integer getWorkerThreads(); @JsonProperty - public Long getChatRetries() - { - return chatRetries; - } + Integer getChatThreads(); @JsonProperty - public Duration getHttpTimeout() - { - return httpTimeout; - } + Long getChatRetries(); @JsonProperty - public Duration getShutdownTimeout() - { - return shutdownTimeout; - } + Duration getHttpTimeout(); @JsonProperty - public Duration getOffsetFetchPeriod() - { - return offsetFetchPeriod; - } + Duration getShutdownTimeout(); @Override - public String toString() - { - return "SeekableStreamSupervisorTuningConfig{" + - "maxRowsInMemory=" + getMaxRowsInMemory() + - ", maxRowsPerSegment=" + getMaxRowsPerSegment() + - ", maxBytesInMemory=" + TuningConfigs.getMaxBytesInMemoryOrDefault(getMaxBytesInMemory()) + - ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + - ", basePersistDirectory=" + getBasePersistDirectory() + - ", maxPendingPersists=" + getMaxPendingPersists() + - ", indexSpec=" + getIndexSpec() + - ", reportParseExceptions=" + isReportParseExceptions() + - ", handoffConditionTimeout=" + getHandoffConditionTimeout() + - ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + - ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() + - ", workerThreads=" + workerThreads + - ", chatThreads=" + chatThreads + - ", chatRetries=" + chatRetries + - ", httpTimeout=" + httpTimeout + - ", shutdownTimeout=" + shutdownTimeout + - ", offsetFetchPeriod=" + offsetFetchPeriod + - ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() + - ", logParseExceptions=" + isLogParseExceptions() + - ", maxParseExceptions=" + getMaxParseExceptions() + - ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + - '}'; - } + String toString(); + + SeekableStreamTuningConfig copyOf(); - private static Duration defaultDuration(final Period period, final String theDefault) + static Duration defaultDuration(final Period period, final String theDefault) { return (period == null ? new Period(theDefault) : period).toStandardDuration(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/TaskReportData.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamTaskReportData.java similarity index 67% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/TaskReportData.java rename to indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamTaskReportData.java index 8c0e84c2f8ee..cc3a485cb1f5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/TaskReportData.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamTaskReportData.java @@ -26,24 +26,27 @@ import javax.annotation.Nullable; import java.util.Map; -public class TaskReportData +abstract public class SeekableStreamTaskReportData { + public enum TaskType + { + ACTIVE, PUBLISHING, UNKNOWN + } + private final String id; - private final Map startingOffsets; + private final Map startingOffsets; private final DateTime startTime; private final Long remainingSeconds; private final TaskType type; - private final Map currentOffsets; - private final Map lag; + private final Map currentOffsets; - public TaskReportData( + public SeekableStreamTaskReportData( String id, - @Nullable Map startingOffsets, - @Nullable Map currentOffsets, + @Nullable Map startingOffsets, + @Nullable Map currentOffsets, @Nullable DateTime startTime, Long remainingSeconds, - TaskType type, - @Nullable Map lag + TaskType type ) { this.id = id; @@ -52,7 +55,6 @@ public TaskReportData( this.startTime = startTime; this.remainingSeconds = remainingSeconds; this.type = type; - this.lag = lag; } @JsonProperty @@ -63,14 +65,14 @@ public String getId() @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getStartingOffsets() + public Map getStartingOffsets() { return startingOffsets; } @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getCurrentOffsets() + public Map getCurrentOffsets() { return currentOffsets; } @@ -93,28 +95,7 @@ public TaskType getType() return type; } - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getLag() - { - return lag; - } - @Override - public String toString() - { - return "{" + - "id='" + id + '\'' + - (startingOffsets != null ? ", startingOffsets=" + startingOffsets : "") + - (currentOffsets != null ? ", currentOffsets=" + currentOffsets : "") + - ", startTime=" + startTime + - ", remainingSeconds=" + remainingSeconds + - (lag != null ? ", lag=" + lag : "") + - '}'; - } + abstract public String toString(); - public enum TaskType - { - ACTIVE, PUBLISHING, UNKNOWN - } } From 0bc685349cda751702f35649d1880c6acec8e9e4 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 12 Sep 2018 18:10:14 -0700 Subject: [PATCH 03/87] first attempt to integrate kafa indexing service to use SeekableStream --- ...ementalPublishingKafkaIndexTaskRunner.java | 2 +- .../kafka/KafkaDataSourceMetadata.java | 33 +- .../druid/indexing/kafka/KafkaIOConfig.java | 88 ++--- .../druid/indexing/kafka/KafkaIndexTask.java | 136 ++----- .../indexing/kafka/KafkaIndexTaskClient.java | 327 +---------------- .../kafka/KafkaIndexTaskClientFactory.java | 17 +- .../indexing/kafka/KafkaIndexTaskRunner.java | 10 +- .../druid/indexing/kafka/KafkaPartitions.java | 26 +- .../indexing/kafka/KafkaTuningConfig.java | 335 +++--------------- .../kafka/LegacyKafkaIndexTaskRunner.java | 8 +- .../kafka/supervisor/KafkaSupervisor.java | 18 +- .../supervisor/KafkaSupervisorIOConfig.java | 126 ++----- .../KafkaSupervisorReportPayload.java | 144 ++------ .../kafka/supervisor/KafkaSupervisorSpec.java | 186 +++------- .../KafkaSupervisorTuningConfig.java | 21 +- .../kafka/supervisor/TaskReportData.java | 76 +--- .../indexing/kafka/KafkaTuningConfigTest.java | 2 +- .../kafka/supervisor/KafkaSupervisorTest.java | 72 ++-- 18 files changed, 359 insertions(+), 1268 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 34eca06c626e..695f85998558 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -44,6 +44,7 @@ import org.apache.druid.discovery.NodeType; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask.Status; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; @@ -56,7 +57,6 @@ import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.RealtimeIndexTask; -import org.apache.druid.indexing.kafka.KafkaIndexTask.Status; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java index 3e5bec6d9530..0452a5d83687 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java @@ -22,49 +22,36 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; +import org.apache.druid.indexing.SeekableStream.SeekableStreamDataSourceMetadata; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.java.util.common.IAE; import java.util.Map; import java.util.Objects; -public class KafkaDataSourceMetadata implements DataSourceMetadata +public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata { - private final KafkaPartitions kafkaPartitions; @JsonCreator public KafkaDataSourceMetadata( @JsonProperty("partitions") KafkaPartitions kafkaPartitions ) { - this.kafkaPartitions = kafkaPartitions; + super(kafkaPartitions); } @JsonProperty("partitions") public KafkaPartitions getKafkaPartitions() { - return kafkaPartitions; + return (KafkaPartitions) super.getSeekableStreamPartitions(); } - @Override - public boolean isValidStart() - { - return true; - } - - @Override - public boolean matches(DataSourceMetadata other) - { - if (getClass() != other.getClass()) { - return false; - } - - return plus(other).equals(other.plus(this)); - } @Override public DataSourceMetadata plus(DataSourceMetadata other) { + KafkaPartitions kafkaPartitions = getKafkaPartitions(); + if (!(other instanceof KafkaDataSourceMetadata)) { throw new IAE( "Expected instance of %s, got %s", @@ -97,6 +84,8 @@ public DataSourceMetadata plus(DataSourceMetadata other) @Override public DataSourceMetadata minus(DataSourceMetadata other) { + KafkaPartitions kafkaPartitions = getKafkaPartitions(); + if (!(other instanceof KafkaDataSourceMetadata)) { throw new IAE( "Expected instance of %s, got %s", @@ -134,20 +123,20 @@ public boolean equals(Object o) return false; } KafkaDataSourceMetadata that = (KafkaDataSourceMetadata) o; - return Objects.equals(kafkaPartitions, that.kafkaPartitions); + return Objects.equals(getKafkaPartitions(), that.getKafkaPartitions()); } @Override public int hashCode() { - return Objects.hash(kafkaPartitions); + return Objects.hash(getKafkaPartitions()); } @Override public String toString() { return "KafkaDataSourceMetadata{" + - "kafkaPartitions=" + kafkaPartitions + + "kafkaPartitions=" + getKafkaPartitions() + '}'; } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java index 6a9af7fcea94..4f84c47c0650 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java @@ -21,28 +21,19 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import org.apache.druid.segment.indexing.IOConfig; +import org.apache.druid.indexing.SeekableStream.SeekableStreamIOConfig; import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.Map; -public class KafkaIOConfig implements IOConfig +public class KafkaIOConfig extends SeekableStreamIOConfig { - private static final boolean DEFAULT_USE_TRANSACTION = true; private static final boolean DEFAULT_SKIP_OFFSET_GAPS = false; @Nullable - private final Integer taskGroupId; - private final String baseSequenceName; - private final KafkaPartitions startPartitions; - private final KafkaPartitions endPartitions; - private final Map consumerProperties; - private final boolean useTransaction; - private final Optional minimumMessageTime; - private final Optional maximumMessageTime; + private final Map consumerProperties; private final boolean skipOffsetGaps; @JsonCreator @@ -51,21 +42,24 @@ public KafkaIOConfig( @JsonProperty("baseSequenceName") String baseSequenceName, @JsonProperty("startPartitions") KafkaPartitions startPartitions, @JsonProperty("endPartitions") KafkaPartitions endPartitions, - @JsonProperty("consumerProperties") Map consumerProperties, + @JsonProperty("consumerProperties") Map consumerProperties, @JsonProperty("useTransaction") Boolean useTransaction, @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, @JsonProperty("skipOffsetGaps") Boolean skipOffsetGaps ) { - this.taskGroupId = taskGroupId; - this.baseSequenceName = Preconditions.checkNotNull(baseSequenceName, "baseSequenceName"); - this.startPartitions = Preconditions.checkNotNull(startPartitions, "startPartitions"); - this.endPartitions = Preconditions.checkNotNull(endPartitions, "endPartitions"); + super( + taskGroupId, + baseSequenceName, + startPartitions, + endPartitions, + useTransaction, + minimumMessageTime, + maximumMessageTime + ); + this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); - this.useTransaction = useTransaction != null ? useTransaction : DEFAULT_USE_TRANSACTION; - this.minimumMessageTime = Optional.fromNullable(minimumMessageTime); - this.maximumMessageTime = Optional.fromNullable(maximumMessageTime); this.skipOffsetGaps = skipOffsetGaps != null ? skipOffsetGaps : DEFAULT_SKIP_OFFSET_GAPS; Preconditions.checkArgument( @@ -88,54 +82,28 @@ public KafkaIOConfig( } } - @Nullable - @JsonProperty - public Integer getTaskGroupId() - { - return taskGroupId; - } - - @JsonProperty - public String getBaseSequenceName() - { - return baseSequenceName; - } - + @Override @JsonProperty public KafkaPartitions getStartPartitions() { - return startPartitions; + return (KafkaPartitions) super.getStartPartitions(); } + @Override @JsonProperty public KafkaPartitions getEndPartitions() { - return endPartitions; + return (KafkaPartitions) super.getEndPartitions(); } - @JsonProperty - public Map getConsumerProperties() - { - return consumerProperties; - } - - @JsonProperty - public boolean isUseTransaction() - { - return useTransaction; - } + @Nullable @JsonProperty - public Optional getMaximumMessageTime() + public Map getConsumerProperties() { - return maximumMessageTime; + return consumerProperties; } - @JsonProperty - public Optional getMinimumMessageTime() - { - return minimumMessageTime; - } @JsonProperty public boolean isSkipOffsetGaps() @@ -147,14 +115,14 @@ public boolean isSkipOffsetGaps() public String toString() { return "KafkaIOConfig{" + - "taskGroupId=" + taskGroupId + - ", baseSequenceName='" + baseSequenceName + '\'' + - ", startPartitions=" + startPartitions + - ", endPartitions=" + endPartitions + + "taskGroupId=" + getTaskGroupId() + + ", baseSequenceName='" + getBaseSequenceName() + '\'' + + ", startPartitions=" + getStartPartitions() + + ", endPartitions=" + getEndPartitions() + ", consumerProperties=" + consumerProperties + - ", useTransaction=" + useTransaction + - ", minimumMessageTime=" + minimumMessageTime + - ", maximumMessageTime=" + maximumMessageTime + + ", useTransaction=" + isUseTransaction() + + ", minimumMessageTime=" + getMinimumMessageTime() + + ", maximumMessageTime=" + getMaximumMessageTime() + ", skipOffsetGaps=" + skipOffsetGaps + '}'; } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index bb73651e6e87..00a37431fae5 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -22,32 +22,22 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Joiner; -import com.google.common.base.Optional; -import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask; import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; -import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.AbstractTask; -import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.TaskResource; -import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor; -import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.metadata.PasswordProvider; import org.apache.druid.query.NoopQueryRunner; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; @@ -56,7 +46,6 @@ import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.Appenderators; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; -import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.utils.CircularBuffer; @@ -64,38 +53,23 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import java.nio.ByteBuffer; import java.util.Map; import java.util.Properties; +import java.util.Random; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -public class KafkaIndexTask extends AbstractTask implements ChatHandler +public class KafkaIndexTask extends SeekableStreamIndexTask { - public enum Status - { - NOT_STARTED, - STARTING, - READING, - PAUSED, - PUBLISHING - // ideally this should be called FINISHING now as the task does incremental publishes - // through out its lifetime - } private static final EmittingLogger log = new EmittingLogger(KafkaIndexTask.class); - private static final String TYPE = "index_kafka"; + private static final String TYPE = "index_kafka"; //TODO: figure something out about TYPE + private static final Random RANDOM = new Random(); static final long POLL_TIMEOUT_MILLIS = TimeUnit.MILLISECONDS.toMillis(100); static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15; - private final DataSchema dataSchema; - private final InputRowParser parser; - private final KafkaTuningConfig tuningConfig; - private final KafkaIOConfig ioConfig; - private final Optional chatHandlerProvider; private final KafkaIndexTaskRunner runner; - private final ObjectMapper configMapper; // This value can be tuned in some tests private long pollRetryMs = 30000; @@ -110,24 +84,21 @@ public KafkaIndexTask( @JsonProperty("context") Map context, @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, - @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, - @JacksonInject ObjectMapper configMapper + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory ) { super( - id == null ? makeTaskId(dataSchema.getDataSource()) : id, - StringUtils.format("%s_%s", TYPE, dataSchema.getDataSource()), + id, taskResource, - dataSchema.getDataSource(), - context + dataSchema, + tuningConfig, + ioConfig, + context, + chatHandlerProvider, + authorizerMapper, + rowIngestionMetersFactory ); - this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); - this.parser = Preconditions.checkNotNull((InputRowParser) dataSchema.getParser(), "parser"); - this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); - this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); - this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); - this.configMapper = configMapper; final CircularBuffer savedParseExceptions; if (tuningConfig.getMaxSavedParseExceptions() > 0) { savedParseExceptions = new CircularBuffer<>(tuningConfig.getMaxSavedParseExceptions()); @@ -162,48 +133,6 @@ long getPollRetryMs() return pollRetryMs; } - private static String makeTaskId(String dataSource) - { - return Joiner.on("_").join(TYPE, dataSource, RealtimeIndexTask.makeRandomId()); - } - - @Override - public int getPriority() - { - return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY); - } - - @Override - public String getType() - { - return TYPE; - } - - @Override - public boolean isReady(TaskActionClient taskActionClient) - { - return true; - } - - @JsonProperty - public DataSchema getDataSchema() - { - return dataSchema; - } - - @JsonProperty - public KafkaTuningConfig getTuningConfig() - { - return tuningConfig; - } - - @JsonProperty("ioConfig") - public KafkaIOConfig getIOConfig() - { - return ioConfig; - } - - @Override public TaskStatus run(final TaskToolbox toolbox) { @@ -290,7 +219,9 @@ KafkaConsumer newConsumer() final Properties props = new Properties(); - addConsumerPropertiesFromConfig(props, configMapper, ioConfig.getConsumerProperties()); + for (Map.Entry entry : ((KafkaIOConfig) ioConfig).getConsumerProperties().entrySet()) { + props.setProperty(entry.getKey(), entry.getValue()); + } props.setProperty("enable.auto.commit", "false"); props.setProperty("auto.offset.reset", "none"); @@ -304,25 +235,6 @@ KafkaConsumer newConsumer() } } - public static void addConsumerPropertiesFromConfig(Properties properties, ObjectMapper configMapper, Map consumerProperties) - { - // Extract passwords before SSL connection to Kafka - for (Map.Entry entry : consumerProperties.entrySet()) { - String propertyKey = entry.getKey(); - if (propertyKey.equals(KafkaSupervisorIOConfig.TRUST_STORE_PASSWORD_KEY) - || propertyKey.equals(KafkaSupervisorIOConfig.KEY_STORE_PASSWORD_KEY) - || propertyKey.equals(KafkaSupervisorIOConfig.KEY_PASSWORD_KEY)) { - PasswordProvider configPasswordProvider = configMapper.convertValue( - entry.getValue(), - PasswordProvider.class - ); - properties.setProperty(propertyKey, configPasswordProvider.getPassword()); - } else { - properties.setProperty(propertyKey, String.valueOf(entry.getValue())); - } - } - } - static void assignPartitions( final KafkaConsumer consumer, final String topic, @@ -371,6 +283,13 @@ boolean withinMinMaxRecordTime(final InputRow row) return !beforeMinimumMessageTime && !afterMaximumMessageTime; } + @Override + @JsonProperty + public KafkaTuningConfig getTuningConfig() + { + return (KafkaTuningConfig) super.getTuningConfig(); + } + @VisibleForTesting void setPollRetryMs(long retryMs) { @@ -388,4 +307,11 @@ KafkaIndexTaskRunner getRunner() { return runner; } + + @Override + @JsonProperty("ioConfig") + public KafkaIOConfig getIOConfig() + { + return (KafkaIOConfig) super.getIOConfig(); + } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java index 803a6bea07f0..1162a251968c 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -19,34 +19,19 @@ package org.apache.druid.indexing.kafka; -import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.indexing.common.IndexTaskClient; import org.apache.druid.indexing.common.TaskInfoProvider; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.RE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.java.util.http.client.response.FullResponseHolder; -import org.jboss.netty.handler.codec.http.HttpMethod; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import org.joda.time.DateTime; import org.joda.time.Duration; -import javax.annotation.Nullable; -import java.io.IOException; -import java.util.Collections; import java.util.Map; -import java.util.TreeMap; -public class KafkaIndexTaskClient extends IndexTaskClient +//Left here for backward compatibility +public class KafkaIndexTaskClient extends SeekableStreamIndexTaskClient { - private static final EmittingLogger log = new EmittingLogger(KafkaIndexTaskClient.class); - private static final TreeMap> EMPTY_TREE_MAP = new TreeMap<>(); + private static ObjectMapper mapper = new ObjectMapper(); public KafkaIndexTaskClient( HttpClient httpClient, @@ -58,298 +43,20 @@ public KafkaIndexTaskClient( long numRetries ) { - super(httpClient, jsonMapper, taskInfoProvider, httpTimeout, dataSource, numThreads, numRetries); + super( + httpClient, + jsonMapper, + taskInfoProvider, + dataSource, + numThreads, + httpTimeout, + numRetries + ); } - public boolean stop(final String id, final boolean publish) + @Override + protected JavaType constructPartitionOffsetMapType(Class mapType) { - log.debug("Stop task[%s] publish[%s]", id, publish); - - try { - final FullResponseHolder response = submitRequestWithEmptyContent( - id, - HttpMethod.POST, - "stop", - publish ? "publish=true" : null, - true - ); - return isSuccess(response); - } - catch (NoTaskLocationException e) { - return false; - } - catch (TaskNotRunnableException e) { - log.info("Task [%s] couldn't be stopped because it is no longer running", id); - return true; - } - catch (Exception e) { - log.warn(e, "Exception while stopping task [%s]", id); - return false; - } - } - - public boolean resume(final String id) - { - log.debug("Resume task[%s]", id); - - try { - final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.POST, "resume", null, true); - return isSuccess(response); - } - catch (NoTaskLocationException | IOException e) { - log.warn(e, "Exception while stopping task [%s]", id); - return false; - } - } - - public Map pause(final String id) - { - log.debug("Pause task[%s]", id); - - try { - final FullResponseHolder response = submitRequestWithEmptyContent( - id, - HttpMethod.POST, - "pause", - null, - true - ); - - if (response.getStatus().equals(HttpResponseStatus.OK)) { - log.info("Task [%s] paused successfully", id); - return deserialize(response.getContent(), new TypeReference>() - { - }); - } - - while (true) { - if (getStatus(id) == KafkaIndexTask.Status.PAUSED) { - return getCurrentOffsets(id, true); - } - - final Duration delay = newRetryPolicy().getAndIncrementRetryDelay(); - if (delay == null) { - log.error("Task [%s] failed to pause, aborting", id); - throw new ISE("Task [%s] failed to pause, aborting", id); - } else { - final long sleepTime = delay.getMillis(); - log.info( - "Still waiting for task [%s] to pause; will try again in [%s]", - id, - new Duration(sleepTime).toString() - ); - Thread.sleep(sleepTime); - } - } - } - catch (NoTaskLocationException e) { - log.error("Exception [%s] while pausing Task [%s]", e.getMessage(), id); - return ImmutableMap.of(); - } - catch (IOException | InterruptedException e) { - throw new RE(e, "Exception [%s] while pausing Task [%s]", e.getMessage(), id); - } - } - - public KafkaIndexTask.Status getStatus(final String id) - { - log.debug("GetStatus task[%s]", id); - - try { - final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "status", null, true); - return deserialize(response.getContent(), KafkaIndexTask.Status.class); - } - catch (NoTaskLocationException e) { - return KafkaIndexTask.Status.NOT_STARTED; - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Nullable - public DateTime getStartTime(final String id) - { - log.debug("GetStartTime task[%s]", id); - - try { - final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "time/start", null, true); - return response.getContent() == null || response.getContent().isEmpty() - ? null - : deserialize(response.getContent(), DateTime.class); - } - catch (NoTaskLocationException e) { - return null; - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - public Map getMovingAverages(final String id) - { - log.debug("GetMovingAverages task[%s]", id); - - try { - final FullResponseHolder response = submitRequestWithEmptyContent( - id, - HttpMethod.GET, - "rowStats", - null, - true - ); - return response.getContent() == null || response.getContent().isEmpty() - ? Collections.emptyMap() - : deserialize(response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT); - } - catch (NoTaskLocationException e) { - return Collections.emptyMap(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - public Map getCurrentOffsets(final String id, final boolean retry) - { - log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry); - - try { - final FullResponseHolder response = submitRequestWithEmptyContent( - id, - HttpMethod.GET, - "offsets/current", - null, - retry - ); - return deserialize(response.getContent(), new TypeReference>() - { - }); - } - catch (NoTaskLocationException e) { - return ImmutableMap.of(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - public TreeMap> getCheckpoints(final String id, final boolean retry) - { - log.debug("GetCheckpoints task[%s] retry[%s]", id, retry); - try { - final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "checkpoints", null, retry); - return deserialize( - response.getContent(), - new TypeReference>>() - { - } - ); - } - catch (NoTaskLocationException e) { - return EMPTY_TREE_MAP; - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - public ListenableFuture>> getCheckpointsAsync( - final String id, - final boolean retry - ) - { - return doAsync(() -> getCheckpoints(id, retry)); - } - - public Map getEndOffsets(final String id) - { - log.debug("GetEndOffsets task[%s]", id); - - try { - final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "offsets/end", null, true); - return deserialize(response.getContent(), new TypeReference>() - { - }); - } - catch (NoTaskLocationException e) { - return ImmutableMap.of(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - public boolean setEndOffsets( - final String id, - final Map endOffsets, - final boolean finalize - ) throws IOException - { - log.debug("SetEndOffsets task[%s] endOffsets[%s] finalize[%s]", id, endOffsets, finalize); - - try { - final FullResponseHolder response = submitJsonRequest( - id, - HttpMethod.POST, - "offsets/end", - StringUtils.format("finish=%s", finalize), - serialize(endOffsets), - true - ); - return isSuccess(response); - } - catch (NoTaskLocationException e) { - return false; - } - } - - public ListenableFuture stopAsync(final String id, final boolean publish) - { - return doAsync(() -> stop(id, publish)); - } - - public ListenableFuture resumeAsync(final String id) - { - return doAsync(() -> resume(id)); - } - - public ListenableFuture> pauseAsync(final String id) - { - return doAsync(() -> pause(id)); - } - - public ListenableFuture getStatusAsync(final String id) - { - return doAsync(() -> getStatus(id)); - } - - public ListenableFuture getStartTimeAsync(final String id) - { - return doAsync(() -> getStartTime(id)); - } - - public ListenableFuture> getCurrentOffsetsAsync(final String id, final boolean retry) - { - return doAsync(() -> getCurrentOffsets(id, retry)); - } - - public ListenableFuture> getEndOffsetsAsync(final String id) - { - return doAsync(() -> getEndOffsets(id)); - } - - public ListenableFuture setEndOffsetsAsync( - final String id, - final Map endOffsets, - final boolean finalize - ) - { - return doAsync(() -> setEndOffsets(id, endOffsets, finalize)); - } - - public ListenableFuture> getMovingAveragesAsync(final String id) - { - return doAsync(() -> getMovingAverages(id)); + return mapper.getTypeFactory().constructMapType(mapType, Integer.class, Long.class); } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java index d7f7f61be4c9..b435facb4df1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java @@ -23,24 +23,23 @@ import com.google.inject.Inject; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.guice.annotations.Json; +import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.common.TaskInfoProvider; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; -public class KafkaIndexTaskClientFactory implements IndexTaskClientFactory +public class KafkaIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory { - private HttpClient httpClient; - private ObjectMapper mapper; - @Inject public KafkaIndexTaskClientFactory( @EscalatedGlobal HttpClient httpClient, @Json ObjectMapper mapper ) { - this.httpClient = httpClient; - this.mapper = mapper; + super( + httpClient, + mapper + ); } @Override @@ -53,8 +52,8 @@ public KafkaIndexTaskClient build( ) { return new KafkaIndexTaskClient( - httpClient, - mapper, + getHttpClient(), + getMapper(), taskInfoProvider, dataSource, numThreads, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java index fd978a59387d..1c7c9219ddb8 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java @@ -20,9 +20,9 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.stats.RowIngestionMeters; -import org.apache.druid.indexing.kafka.KafkaIndexTask.Status; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.firehose.ChatHandler; @@ -33,7 +33,7 @@ * This class is used by only {@link KafkaIndexTask}. We currently have two implementations of this interface, i.e., * {@link IncrementalPublishingKafkaIndexTaskRunner} and {@link LegacyKafkaIndexTaskRunner}. The latter one was used in * the versions prior to 0.12.0, but being kept to support rolling update from them. - * + *

* We don't have a good reason for having this interface except for better code maintenance for the latest kakfa * indexing algorithm. As a result, this interface can be removed in the future when {@link LegacyKafkaIndexTaskRunner} * is removed and it's no longer useful. @@ -50,13 +50,16 @@ public interface KafkaIndexTaskRunner extends ChatHandler @VisibleForTesting RowIngestionMeters getRowIngestionMeters(); + @VisibleForTesting - Status getStatus(); + SeekableStreamIndexTask.Status getStatus(); @VisibleForTesting Map getCurrentOffsets(); + @VisibleForTesting Map getEndOffsets(); + @VisibleForTesting Response setEndOffsets( Map offsets, @@ -65,6 +68,7 @@ Response setEndOffsets( @VisibleForTesting Response pause() throws InterruptedException; + @VisibleForTesting void resume() throws InterruptedException; } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java index a5b36e9e8e7e..eb468c026f27 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java @@ -22,16 +22,14 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexing.SeekableStream.SeekableStreamPartitions; import org.apache.druid.java.util.common.StringUtils; import java.util.Map; import java.util.Objects; -public class KafkaPartitions +public class KafkaPartitions extends SeekableStreamPartitions { - private final String topic; - private final Map partitionOffsetMap; @JsonCreator public KafkaPartitions( @@ -39,8 +37,10 @@ public KafkaPartitions( @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap ) { - this.topic = topic; - this.partitionOffsetMap = ImmutableMap.copyOf(partitionOffsetMap); + super( + topic, + partitionOffsetMap + ); // Validate partitionOffsetMap for (Map.Entry entry : partitionOffsetMap.entrySet()) { @@ -58,13 +58,13 @@ public KafkaPartitions( @JsonProperty public String getTopic() { - return topic; + return getId(); } @JsonProperty public Map getPartitionOffsetMap() { - return partitionOffsetMap; + return getPartitionSequenceMap(); } @Override @@ -77,22 +77,22 @@ public boolean equals(Object o) return false; } KafkaPartitions that = (KafkaPartitions) o; - return Objects.equals(topic, that.topic) && - Objects.equals(partitionOffsetMap, that.partitionOffsetMap); + return Objects.equals(getTopic(), that.getTopic()) && + Objects.equals(getPartitionOffsetMap(), that.getPartitionOffsetMap()); } @Override public int hashCode() { - return Objects.hash(topic, partitionOffsetMap); + return Objects.hash(getTopic(), getPartitionOffsetMap()); } @Override public String toString() { return "KafkaPartitions{" + - "topic='" + topic + '\'' + - ", partitionOffsetMap=" + partitionOffsetMap + + "topic='" + getTopic() + '\'' + + ", partitionOffsetMap=" + getPartitionOffsetMap() + '}'; } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java index fd5e666166b3..4b6f76a0a0b8 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java @@ -21,49 +21,21 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.SeekableStream.SeekableStreamTuningConfig; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.indexing.TuningConfig; -import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; import javax.annotation.Nullable; import java.io.File; -import java.util.Objects; -public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig +public class KafkaTuningConfig extends SeekableStreamTuningConfig { - private static final int DEFAULT_MAX_ROWS_PER_SEGMENT = 5_000_000; - private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false; - - private final int maxRowsInMemory; - private final long maxBytesInMemory; - private final int maxRowsPerSegment; - @Nullable - private final Long maxTotalRows; - private final Period intermediatePersistPeriod; - private final File basePersistDirectory; - @Deprecated - private final int maxPendingPersists; - private final IndexSpec indexSpec; - private final boolean reportParseExceptions; - private final long handoffConditionTimeout; - private final boolean resetOffsetAutomatically; - @Nullable - private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; - private final Period intermediateHandoffPeriod; - - private final boolean logParseExceptions; - private final int maxParseExceptions; - private final int maxSavedParseExceptions; - @JsonCreator public KafkaTuningConfig( @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, - @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows, @JsonProperty("intermediatePersistPeriod") @Nullable Period intermediatePersistPeriod, @JsonProperty("basePersistDirectory") @Nullable File basePersistDirectory, @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, @@ -80,199 +52,12 @@ public KafkaTuningConfig( @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions ) { - // Cannot be a static because default basePersistDirectory is unique per-instance - final RealtimeTuningConfig defaults = RealtimeTuningConfig.makeDefaultTuningConfig(basePersistDirectory); - - this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; - this.maxRowsPerSegment = maxRowsPerSegment == null ? DEFAULT_MAX_ROWS_PER_SEGMENT : maxRowsPerSegment; - // initializing this to 0, it will be lazily initialized to a value - // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) - this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; - this.maxTotalRows = maxTotalRows; - this.intermediatePersistPeriod = intermediatePersistPeriod == null - ? defaults.getIntermediatePersistPeriod() - : intermediatePersistPeriod; - this.basePersistDirectory = defaults.getBasePersistDirectory(); - this.maxPendingPersists = 0; - this.indexSpec = indexSpec == null ? defaults.getIndexSpec() : indexSpec; - this.reportParseExceptions = reportParseExceptions == null - ? defaults.isReportParseExceptions() - : reportParseExceptions; - this.handoffConditionTimeout = handoffConditionTimeout == null - ? defaults.getHandoffConditionTimeout() - : handoffConditionTimeout; - this.resetOffsetAutomatically = resetOffsetAutomatically == null - ? DEFAULT_RESET_OFFSET_AUTOMATICALLY - : resetOffsetAutomatically; - this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; - this.intermediateHandoffPeriod = intermediateHandoffPeriod == null - ? new Period().withDays(Integer.MAX_VALUE) - : intermediateHandoffPeriod; - - if (this.reportParseExceptions) { - this.maxParseExceptions = 0; - this.maxSavedParseExceptions = maxSavedParseExceptions == null ? 0 : Math.min(1, maxSavedParseExceptions); - } else { - this.maxParseExceptions = maxParseExceptions == null ? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS : maxParseExceptions; - this.maxSavedParseExceptions = maxSavedParseExceptions == null - ? TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS - : maxSavedParseExceptions; - } - this.logParseExceptions = logParseExceptions == null ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS : logParseExceptions; - } - - public static KafkaTuningConfig copyOf(KafkaTuningConfig config) - { - return new KafkaTuningConfig( - config.maxRowsInMemory, - config.maxBytesInMemory, - config.maxRowsPerSegment, - config.maxTotalRows, - config.intermediatePersistPeriod, - config.basePersistDirectory, - config.maxPendingPersists, - config.indexSpec, - true, - config.reportParseExceptions, - config.handoffConditionTimeout, - config.resetOffsetAutomatically, - config.segmentWriteOutMediumFactory, - config.intermediateHandoffPeriod, - config.logParseExceptions, - config.maxParseExceptions, - config.maxSavedParseExceptions - ); - } - - @Override - @JsonProperty - public int getMaxRowsInMemory() - { - return maxRowsInMemory; - } - - @Override - @JsonProperty - public long getMaxBytesInMemory() - { - return maxBytesInMemory; - } - - @Override - @JsonProperty - public int getMaxRowsPerSegment() - { - return maxRowsPerSegment; - } - - - @JsonProperty - @Override - @Nullable - public Long getMaxTotalRows() - { - return maxTotalRows; - } - - @Override - @JsonProperty - public Period getIntermediatePersistPeriod() - { - return intermediatePersistPeriod; - } - - @Override - @JsonProperty - public File getBasePersistDirectory() - { - return basePersistDirectory; - } - - @Override - @JsonProperty - @Deprecated - public int getMaxPendingPersists() - { - return maxPendingPersists; - } - - @Override - @JsonProperty - public IndexSpec getIndexSpec() - { - return indexSpec; - } - - /** - * Always returns true, doesn't affect the version being built. - */ - @Deprecated - @JsonProperty - public boolean getBuildV9Directly() - { - return true; - } - - @Override - @JsonProperty - public boolean isReportParseExceptions() - { - return reportParseExceptions; - } - - @JsonProperty - public long getHandoffConditionTimeout() - { - return handoffConditionTimeout; - } - - @JsonProperty - public boolean isResetOffsetAutomatically() - { - return resetOffsetAutomatically; - } - - @Override - @JsonProperty - @Nullable - public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() - { - return segmentWriteOutMediumFactory; - } - - @JsonProperty - public Period getIntermediateHandoffPeriod() - { - return intermediateHandoffPeriod; - } - - @JsonProperty - public boolean isLogParseExceptions() - { - return logParseExceptions; - } - - @JsonProperty - public int getMaxParseExceptions() - { - return maxParseExceptions; - } - - @JsonProperty - public int getMaxSavedParseExceptions() - { - return maxSavedParseExceptions; - } - - public KafkaTuningConfig withBasePersistDirectory(File dir) - { - return new KafkaTuningConfig( + super( maxRowsInMemory, maxBytesInMemory, maxRowsPerSegment, - maxTotalRows, intermediatePersistPeriod, - dir, + basePersistDirectory, maxPendingPersists, indexSpec, true, @@ -288,76 +73,72 @@ public KafkaTuningConfig withBasePersistDirectory(File dir) } @Override - public boolean equals(Object o) + public KafkaTuningConfig copyOf() { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - KafkaTuningConfig that = (KafkaTuningConfig) o; - return maxRowsInMemory == that.maxRowsInMemory && - maxRowsPerSegment == that.maxRowsPerSegment && - maxBytesInMemory == that.maxBytesInMemory && - Objects.equals(maxTotalRows, that.maxTotalRows) && - maxPendingPersists == that.maxPendingPersists && - reportParseExceptions == that.reportParseExceptions && - handoffConditionTimeout == that.handoffConditionTimeout && - resetOffsetAutomatically == that.resetOffsetAutomatically && - Objects.equals(intermediatePersistPeriod, that.intermediatePersistPeriod) && - Objects.equals(basePersistDirectory, that.basePersistDirectory) && - Objects.equals(indexSpec, that.indexSpec) && - Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory) && - Objects.equals(intermediateHandoffPeriod, that.intermediateHandoffPeriod) && - logParseExceptions == that.logParseExceptions && - maxParseExceptions == that.maxParseExceptions && - maxSavedParseExceptions == that.maxSavedParseExceptions; + return new KafkaTuningConfig( + getMaxRowsInMemory(), + getMaxBytesInMemory(), + getMaxRowsPerSegment(), + getIntermediatePersistPeriod(), + getBasePersistDirectory(), + getMaxPendingPersists(), + getIndexSpec(), + true, + isReportParseExceptions(), + getHandoffConditionTimeout(), + isResetOffsetAutomatically(), + getSegmentWriteOutMediumFactory(), + getIntermediateHandoffPeriod(), + isLogParseExceptions(), + getMaxParseExceptions(), + getMaxSavedParseExceptions() + ); } @Override - public int hashCode() + public KafkaTuningConfig withBasePersistDirectory(File dir) { - return Objects.hash( - maxRowsInMemory, - maxRowsPerSegment, - maxBytesInMemory, - maxTotalRows, - intermediatePersistPeriod, - basePersistDirectory, - maxPendingPersists, - indexSpec, - reportParseExceptions, - handoffConditionTimeout, - resetOffsetAutomatically, - segmentWriteOutMediumFactory, - intermediateHandoffPeriod, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions + return new KafkaTuningConfig( + getMaxRowsInMemory(), + getMaxBytesInMemory(), + getMaxRowsPerSegment(), + getIntermediatePersistPeriod(), + dir, + getMaxPendingPersists(), + getIndexSpec(), + true, + isReportParseExceptions(), + getHandoffConditionTimeout(), + isResetOffsetAutomatically(), + getSegmentWriteOutMediumFactory(), + getIntermediateHandoffPeriod(), + isLogParseExceptions(), + getMaxParseExceptions(), + getMaxSavedParseExceptions() ); } + @Override public String toString() { return "KafkaTuningConfig{" + - "maxRowsInMemory=" + maxRowsInMemory + - ", maxRowsPerSegment=" + maxRowsPerSegment + - ", maxTotalRows=" + maxTotalRows + - ", maxBytesInMemory=" + maxBytesInMemory + - ", intermediatePersistPeriod=" + intermediatePersistPeriod + - ", basePersistDirectory=" + basePersistDirectory + - ", maxPendingPersists=" + maxPendingPersists + - ", indexSpec=" + indexSpec + - ", reportParseExceptions=" + reportParseExceptions + - ", handoffConditionTimeout=" + handoffConditionTimeout + - ", resetOffsetAutomatically=" + resetOffsetAutomatically + - ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + - ", intermediateHandoffPeriod=" + intermediateHandoffPeriod + - ", logParseExceptions=" + logParseExceptions + - ", maxParseExceptions=" + maxParseExceptions + - ", maxSavedParseExceptions=" + maxSavedParseExceptions + + "maxRowsInMemory=" + getMaxRowsInMemory() + + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + + ", maxBytesInMemory=" + getMaxBytesInMemory() + + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + + ", basePersistDirectory=" + getBasePersistDirectory() + + ", maxPendingPersists=" + getMaxPendingPersists() + + ", indexSpec=" + getIndexSpec() + + ", reportParseExceptions=" + isReportParseExceptions() + + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + + ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() + + ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() + + ", logParseExceptions=" + isLogParseExceptions() + + ", maxParseExceptions=" + getMaxParseExceptions() + + ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + '}'; } + } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 88dfe70ef7b5..dc96bbd630f4 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -35,6 +35,7 @@ import org.apache.druid.discovery.NodeType; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask.Status; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; @@ -46,7 +47,6 @@ import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.RealtimeIndexTask; -import org.apache.druid.indexing.kafka.KafkaIndexTask.Status; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -530,9 +530,9 @@ public void run() ).get(); final List publishedSegments = published.getSegments() - .stream() - .map(DataSegment::getIdentifier) - .collect(Collectors.toList()); + .stream() + .map(DataSegment::getIdentifier) + .collect(Collectors.toList()); log.info( "Published segments[%s] with metadata[%s].", diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index fedc77f5e89a..765cfd3dcf9e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -44,6 +44,8 @@ import org.apache.commons.codec.digest.DigestUtils; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask; +import org.apache.druid.indexing.SeekableStream.supervisor.SeekableStreamTaskReportData; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.RealtimeIndexTask; @@ -291,9 +293,9 @@ public KafkaSupervisor( this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.dataSource = spec.getDataSchema().getDataSource(); - this.ioConfig = spec.getIoConfig(); - this.tuningConfig = spec.getTuningConfig(); - this.taskTuningConfig = KafkaTuningConfig.copyOf(this.tuningConfig); + this.ioConfig = (KafkaSupervisorIOConfig) spec.getIoConfig(); + this.tuningConfig = (KafkaSupervisorTuningConfig) spec.getTuningConfig(); + this.taskTuningConfig = (KafkaTuningConfig) this.tuningConfig.copyOf(); this.supervisorId = StringUtils.format("KafkaSupervisor-%s", dataSource); this.exec = Execs.singleThreaded(supervisorId); this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d"); @@ -1117,14 +1119,14 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti futureTaskIds.add(taskId); futures.add( Futures.transform( - taskClient.getStatusAsync(taskId), new Function() + taskClient.getStatusAsync(taskId), new Function() { @Override - public Boolean apply(KafkaIndexTask.Status status) + public Boolean apply(SeekableStreamIndexTask.Status status) { try { log.debug("Task [%s], status [%s]", taskId, status); - if (status == KafkaIndexTask.Status.PUBLISHING) { + if (status == SeekableStreamIndexTask.Status.PUBLISHING) { kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap().keySet().forEach( partition -> addDiscoveredTaskToPendingCompletionTaskGroups( getTaskGroupIdForPartition(partition), @@ -2207,7 +2209,7 @@ private SupervisorReport generateReport(boolean in includeOffsets ? currentOffsets : null, startTime, remainingSeconds, - TaskReportData.TaskType.ACTIVE, + SeekableStreamTaskReportData.TaskType.ACTIVE, includeOffsets ? getLagPerPartition(currentOffsets) : null ) ); @@ -2234,7 +2236,7 @@ private SupervisorReport generateReport(boolean in includeOffsets ? currentOffsets : null, startTime, remainingSeconds, - TaskReportData.TaskType.PUBLISHING, + SeekableStreamTaskReportData.TaskType.PUBLISHING, null ) ); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java index 44c2bb2d6f73..31b6315f60ec 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java @@ -21,32 +21,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; +import org.apache.druid.indexing.SeekableStream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.java.util.common.StringUtils; -import org.joda.time.Duration; import org.joda.time.Period; import java.util.Map; -public class KafkaSupervisorIOConfig +public class KafkaSupervisorIOConfig extends SeekableStreamSupervisorIOConfig { public static final String BOOTSTRAP_SERVERS_KEY = "bootstrap.servers"; - public static final String TRUST_STORE_PASSWORD_KEY = "ssl.truststore.password"; - public static final String KEY_STORE_PASSWORD_KEY = "ssl.keystore.password"; - public static final String KEY_PASSWORD_KEY = "ssl.key.password"; - private final String topic; - private final Integer replicas; - private final Integer taskCount; - private final Duration taskDuration; - private final Map consumerProperties; - private final Duration startDelay; - private final Duration period; - private final boolean useEarliestOffset; - private final Duration completionTimeout; - private final Optional lateMessageRejectionPeriod; - private final Optional earlyMessageRejectionPeriod; + private final Map consumerProperties; private final boolean skipOffsetGaps; @JsonCreator @@ -55,7 +41,7 @@ public KafkaSupervisorIOConfig( @JsonProperty("replicas") Integer replicas, @JsonProperty("taskCount") Integer taskCount, @JsonProperty("taskDuration") Period taskDuration, - @JsonProperty("consumerProperties") Map consumerProperties, + @JsonProperty("consumerProperties") Map consumerProperties, @JsonProperty("startDelay") Period startDelay, @JsonProperty("period") Period period, @JsonProperty("useEarliestOffset") Boolean useEarliestOffset, @@ -65,93 +51,43 @@ public KafkaSupervisorIOConfig( @JsonProperty("skipOffsetGaps") Boolean skipOffsetGaps ) { - this.topic = Preconditions.checkNotNull(topic, "topic"); + super( + Preconditions.checkNotNull(topic, "topic"), + replicas, + taskCount, + taskDuration, + startDelay, + period, + useEarliestOffset, + completionTimeout, + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod + ); + this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); Preconditions.checkNotNull( consumerProperties.get(BOOTSTRAP_SERVERS_KEY), StringUtils.format("consumerProperties must contain entry for [%s]", BOOTSTRAP_SERVERS_KEY) ); - - this.replicas = replicas != null ? replicas : 1; - this.taskCount = taskCount != null ? taskCount : 1; - this.taskDuration = defaultDuration(taskDuration, "PT1H"); - this.startDelay = defaultDuration(startDelay, "PT5S"); - this.period = defaultDuration(period, "PT30S"); - this.useEarliestOffset = useEarliestOffset != null ? useEarliestOffset : false; - this.completionTimeout = defaultDuration(completionTimeout, "PT30M"); - this.lateMessageRejectionPeriod = lateMessageRejectionPeriod == null - ? Optional.absent() - : Optional.of(lateMessageRejectionPeriod.toStandardDuration()); - this.earlyMessageRejectionPeriod = earlyMessageRejectionPeriod == null - ? Optional.absent() - : Optional.of(earlyMessageRejectionPeriod.toStandardDuration()); this.skipOffsetGaps = skipOffsetGaps != null ? skipOffsetGaps : false; } @JsonProperty public String getTopic() { - return topic; - } - - @JsonProperty - public Integer getReplicas() - { - return replicas; + return getId(); } @JsonProperty - public Integer getTaskCount() - { - return taskCount; - } - - @JsonProperty - public Duration getTaskDuration() - { - return taskDuration; - } - - @JsonProperty - public Map getConsumerProperties() + public Map getConsumerProperties() { return consumerProperties; } - @JsonProperty - public Duration getStartDelay() - { - return startDelay; - } - - @JsonProperty - public Duration getPeriod() - { - return period; - } - @JsonProperty public boolean isUseEarliestOffset() { - return useEarliestOffset; - } - - @JsonProperty - public Duration getCompletionTimeout() - { - return completionTimeout; - } - - @JsonProperty - public Optional getEarlyMessageRejectionPeriod() - { - return earlyMessageRejectionPeriod; - } - - @JsonProperty - public Optional getLateMessageRejectionPeriod() - { - return lateMessageRejectionPeriod; + return isUseEarliestSequenceNumber(); } @JsonProperty @@ -164,22 +100,18 @@ public boolean isSkipOffsetGaps() public String toString() { return "KafkaSupervisorIOConfig{" + - "topic='" + topic + '\'' + - ", replicas=" + replicas + - ", taskCount=" + taskCount + - ", taskDuration=" + taskDuration + + "topic='" + getTopic() + '\'' + + ", replicas=" + getReplicas() + + ", taskCount=" + getTaskCount() + + ", taskDuration=" + getTaskDuration() + ", consumerProperties=" + consumerProperties + - ", startDelay=" + startDelay + - ", period=" + period + - ", useEarliestOffset=" + useEarliestOffset + - ", completionTimeout=" + completionTimeout + - ", lateMessageRejectionPeriod=" + lateMessageRejectionPeriod + + ", startDelay=" + getStartDelay() + + ", period=" + getPeriod() + + ", useEarliestOffset=" + isUseEarliestOffset() + + ", completionTimeout=" + getCompletionTimeout() + + ", lateMessageRejectionPeriod=" + getLateMessageRejectionPeriod() + ", skipOffsetGaps=" + skipOffsetGaps + '}'; } - private static Duration defaultDuration(final Period period, final String theDefault) - { - return (period == null ? new Period(theDefault) : period).toStandardDuration(); - } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java index d9533a37fb24..b50e59d08720 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java @@ -19,30 +19,17 @@ package org.apache.druid.indexing.kafka.supervisor; -import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.java.util.common.IAE; +import org.apache.druid.indexing.SeekableStream.supervisor.SeekableStreamSupervisorReportPayload; import org.joda.time.DateTime; import javax.annotation.Nullable; -import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; -public class KafkaSupervisorReportPayload +public class KafkaSupervisorReportPayload extends SeekableStreamSupervisorReportPayload { - private final String dataSource; - private final String topic; - private final int partitions; - private final int replicas; - private final long durationSeconds; - private final List activeTasks; - private final List publishingTasks; - private final Map latestOffsets; - private final Map minimumLag; - private final Long aggregateLag; - private final DateTime offsetsLastUpdated; - private final boolean suspended; public KafkaSupervisorReportPayload( String dataSource, @@ -53,126 +40,59 @@ public KafkaSupervisorReportPayload( @Nullable Map latestOffsets, @Nullable Map minimumLag, @Nullable Long aggregateLag, - @Nullable DateTime offsetsLastUpdated, - boolean suspended + @Nullable DateTime offsetsLastUpdated ) { - this.dataSource = dataSource; - this.topic = topic; - this.partitions = partitions; - this.replicas = replicas; - this.durationSeconds = durationSeconds; - this.activeTasks = new ArrayList<>(); - this.publishingTasks = new ArrayList<>(); - this.latestOffsets = latestOffsets; - this.minimumLag = minimumLag; - this.aggregateLag = aggregateLag; - this.offsetsLastUpdated = offsetsLastUpdated; - this.suspended = suspended; + super( + dataSource, + topic, + partitions, + replicas, + durationSeconds, + latestOffsets, + minimumLag, + aggregateLag, + offsetsLastUpdated + ); } - public void addTask(TaskReportData data) - { - if (data.getType().equals(TaskReportData.TaskType.ACTIVE)) { - activeTasks.add(data); - } else if (data.getType().equals(TaskReportData.TaskType.PUBLISHING)) { - publishingTasks.add(data); - } else { - throw new IAE("Unknown task type [%s]", data.getType().name()); - } - } - - @JsonProperty - public String getDataSource() - { - return dataSource; - } @JsonProperty public String getTopic() { - return topic; - } - - @JsonProperty - public int getPartitions() - { - return partitions; - } - - @JsonProperty - public int getReplicas() - { - return replicas; - } - - @JsonProperty - public long getDurationSeconds() - { - return durationSeconds; + return getId(); } + @Override @JsonProperty public List getActiveTasks() { - return activeTasks; + return super.getActiveTasks().stream().map(e -> (TaskReportData) e).collect(Collectors.toList()); } + @Override @JsonProperty public List getPublishingTasks() { - return publishingTasks; - } - - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getLatestOffsets() - { - return latestOffsets; - } - - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getMinimumLag() - { - return minimumLag; - } - - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) - public Long getAggregateLag() - { - return aggregateLag; - } - - @JsonProperty - public DateTime getOffsetsLastUpdated() - { - return offsetsLastUpdated; - } - - @JsonProperty - public boolean getSuspended() - { - return suspended; + return super.getPublishingTasks().stream().map(e -> (TaskReportData) e).collect(Collectors.toList()); } @Override public String toString() { return "{" + - "dataSource='" + dataSource + '\'' + - ", topic='" + topic + '\'' + - ", partitions=" + partitions + - ", replicas=" + replicas + - ", durationSeconds=" + durationSeconds + - ", active=" + activeTasks + - ", publishing=" + publishingTasks + - (latestOffsets != null ? ", latestOffsets=" + latestOffsets : "") + - (minimumLag != null ? ", minimumLag=" + minimumLag : "") + - (aggregateLag != null ? ", aggregateLag=" + aggregateLag : "") + - (offsetsLastUpdated != null ? ", offsetsLastUpdated=" + offsetsLastUpdated : "") + - ", suspended=" + suspended + + "dataSource='" + getDataSource() + '\'' + + ", topic='" + getTopic() + '\'' + + ", partitions=" + getPartitions() + + ", replicas=" + getReplicas() + + ", durationSeconds=" + getDurationSeconds() + + ", active=" + getActiveTasks() + + ", publishing=" + getPublishingTasks() + + (getLatestOffsets() != null ? ", latestOffsets=" + getLatestOffsets() : "") + + (getMinimumLag() != null ? ", minimumLag=" + getMinimumLag() : "") + + (getAggregateLag() != null ? ", aggregateLag=" + getAggregateLag() : "") + + (getOffsetsLastUpdated() != null ? ", offsetsLastUpdated=" + getOffsetsLastUpdated() : "") + '}'; } + } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index 28dab02c56e2..a373cbcc30a1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -23,16 +23,15 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import org.apache.druid.guice.annotations.Json; +import org.apache.druid.indexing.SeekableStream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; -import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; @@ -40,30 +39,14 @@ import java.util.List; import java.util.Map; -public class KafkaSupervisorSpec implements SupervisorSpec +public class KafkaSupervisorSpec extends SeekableStreamSupervisorSpec { - private final DataSchema dataSchema; - private final KafkaSupervisorTuningConfig tuningConfig; - private final KafkaSupervisorIOConfig ioConfig; - private final Map context; - - private final TaskStorage taskStorage; - private final TaskMaster taskMaster; - private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; - private final KafkaIndexTaskClientFactory kafkaIndexTaskClientFactory; - private final ObjectMapper mapper; - private final ServiceEmitter emitter; - private final DruidMonitorSchedulerConfig monitorSchedulerConfig; - private final RowIngestionMetersFactory rowIngestionMetersFactory; - private final boolean suspended; - @JsonCreator public KafkaSupervisorSpec( @JsonProperty("dataSchema") DataSchema dataSchema, @JsonProperty("tuningConfig") KafkaSupervisorTuningConfig tuningConfig, @JsonProperty("ioConfig") KafkaSupervisorIOConfig ioConfig, @JsonProperty("context") Map context, - @JsonProperty("suspended") Boolean suspended, @JacksonInject TaskStorage taskStorage, @JacksonInject TaskMaster taskMaster, @JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, @@ -74,93 +57,45 @@ public KafkaSupervisorSpec( @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory ) { - this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); - this.tuningConfig = tuningConfig != null - ? tuningConfig - : new KafkaSupervisorTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); - this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); - this.context = context; - - this.taskStorage = taskStorage; - this.taskMaster = taskMaster; - this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; - this.kafkaIndexTaskClientFactory = kafkaIndexTaskClientFactory; - this.mapper = mapper; - this.emitter = emitter; - this.monitorSchedulerConfig = monitorSchedulerConfig; - this.rowIngestionMetersFactory = rowIngestionMetersFactory; - this.suspended = suspended != null ? suspended : false; - } - - @JsonProperty - public DataSchema getDataSchema() - { - return dataSchema; - } - - @JsonProperty - public KafkaSupervisorTuningConfig getTuningConfig() - { - return tuningConfig; - } - - @JsonProperty - public KafkaSupervisorIOConfig getIoConfig() - { - return ioConfig; - } - - @JsonProperty - public Map getContext() - { - return context; - } - - @Override - @JsonProperty("suspended") - public boolean isSuspended() - { - return suspended; - } - - public ServiceEmitter getEmitter() - { - return emitter; - } - - @Override - public String getId() - { - return dataSchema.getDataSource(); - } - - public DruidMonitorSchedulerConfig getMonitorSchedulerConfig() - { - return monitorSchedulerConfig; + super( + dataSchema, + tuningConfig != null + ? tuningConfig + : new KafkaSupervisorTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ), + ioConfig, + context, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + kafkaIndexTaskClientFactory, + mapper, + emitter, + monitorSchedulerConfig, + rowIngestionMetersFactory + ); } @Override @@ -170,7 +105,7 @@ public Supervisor createSupervisor() taskStorage, taskMaster, indexerMetadataStorageCoordinator, - kafkaIndexTaskClientFactory, + (KafkaIndexTaskClientFactory) indexTaskClientFactory, mapper, this, rowIngestionMetersFactory @@ -187,40 +122,9 @@ public List getDataSources() public String toString() { return "KafkaSupervisorSpec{" + - "dataSchema=" + dataSchema + - ", tuningConfig=" + tuningConfig + - ", ioConfig=" + ioConfig + + "dataSchema=" + getDataSchema() + + ", tuningConfig=" + getTuningConfig() + + ", ioConfig=" + getIoConfig() + '}'; } - - @Override - public KafkaSupervisorSpec createSuspendedSpec() - { - return toggleSuspend(true); - } - - @Override - public KafkaSupervisorSpec createRunningSpec() - { - return toggleSuspend(false); - } - - private KafkaSupervisorSpec toggleSuspend(boolean suspend) - { - return new KafkaSupervisorSpec( - dataSchema, - tuningConfig, - ioConfig, - context, - suspend, - taskStorage, - taskMaster, - indexerMetadataStorageCoordinator, - kafkaIndexTaskClientFactory, - mapper, - emitter, - monitorSchedulerConfig, - rowIngestionMetersFactory - ); - } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index 4715f336f759..8e72ff3391e0 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.kafka.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.SeekableStream.supervisor.SeekableStreamSupervisorTuningConfig; import org.apache.druid.indexing.kafka.KafkaTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.TuningConfigs; @@ -30,8 +31,9 @@ import javax.annotation.Nullable; import java.io.File; -public class KafkaSupervisorTuningConfig extends KafkaTuningConfig +public class KafkaSupervisorTuningConfig extends KafkaTuningConfig implements SeekableStreamSupervisorTuningConfig { + private final Integer workerThreads; private final Integer chatThreads; private final Long chatRetries; @@ -85,39 +87,43 @@ public KafkaSupervisorTuningConfig( maxParseExceptions, maxSavedParseExceptions ); - this.workerThreads = workerThreads; this.chatThreads = chatThreads; this.chatRetries = (chatRetries != null ? chatRetries : 8); - this.httpTimeout = defaultDuration(httpTimeout, "PT10S"); - this.shutdownTimeout = defaultDuration(shutdownTimeout, "PT80S"); - this.offsetFetchPeriod = defaultDuration(offsetFetchPeriod, "PT30S"); + this.httpTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(httpTimeout, "PT10S"); + this.shutdownTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(shutdownTimeout, "PT80S"); + this.offsetFetchPeriod = SeekableStreamSupervisorTuningConfig.defaultDuration(offsetFetchPeriod, "PT30S"); } + @Override @JsonProperty public Integer getWorkerThreads() { return workerThreads; } + @Override @JsonProperty public Integer getChatThreads() { return chatThreads; } + @Override @JsonProperty public Long getChatRetries() { return chatRetries; } + @Override @JsonProperty public Duration getHttpTimeout() { return httpTimeout; } + @Override @JsonProperty public Duration getShutdownTimeout() { @@ -159,8 +165,5 @@ public String toString() '}'; } - private static Duration defaultDuration(final Period period, final String theDefault) - { - return (period == null ? new Period(theDefault) : period).toStandardDuration(); - } + } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/TaskReportData.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/TaskReportData.java index 85b05c7a5f0d..cf9bab8d0d60 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/TaskReportData.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/TaskReportData.java @@ -21,24 +21,15 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.SeekableStream.supervisor.SeekableStreamTaskReportData; import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.Map; -public class TaskReportData +public class TaskReportData extends SeekableStreamTaskReportData { - public enum TaskType - { - ACTIVE, PUBLISHING, UNKNOWN - } - private final String id; - private final Map startingOffsets; - private final DateTime startTime; - private final Long remainingSeconds; - private final TaskType type; - private final Map currentOffsets; private final Map lag; public TaskReportData( @@ -51,51 +42,16 @@ public TaskReportData( @Nullable Map lag ) { - this.id = id; - this.startingOffsets = startingOffsets; - this.currentOffsets = currentOffsets; - this.startTime = startTime; - this.remainingSeconds = remainingSeconds; - this.type = type; - this.lag = lag; - } - - @JsonProperty - public String getId() - { - return id; - } + super( + id, + startingOffsets, + currentOffsets, + startTime, + remainingSeconds, + type + ); - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getStartingOffsets() - { - return startingOffsets; - } - - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getCurrentOffsets() - { - return currentOffsets; - } - - @JsonProperty - public DateTime getStartTime() - { - return startTime; - } - - @JsonProperty - public Long getRemainingSeconds() - { - return remainingSeconds; - } - - @JsonProperty - public TaskType getType() - { - return type; + this.lag = lag; } @JsonProperty @@ -109,11 +65,11 @@ public Map getLag() public String toString() { return "{" + - "id='" + id + '\'' + - (startingOffsets != null ? ", startingOffsets=" + startingOffsets : "") + - (currentOffsets != null ? ", currentOffsets=" + currentOffsets : "") + - ", startTime=" + startTime + - ", remainingSeconds=" + remainingSeconds + + "id='" + getId() + '\'' + + (getStartingOffsets() != null ? ", startingOffsets=" + getStartingOffsets() : "") + + (getCurrentOffsets() != null ? ", currentOffsets=" + getCurrentOffsets() : "") + + ", startTime=" + getStartTime() + + ", remainingSeconds=" + getRemainingSeconds() + (lag != null ? ", lag=" + lag : "") + '}'; } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaTuningConfigTest.java index 69cf186a3c3b..0acea38b1747 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaTuningConfigTest.java @@ -124,7 +124,7 @@ public void testCopyOf() null, null ); - KafkaTuningConfig copy = KafkaTuningConfig.copyOf(original); + KafkaTuningConfig copy = original.copyOf(); Assert.assertEquals(1, copy.getMaxRowsInMemory()); Assert.assertEquals(2, copy.getMaxRowsPerSegment()); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 237912346ec9..cd126070baa4 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -39,6 +39,7 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask.Status; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -50,7 +51,6 @@ import org.apache.druid.indexing.kafka.KafkaIndexTaskClient; import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; import org.apache.druid.indexing.kafka.KafkaPartitions; -import org.apache.druid.indexing.kafka.KafkaTuningConfig; import org.apache.druid.indexing.kafka.test.TestBroker; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -282,7 +282,7 @@ public void testNoInitialState() throws Exception KafkaIndexTask task = captured.getValue(); Assert.assertEquals(dataSchema, task.getDataSchema()); - Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), task.getTuningConfig()); + Assert.assertEquals(tuningConfig.copyOf(), task.getTuningConfig()); KafkaIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -643,7 +643,7 @@ public void testKillIncompatibleTasks() throws Exception 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(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( @@ -739,7 +739,7 @@ public void testKillBadPartitionAssignment() throws Exception expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); expect(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); - expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( @@ -783,7 +783,7 @@ public void testRequeueTaskWhenFailed() throws Exception expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( @@ -880,7 +880,7 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes(); expect(taskQueue.add(capture(captured))).andReturn(true); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( @@ -926,7 +926,7 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception expect(taskStorage.getStatus(runningTaskId)).andReturn(Optional.of(TaskStatus.running(runningTaskId))).anyTimes(); expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); - expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); + expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(Status.READING)); expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes(); expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); replay(taskStorage); @@ -962,7 +962,7 @@ public void testQueueNextTasksOnSuccess() throws Exception expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( @@ -984,7 +984,7 @@ public void testQueueNextTasksOnSuccess() throws Exception reset(taskStorage); reset(taskClient); - expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)) + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); TreeMap> checkpoints1 = new TreeMap<>(); @@ -1080,7 +1080,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception } expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); expect(taskClient.getStatusAsync(anyString())) - .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)) + .andReturn(Futures.immediateFuture(Status.READING)) .anyTimes(); expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) @@ -1119,7 +1119,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception for (Task task : captured.getValues()) { KafkaIndexTask kafkaIndexTask = (KafkaIndexTask) task; Assert.assertEquals(dataSchema, kafkaIndexTask.getDataSchema()); - Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), kafkaIndexTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.copyOf(), kafkaIndexTask.getTuningConfig()); KafkaIOConfig taskConfig = kafkaIndexTask.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); @@ -1165,7 +1165,7 @@ public void testDiscoverExistingPublishingTask() throws Exception null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); @@ -1204,7 +1204,7 @@ public void testDiscoverExistingPublishingTask() throws Exception KafkaIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.copyOf(), capturedTask.getTuningConfig()); KafkaIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -1257,7 +1257,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 2, 30L))); expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); @@ -1292,7 +1292,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() KafkaIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.copyOf(), capturedTask.getTuningConfig()); KafkaIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -1359,8 +1359,8 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 1L, 1, 2L, 2, 3L))); @@ -1461,7 +1461,7 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); expect(taskClient.getStatusAsync(task.getId())) - .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.NOT_STARTED)); + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)); expect(taskClient.getStartTimeAsync(task.getId())) .andReturn(Futures.immediateFailedFuture(new RuntimeException())); taskQueue.shutdown(task.getId()); @@ -1525,7 +1525,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception } expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); expect(taskClient.getStatusAsync(anyString())) - .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)) + .andReturn(Futures.immediateFuture(Status.READING)) .anyTimes(); expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) @@ -1604,7 +1604,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception } expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); expect(taskClient.getStatusAsync(anyString())) - .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)) + .andReturn(Futures.immediateFuture(Status.READING)) .anyTimes(); expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) @@ -1719,9 +1719,9 @@ public void testStopGracefully() throws Exception null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); - expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); @@ -1925,9 +1925,9 @@ public void testResetRunningTasks() throws Exception null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); - expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); @@ -2008,9 +2008,9 @@ public void testNoDataIngestionTasks() throws Exception 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)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); @@ -2098,12 +2098,12 @@ public void testCheckpointForInactiveTaskGroup() 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)); + expect( + indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KafkaDataSourceMetadata(null) + ).anyTimes(); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); final DateTime startTime = DateTimes.nowUtc(); expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); @@ -2276,7 +2276,7 @@ public void testCheckpointWithNullTaskGroupId() ).anyTimes(); taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); expect(taskClient.getStatusAsync(anyString())) - .andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)) + .andReturn(Futures.immediateFuture(Status.READING)) .anyTimes(); final TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L)); From 462fb10e1c79b38e3473165ea7d5989e40bd7705 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 12 Sep 2018 18:14:17 -0700 Subject: [PATCH 04/87] seekablestream bug fixes --- ...ementalPublishingKafkaIndexTaskRunner.java | 2 +- .../kafka/KafkaDataSourceMetadata.java | 2 +- .../druid/indexing/kafka/KafkaIOConfig.java | 2 +- .../druid/indexing/kafka/KafkaIndexTask.java | 2 +- .../indexing/kafka/KafkaIndexTaskClient.java | 6 ++-- .../kafka/KafkaIndexTaskClientFactory.java | 2 +- .../indexing/kafka/KafkaIndexTaskRunner.java | 2 +- .../druid/indexing/kafka/KafkaPartitions.java | 2 +- .../indexing/kafka/KafkaTuningConfig.java | 2 +- .../kafka/LegacyKafkaIndexTaskRunner.java | 2 +- .../kafka/supervisor/KafkaSupervisor.java | 4 +-- .../supervisor/KafkaSupervisorIOConfig.java | 2 +- .../KafkaSupervisorReportPayload.java | 2 +- .../kafka/supervisor/KafkaSupervisorSpec.java | 2 +- .../KafkaSupervisorTuningConfig.java | 2 +- .../kafka/supervisor/TaskReportData.java | 2 +- .../kafka/supervisor/KafkaSupervisorTest.java | 2 +- .../SeekableStream/common/RecordSupplier.java | 33 ----------------- .../SeekableStreamDataSourceMetadata.java | 2 +- .../SeekableStreamIOConfig.java | 12 +------ .../SeekableStreamIndexTask.java | 2 +- .../SeekableStreamIndexTaskClient.java | 22 +++++------- .../SeekableStreamIndexTaskClientFactory.java | 2 +- .../SeekableStreamIndexTaskRunner.java | 2 +- .../SeekableStreamPartitions.java | 2 +- .../SeekableStreamTuningConfig.java | 2 +- .../common/Record.java | 14 ++++---- .../seekablestream/common/RecordSupplier.java | 35 +++++++++++++++++++ .../common/SequenceNumberPlus.java | 2 +- .../common/StreamPartition.java | 12 +++---- .../supervisor/SeekableStreamSupervisor.java | 14 ++++---- .../SeekableStreamSupervisorIOConfig.java | 2 +- ...SeekableStreamSupervisorReportPayload.java | 2 +- .../SeekableStreamSupervisorSpec.java | 4 +-- .../SeekableStreamSupervisorTuningConfig.java | 4 +-- .../SeekableStreamTaskReportData.java | 2 +- 36 files changed, 99 insertions(+), 111 deletions(-) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/RecordSupplier.java rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/SeekableStreamDataSourceMetadata.java (97%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/SeekableStreamIOConfig.java (89%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/SeekableStreamIndexTask.java (99%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/SeekableStreamIndexTaskClient.java (95%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/SeekableStreamIndexTaskClientFactory.java (97%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/SeekableStreamIndexTaskRunner.java (97%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/SeekableStreamPartitions.java (97%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/SeekableStreamTuningConfig.java (99%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/common/Record.java (60%) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/common/SequenceNumberPlus.java (90%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/common/StreamPartition.java (76%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/supervisor/SeekableStreamSupervisor.java (99%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/supervisor/SeekableStreamSupervisorIOConfig.java (98%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/supervisor/SeekableStreamSupervisorReportPayload.java (98%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/supervisor/SeekableStreamSupervisorSpec.java (97%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/supervisor/SeekableStreamSupervisorTuningConfig.java (92%) rename indexing-service/src/main/java/org/apache/druid/indexing/{SeekableStream => seekablestream}/supervisor/SeekableStreamTaskReportData.java (97%) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 695f85998558..3c017636cc21 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -44,7 +44,7 @@ import org.apache.druid.discovery.NodeType; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask.Status; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java index 0452a5d83687..c7f0459b9c99 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; -import org.apache.druid.indexing.SeekableStream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.java.util.common.IAE; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java index 4f84c47c0650..5699a07658c2 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.indexing.SeekableStream.SeekableStreamIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 00a37431fae5..618abc386872 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -26,7 +26,7 @@ import com.google.common.collect.Lists; import org.apache.druid.data.input.InputRow; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import org.apache.druid.indexing.common.TaskToolbox; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java index 1162a251968c..390dfe93770d 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; +import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; @@ -55,8 +55,10 @@ public KafkaIndexTaskClient( } @Override - protected JavaType constructPartitionOffsetMapType(Class mapType) + protected JavaType constructMapType(Class mapType) { return mapper.getTypeFactory().constructMapType(mapType, Integer.class, Long.class); } + + } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java index b435facb4df1..b39f49c5555a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java @@ -23,7 +23,7 @@ import com.google.inject.Inject; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.guice.annotations.Json; -import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTaskClientFactory; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java index 1c7c9219ddb8..fae889156a8b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java @@ -20,7 +20,7 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.segment.realtime.appenderator.Appenderator; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java index eb468c026f27..a69ff0a6509d 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.indexing.SeekableStream.SeekableStreamPartitions; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.java.util.common.StringUtils; import java.util.Map; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java index 4b6f76a0a0b8..4a0321ec8f08 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.SeekableStream.SeekableStreamTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index dc96bbd630f4..dc6ac9746485 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -35,7 +35,7 @@ import org.apache.druid.discovery.NodeType; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask.Status; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 765cfd3dcf9e..5b6193a40d3e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -44,8 +44,8 @@ import org.apache.commons.codec.digest.DigestUtils; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask; -import org.apache.druid.indexing.SeekableStream.supervisor.SeekableStreamTaskReportData; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamTaskReportData; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.RealtimeIndexTask; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java index 31b6315f60ec..90ff600e3dfd 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.indexing.SeekableStream.supervisor.SeekableStreamSupervisorIOConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.java.util.common.StringUtils; import org.joda.time.Period; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java index b50e59d08720..b1647d182620 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java @@ -20,7 +20,7 @@ package org.apache.druid.indexing.kafka.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.SeekableStream.supervisor.SeekableStreamSupervisorReportPayload; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index a373cbcc30a1..bc058ba20eb5 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -25,7 +25,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import org.apache.druid.guice.annotations.Json; -import org.apache.druid.indexing.SeekableStream.supervisor.SeekableStreamSupervisorSpec; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index 8e72ff3391e0..2892c5a544a2 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -20,7 +20,7 @@ package org.apache.druid.indexing.kafka.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.SeekableStream.supervisor.SeekableStreamSupervisorTuningConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; import org.apache.druid.indexing.kafka.KafkaTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.TuningConfigs; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/TaskReportData.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/TaskReportData.java index cf9bab8d0d60..5f011d79ab0f 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/TaskReportData.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/TaskReportData.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.SeekableStream.supervisor.SeekableStreamTaskReportData; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamTaskReportData; import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index cd126070baa4..0ef426381508 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -39,7 +39,7 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask.Status; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/RecordSupplier.java deleted file mode 100644 index 6e886a4ae895..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/RecordSupplier.java +++ /dev/null @@ -1,33 +0,0 @@ -package org.apache.druid.indexing.SeekableStream.common; - -import java.io.Closeable; -import java.util.Set; -import java.util.concurrent.TimeoutException; - -/** - * The RecordSupplier interface is a wrapper for the incoming seekable data stream - * (i.e. Kafka consumer) - */ -public interface RecordSupplier extends Closeable -{ - void assign(Set> partitions); - - void seek(StreamPartition partition, SequenceType sequenceNumber); - - void seekAfter(StreamPartition partition, SequenceType sequenceNumber); - - void seekToEarliest(StreamPartition partition); - - void seekToLatest(StreamPartition partition); - - Record poll(long timeout); - - SequenceType getLatestSequenceNumber(StreamPartition partition) throws TimeoutException; - - SequenceType getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException; - - Set getPartitionIds(String streamName); - - @Override - void close(); -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java similarity index 97% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamDataSourceMetadata.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java index 4f5875eddce2..244429e15a3c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamDataSourceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.SeekableStream; +package org.apache.druid.indexing.seekablestream; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java similarity index 89% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIOConfig.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java index 48586b9b3d8e..4b132d7c36a8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.SeekableStream; +package org.apache.druid.indexing.seekablestream; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -59,16 +59,6 @@ public SeekableStreamIOConfig( this.useTransaction = useTransaction != null ? useTransaction : DEFAULT_USE_TRANSACTION; this.minimumMessageTime = Optional.fromNullable(minimumMessageTime); this.maximumMessageTime = Optional.fromNullable(maximumMessageTime); - - Preconditions.checkArgument( - startPartitions.getId().equals(endPartitions.getId()), - "start id and end id must match" - ); - - Preconditions.checkArgument( - startPartitions.getPartitionSequenceMap().keySet().equals(endPartitions.getPartitionSequenceMap().keySet()), - "start partition set and end partition set must match" - ); } @Nullable diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java similarity index 99% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTask.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index d69a8a0bd18c..a077a4e3d7e2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.SeekableStream; +package org.apache.druid.indexing.seekablestream; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java similarity index 95% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClient.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index ccb4f3e2037a..66b4e73404c4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -17,9 +17,9 @@ * under the License. */ -package org.apache.druid.indexing.SeekableStream; +package org.apache.druid.indexing.seekablestream; -import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.ListenableFuture; @@ -112,9 +112,7 @@ public Map pause(final String id) if (response.getStatus().equals(HttpResponseStatus.OK)) { log.info("Task [%s] paused successfully", id); - return deserialize(response.getContent(), new TypeReference>() - { - }); + return deserialize(response.getContent(), constructMapType(Map.class)); } while (true) { @@ -221,9 +219,7 @@ public Map getCurrentOffsets(final String id, final boolean retry) null, retry ); - return deserialize(response.getContent(), new TypeReference>() - { - }); + return deserialize(response.getContent(), constructMapType(Map.class)); } catch (NoTaskLocationException e) { return ImmutableMap.of(); @@ -240,9 +236,7 @@ public TreeMap> getCheckpoints(final String id, final boole final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "checkpoints", null, retry); return deserialize( response.getContent(), - new TypeReference>>() - { - } + constructMapType(TreeMap.class) ); } catch (NoTaskLocationException e) { @@ -267,9 +261,7 @@ public Map getEndOffsets(final String id) try { final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "offsets/end", null, true); - return deserialize(response.getContent(), new TypeReference>() - { - }); + return deserialize(response.getContent(), constructMapType(Map.class)); } catch (NoTaskLocationException e) { return ImmutableMap.of(); @@ -357,5 +349,7 @@ public ListenableFuture getStatusAsync(final Str return doAsync(() -> getStatus(id)); } + abstract protected JavaType constructMapType(Class mapType); + } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClientFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientFactory.java similarity index 97% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClientFactory.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientFactory.java index 313c4de80d6a..15cf078b6dcc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskClientFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.SeekableStream; +package org.apache.druid.indexing.seekablestream; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java similarity index 97% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskRunner.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 621af0d8819e..7e2d7ea99af1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.indexing.SeekableStream; +package org.apache.druid.indexing.seekablestream; import com.google.common.annotations.VisibleForTesting; import org.apache.druid.indexer.TaskStatus; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java similarity index 97% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamPartitions.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index 6821b6fdbcdd..313fc5c2853b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.SeekableStream; +package org.apache.druid.indexing.seekablestream; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java similarity index 99% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamTuningConfig.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java index 0012ad9d9884..768e04639ac7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/SeekableStreamTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.SeekableStream; +package org.apache.druid.indexing.seekablestream; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/Record.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java similarity index 60% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/Record.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java index 770614f3ef5c..7192a12a7d4b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/Record.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java @@ -1,17 +1,17 @@ -package org.apache.druid.indexing.SeekableStream.common; +package org.apache.druid.indexing.seekablestream.common; import java.util.List; -public class Record +public class Record { public static final String END_OF_SHARD_MARKER = "EOS"; private final String streamName; - private final PartitionType partitionId; - private final SequenceType sequenceNumber; + private final T1 partitionId; + private final T2 sequenceNumber; private final List data; - public Record(String streamName, PartitionType partitionId, SequenceType sequenceNumber, List data) + public Record(String streamName, T1 partitionId, T2 sequenceNumber, List data) { this.streamName = streamName; this.partitionId = partitionId; @@ -24,12 +24,12 @@ public String getStreamName() return streamName; } - public PartitionType getPartitionId() + public T1 getPartitionId() { return partitionId; } - public SequenceType getSequenceNumber() + public T2 getSequenceNumber() { return sequenceNumber; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java new file mode 100644 index 000000000000..ecdb251d11de --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java @@ -0,0 +1,35 @@ +package org.apache.druid.indexing.seekablestream.common; + +import java.io.Closeable; +import java.util.Set; +import java.util.concurrent.TimeoutException; + +// TODO: need to integrate this with Kafka + +/** + * The RecordSupplier interface is a wrapper for the incoming seekable data stream + * (i.e. Kafka consumer) + */ +public interface RecordSupplier extends Closeable +{ + void assign(Set> partitions); + + void seek(StreamPartition partition, T2 sequenceNumber); + + void seekAfter(StreamPartition partition, T2 sequenceNumber); + + void seekToEarliest(StreamPartition partition); + + void seekToLatest(StreamPartition partition); + + Record poll(long timeout); + + T2 getLatestSequenceNumber(StreamPartition partition) throws TimeoutException; + + T2 getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException; + + Set getPartitionIds(String streamName); + + @Override + void close(); +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/SequenceNumberPlus.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java similarity index 90% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/SequenceNumberPlus.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java index cc9789edda7d..49cd782ccaad 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/SequenceNumberPlus.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java @@ -1,4 +1,4 @@ -package org.apache.druid.indexing.SeekableStream.common; +package org.apache.druid.indexing.seekablestream.common; public class SequenceNumberPlus diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/StreamPartition.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java similarity index 76% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/StreamPartition.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java index 585fd539e7f8..1bece8f3c2d1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/common/StreamPartition.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java @@ -1,11 +1,11 @@ -package org.apache.druid.indexing.SeekableStream.common; +package org.apache.druid.indexing.seekablestream.common; -public class StreamPartition +public class StreamPartition { private final String streamName; - private final PartitionType partitionId; + private final T1 partitionId; - public StreamPartition(String streamName, PartitionType partitionId) + public StreamPartition(String streamName, T1 partitionId) { this.streamName = streamName; this.partitionId = partitionId; @@ -16,7 +16,7 @@ public String getStreamName() return streamName; } - public PartitionType getPartitionId() + public T1 getPartitionId() { return partitionId; } @@ -56,7 +56,7 @@ public String toString() '}'; } - public static StreamPartition of(String streamName, PartitionType partitionId) + public static StreamPartition of(String streamName, T1 partitionId) { return new StreamPartition<>(streamName, partitionId); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java similarity index 99% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisor.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 3e6b2bbb38de..386934b4a5be 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1,4 +1,4 @@ -package org.apache.druid.indexing.SeekableStream.supervisor; +package org.apache.druid.indexing.seekablestream.supervisor; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.MapperFeature; @@ -22,12 +22,12 @@ import org.apache.commons.codec.digest.DigestUtils; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.SeekableStream.SeekableStreamDataSourceMetadata; -import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTask; -import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTaskClient; -import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTaskClientFactory; -import org.apache.druid.indexing.SeekableStream.SeekableStreamTuningConfig; -import org.apache.druid.indexing.SeekableStream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; +import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.common.IndexTaskClient; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java similarity index 98% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorIOConfig.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java index d17135520bdd..a2c38c7a0cfe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.SeekableStream.supervisor; +package org.apache.druid.indexing.seekablestream.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java similarity index 98% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorReportPayload.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java index 7864676c5e1a..00db07fbae26 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorReportPayload.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.SeekableStream.supervisor; +package org.apache.druid.indexing.seekablestream.supervisor; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java similarity index 97% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorSpec.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java index bcd09044a54f..ea89a08c2869 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.SeekableStream.supervisor; +package org.apache.druid.indexing.seekablestream.supervisor; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; @@ -26,7 +26,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import org.apache.druid.guice.annotations.Json; -import org.apache.druid.indexing.SeekableStream.SeekableStreamIndexTaskClientFactory; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java similarity index 92% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorTuningConfig.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java index a370d44b0388..da4bf33a6350 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamSupervisorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java @@ -17,10 +17,10 @@ * under the License. */ -package org.apache.druid.indexing.SeekableStream.supervisor; +package org.apache.druid.indexing.seekablestream.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.SeekableStream.SeekableStreamTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.joda.time.Duration; import org.joda.time.Period; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamTaskReportData.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamTaskReportData.java similarity index 97% rename from indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamTaskReportData.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamTaskReportData.java index cc3a485cb1f5..7a84bd447ccd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/SeekableStream/supervisor/SeekableStreamTaskReportData.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamTaskReportData.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.SeekableStream.supervisor; +package org.apache.druid.indexing.seekablestream.supervisor; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; From f2f99a9ad9743bc675204386fb5789bb9f98d5de Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Fri, 14 Sep 2018 17:30:05 -0700 Subject: [PATCH 05/87] kafkarecordsupplier --- .../org/apache/druid/indexing/kafka/KafkaRecordSupplier.java | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java new file mode 100644 index 000000000000..97e49b9d330e --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -0,0 +1,5 @@ +package org.apache.druid.indexing.kafka; + +public class KafkaRecordSupplier +{ +} From 17791e641dcb0cee871afc50cb7d51b7617e6286 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Fri, 14 Sep 2018 17:30:48 -0700 Subject: [PATCH 06/87] integrated kafka indexing service with seekablestream --- .../kafka/KafkaDataSourceMetadata.java | 2 +- .../druid/indexing/kafka/KafkaIOConfig.java | 7 + .../druid/indexing/kafka/KafkaIndexTask.java | 3 +- .../indexing/kafka/KafkaIndexTaskClient.java | 9 +- .../kafka/KafkaIndexTaskClientFactory.java | 2 +- .../indexing/kafka/KafkaIndexTaskRunner.java | 2 +- .../indexing/kafka/KafkaRecordSupplier.java | 161 +- .../kafka/supervisor/KafkaSupervisor.java | 2601 ++--------------- .../KafkaSupervisorReportPayload.java | 1 + .../kafka/supervisor/KafkaSupervisorSpec.java | 16 +- .../KafkaSupervisorTuningConfig.java | 3 +- .../kafka/supervisor/TaskReportData.java | 76 - .../indexing/kafka/KafkaIndexTaskTest.java | 27 +- .../kafka/supervisor/KafkaSupervisorTest.java | 5 +- .../druid/indexing/kafka/test/TestBroker.java | 2 +- .../indexing/common/IndexTaskClient.java | 6 + .../SeekableStreamDataSourceMetadata.java | 28 +- .../SeekableStreamIOConfig.java | 8 +- .../SeekableStreamIndexTask.java | 31 +- .../SeekableStreamIndexTaskClient.java | 6 +- .../SeekableStreamIndexTaskClientFactory.java | 4 +- .../SeekableStreamIndexTaskRunner.java | 7 +- .../SeekableStreamPartitions.java | 4 +- .../SeekableStreamTuningConfig.java | 8 +- .../seekablestream/common/Record.java | 19 + .../seekablestream/common/RecordSupplier.java | 29 +- .../common/SequenceNumberPlus.java | 29 +- .../common/StreamPartition.java | 29 +- .../supervisor/SeekableStreamSupervisor.java | 1090 +++++-- .../SeekableStreamSupervisorIOConfig.java | 14 +- ...SeekableStreamSupervisorReportPayload.java | 18 +- .../SeekableStreamSupervisorSpec.java | 21 +- .../SeekableStreamSupervisorTuningConfig.java | 13 +- ...askReportData.java => TaskReportData.java} | 38 +- 34 files changed, 1522 insertions(+), 2797 deletions(-) delete mode 100644 extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/TaskReportData.java rename indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/{SeekableStreamTaskReportData.java => TaskReportData.java} (77%) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java index c7f0459b9c99..dfec33b0d10c 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java @@ -22,8 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; -import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; import org.apache.druid.java.util.common.IAE; import java.util.Map; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java index 5699a07658c2..30673e2ac7b8 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java @@ -27,6 +27,7 @@ import javax.annotation.Nullable; import java.util.Map; +import java.util.Set; public class KafkaIOConfig extends SeekableStreamIOConfig { @@ -96,6 +97,12 @@ public KafkaPartitions getEndPartitions() return (KafkaPartitions) super.getEndPartitions(); } + @Override + public Set getExclusiveStartSequenceNumberPartitions() + { + return null; + } + @Nullable @JsonProperty diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 618abc386872..9d0157f1fcac 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -26,7 +26,6 @@ import com.google.common.collect.Lists; import org.apache.druid.data.input.InputRow; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import org.apache.druid.indexing.common.TaskToolbox; @@ -34,6 +33,7 @@ import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; @@ -74,6 +74,7 @@ public class KafkaIndexTask extends SeekableStreamIndexTask // This value can be tuned in some tests private long pollRetryMs = 30000; + @JsonCreator public KafkaIndexTask( @JsonProperty("id") String id, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java index 390dfe93770d..04d254dbdc4b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -21,8 +21,9 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; import org.apache.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; @@ -60,5 +61,11 @@ protected JavaType constructMapType(Class mapType) return mapper.getTypeFactory().constructMapType(mapType, Integer.class, Long.class); } + @Override + public Class getTaskClassType() + { + return KafkaIndexTask.class; + } + } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java index b39f49c5555a..5b148f2c010d 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java @@ -23,8 +23,8 @@ import com.google.inject.Inject; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.guice.annotations.Json; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java index fae889156a8b..a85f18ecdd3a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java @@ -20,9 +20,9 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.stats.RowIngestionMeters; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.firehose.ChatHandler; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 97e49b9d330e..0b5e99593995 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -1,5 +1,164 @@ package org.apache.druid.indexing.kafka; -public class KafkaRecordSupplier +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; +import org.apache.druid.indexing.seekablestream.common.Record; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.stream.Collectors; + +public class KafkaRecordSupplier implements RecordSupplier { + private static final Random RANDOM = new Random(); + + private final KafkaConsumer consumer; + private final KafkaSupervisorIOConfig ioConfig; + private boolean closed; + + + public KafkaRecordSupplier( + KafkaSupervisorIOConfig ioConfig + ) + { + this.ioConfig = ioConfig; + this.consumer = getKafkaConsumer(); + this.closed = false; + } + + @Override + public void assign(Set> streamPartitions) + { + consumer.assign(streamPartitions + .stream() + .map(x -> new TopicPartition(x.getStreamName(), x.getPartitionId())) + .collect(Collectors.toSet())); + } + + @Override + public void seek(StreamPartition partition, Long sequenceNumber) + { + consumer.seek(new TopicPartition(partition.getStreamName(), partition.getPartitionId()), sequenceNumber); + } + + @Override + public void seekAfter(StreamPartition partition, Long sequenceNumber) + { + seek(partition, sequenceNumber + 1); + } + + @Override + public void seekToEarliest(Set> partitions) + { + consumer.seekToBeginning(partitions + .stream() + .map(e -> new TopicPartition(e.getStreamName(), e.getPartitionId())) + .collect(Collectors.toList())); + } + + @Override + public void seekToLatest(Set> partitions) + { + consumer.seekToEnd(partitions + .stream() + .map(e -> new TopicPartition(e.getStreamName(), e.getPartitionId())) + .collect(Collectors.toList())); + } + + @Override + public Set> getAssignment() + { + Set topicPartitions = consumer.assignment(); + return topicPartitions + .stream() + .map((TopicPartition e) -> new StreamPartition<>(e.topic(), e.partition())) + .collect(Collectors.toSet()); + } + + @Override + public Record poll(long timeout) + { + throw new UnsupportedOperationException(); + } + + @Override + public Long getLatestSequenceNumber(StreamPartition partition) + { + seekToLatest(Collections.singleton(partition)); + return consumer.position(new TopicPartition(partition.getStreamName(), partition.getPartitionId())); + } + + @Override + public Long getEarliestSequenceNumber(StreamPartition partition) + { + seekToEarliest(Collections.singleton(partition)); + return consumer.position(new TopicPartition(partition.getStreamName(), partition.getPartitionId())); + } + + @Override + public Long position(StreamPartition partition) + { + return consumer.position(new TopicPartition(partition.getStreamName(), partition.getPartitionId())); + } + + @Override + public Set getPartitionIds(String streamName) + { + final Map> topics = consumer.listTopics(); + if (topics == null || !topics.containsKey(streamName)) { + throw new ISE("Could not retrieve partitions for topic [%s]", streamName); + } + return topics.get(streamName).stream().map(PartitionInfo::partition).collect(Collectors.toSet()); + } + + @Override + public void close() + { + if (closed) { + return; + } + closed = true; + consumer.close(); + } + + private KafkaConsumer getKafkaConsumer() + { + final Properties props = new Properties(); + + props.setProperty("metadata.max.age.ms", "10000"); + props.setProperty("group.id", StringUtils.format("kafka-supervisor-%s", getRandomId())); + + props.putAll(ioConfig.getConsumerProperties()); + + props.setProperty("enable.auto.commit", "false"); + + ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + return new KafkaConsumer<>(props, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + } + finally { + Thread.currentThread().setContextClassLoader(currCtxCl); + } + } + + private static String getRandomId() + { + final StringBuilder suffix = new StringBuilder(8); + for (int i = 0; i < Integer.BYTES * 2; ++i) { + suffix.append((char) ('a' + ((RANDOM.nextInt() >>> (i * 4)) & 0x0F))); + } + return suffix.toString(); + } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 5b6193a40d3e..1f90e86322a7 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -21,98 +21,51 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; import com.google.common.base.Joiner; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.primitives.Longs; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import org.apache.commons.codec.digest.DigestUtils; -import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamTaskReportData; -import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.kafka.KafkaDataSourceMetadata; import org.apache.druid.indexing.kafka.KafkaIOConfig; import org.apache.druid.indexing.kafka.KafkaIndexTask; -import org.apache.druid.indexing.kafka.KafkaIndexTaskClient; import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; import org.apache.druid.indexing.kafka.KafkaPartitions; +import org.apache.druid.indexing.kafka.KafkaRecordSupplier; import org.apache.druid.indexing.kafka.KafkaTuningConfig; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; -import org.apache.druid.indexing.overlord.TaskQueue; -import org.apache.druid.indexing.overlord.TaskRunner; -import org.apache.druid.indexing.overlord.TaskRunnerListener; -import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorage; -import org.apache.druid.indexing.overlord.supervisor.Supervisor; -import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.IAE; +import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.joda.time.DateTime; import javax.annotation.Nullable; -import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; -import java.util.Properties; import java.util.Set; -import java.util.SortedMap; import java.util.TreeMap; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; -import java.util.stream.Stream; /** * Supervisor responsible for managing the KafkaIndexTasks for a single dataSource. At a high level, the class accepts a @@ -122,156 +75,18 @@ * tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of * Kafka offsets. */ -public class KafkaSupervisor implements Supervisor +public class KafkaSupervisor extends SeekableStreamSupervisor { private static final EmittingLogger log = new EmittingLogger(KafkaSupervisor.class); - private static final long MAX_RUN_FREQUENCY_MILLIS = 1000; // prevent us from running too often in response to events - private static final long NOT_SET = -1; - private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120; private static final long MINIMUM_GET_OFFSET_PERIOD_MILLIS = 5000; private static final long INITIAL_GET_OFFSET_DELAY_MILLIS = 15000; private static final long INITIAL_EMIT_LAG_METRIC_DELAY_MILLIS = 25000; - private static final int MAX_INITIALIZATION_RETRIES = 20; - private static final CopyOnWriteArrayList EMPTY_LIST = Lists.newCopyOnWriteArrayList(); - - public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED"; - - // Internal data structures - // -------------------------------------------------------- - - /** - * A TaskGroup is the main data structure used by KafkaSupervisor to organize and monitor Kafka partitions and - * indexing tasks. All the tasks in a TaskGroup should always be doing the same thing (reading the same partitions and - * starting from the same offset) and if [replicas] is configured to be 1, a TaskGroup will contain a single task (the - * exception being if the supervisor started up and discovered and adopted some already running tasks). At any given - * 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]). - */ - private class TaskGroup - { - final int groupId; - - // 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 - // this task group has completed successfully, at which point this will be destroyed and a new task group will be - // created with new starting offsets. This allows us to create replacement tasks for failed tasks that process the - // same offsets, even if the values in [partitionGroups] has been changed. - final ImmutableMap partitionOffsets; - - final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); - final Optional minimumMessageTime; - final Optional maximumMessageTime; - DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action - final TreeMap> sequenceOffsets = new TreeMap<>(); - final String baseSequenceName; - - TaskGroup( - int groupId, - ImmutableMap partitionOffsets, - Optional minimumMessageTime, - Optional maximumMessageTime - ) - { - this.groupId = groupId; - this.partitionOffsets = partitionOffsets; - this.minimumMessageTime = minimumMessageTime; - this.maximumMessageTime = maximumMessageTime; - this.sequenceOffsets.put(0, partitionOffsets); - this.baseSequenceName = generateSequenceName(partitionOffsets, minimumMessageTime, maximumMessageTime); - } - - int addNewCheckpoint(Map checkpoint) - { - sequenceOffsets.put(sequenceOffsets.lastKey() + 1, checkpoint); - return sequenceOffsets.lastKey(); - } - - Set taskIds() - { - return tasks.keySet(); - } - } - - private static class TaskData - { - @Nullable - volatile TaskStatus status; - @Nullable - volatile DateTime startTime; - volatile Map currentOffsets = new HashMap<>(); - - @Override - public String toString() - { - return "TaskData{" + - "status=" + status + - ", startTime=" + startTime + - ", currentOffsets=" + currentOffsets + - '}'; - } - } - // Map<{group ID}, {actively reading task group}>; see documentation for TaskGroup class - private final ConcurrentHashMap taskGroups = new ConcurrentHashMap<>(); - - // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [taskGroups] to here so - // we can monitor its status while we queue new tasks to read the next range of offsets. This is a list since we could - // have multiple sets of tasks publishing at once if time-to-publish > taskDuration. - // Map<{group ID}, List<{pending completion task groups}>> - private final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); - - // 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 - // offset value from the metadata store, and if it can't find it there, from Kafka. Once a task begins - // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet- - // completed task, which will cause the next set of tasks to begin reading from where the previous task left - // off. If that previous task now fails, we will set the offset in [partitionGroups] back to NOT_SET which will - // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to - // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task - // failures during publishing. - // Map<{group ID}, Map<{partition ID}, {startingOffset}>> - private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); - // -------------------------------------------------------- - - private final TaskStorage taskStorage; - private final TaskMaster taskMaster; - private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; - private final KafkaIndexTaskClient taskClient; - private final ObjectMapper sortingMapper; - private final KafkaSupervisorSpec spec; private final ServiceEmitter emitter; private final DruidMonitorSchedulerConfig monitorSchedulerConfig; - private final String dataSource; - private final KafkaSupervisorIOConfig ioConfig; - private final KafkaSupervisorTuningConfig tuningConfig; - private final KafkaTuningConfig taskTuningConfig; - private final String supervisorId; - private final TaskInfoProvider taskInfoProvider; - private final long futureTimeoutInSeconds; // how long to wait for async operations to complete - private final RowIngestionMetersFactory rowIngestionMetersFactory; - private final ExecutorService exec; - private final ScheduledExecutorService scheduledExec; - private final ScheduledExecutorService reportingExec; - private final ListeningExecutorService workerExec; - private final BlockingQueue notices = new LinkedBlockingDeque<>(); - private final Object stopLock = new Object(); - private final Object stateChangeLock = new Object(); - private final Object consumerLock = new Object(); - private boolean listenerRegistered = false; - private long lastRunTime; - - private int initRetryCounter = 0; - - private volatile DateTime firstRunTime; - private volatile KafkaConsumer consumer; - - private volatile boolean lifecycleStarted = false; - private volatile boolean started = false; - private volatile boolean stopped = false; - private volatile Map latestOffsetsFromKafka; - private volatile DateTime offsetsLastUpdated; + private final KafkaSupervisorSpec spec; public KafkaSupervisor( final TaskStorage taskStorage, @@ -283,222 +98,26 @@ public KafkaSupervisor( final RowIngestionMetersFactory rowIngestionMetersFactory ) { - this.taskStorage = taskStorage; - this.taskMaster = taskMaster; - this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; - this.sortingMapper = mapper.copy().configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); + super( + StringUtils.format("KafkaSupervisor-%s", spec.getDataSchema().getDataSource()), + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + mapper, + spec, + rowIngestionMetersFactory, + -1L, + Long.MAX_VALUE, + true + ); + this.spec = spec; this.emitter = spec.getEmitter(); this.monitorSchedulerConfig = spec.getMonitorSchedulerConfig(); - this.rowIngestionMetersFactory = rowIngestionMetersFactory; - - this.dataSource = spec.getDataSchema().getDataSource(); - this.ioConfig = (KafkaSupervisorIOConfig) spec.getIoConfig(); - this.tuningConfig = (KafkaSupervisorTuningConfig) spec.getTuningConfig(); - this.taskTuningConfig = (KafkaTuningConfig) this.tuningConfig.copyOf(); - this.supervisorId = StringUtils.format("KafkaSupervisor-%s", dataSource); - this.exec = Execs.singleThreaded(supervisorId); - this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d"); - this.reportingExec = Execs.scheduledSingleThreaded(supervisorId + "-Reporting-%d"); - - int workerThreads = (this.tuningConfig.getWorkerThreads() != null - ? this.tuningConfig.getWorkerThreads() - : Math.min(10, this.ioConfig.getTaskCount())); - this.workerExec = MoreExecutors.listeningDecorator(Execs.multiThreaded(workerThreads, supervisorId + "-Worker-%d")); - log.info("Created worker pool with [%d] threads for dataSource [%s]", workerThreads, this.dataSource); - - this.taskInfoProvider = new TaskInfoProvider() - { - @Override - public TaskLocation getTaskLocation(final String id) - { - Preconditions.checkNotNull(id, "id"); - Optional taskRunner = taskMaster.getTaskRunner(); - if (taskRunner.isPresent()) { - Optional item = Iterables.tryFind( - taskRunner.get().getRunningTasks(), - (Predicate) taskRunnerWorkItem -> id.equals(taskRunnerWorkItem.getTaskId()) - ); - - if (item.isPresent()) { - return item.get().getLocation(); - } - } else { - log.error("Failed to get task runner because I'm not the leader!"); - } - - return TaskLocation.unknown(); - } - - @Override - public Optional getTaskStatus(String id) - { - return taskStorage.getStatus(id); - } - }; - - this.futureTimeoutInSeconds = Math.max( - MINIMUM_FUTURE_TIMEOUT_IN_SECONDS, - tuningConfig.getChatRetries() * (tuningConfig.getHttpTimeout().getStandardSeconds() - + KafkaIndexTaskClient.MAX_RETRY_WAIT_SECONDS) - ); - - int chatThreads = (this.tuningConfig.getChatThreads() != null - ? this.tuningConfig.getChatThreads() - : Math.min(10, this.ioConfig.getTaskCount() * this.ioConfig.getReplicas())); - this.taskClient = taskClientFactory.build( - taskInfoProvider, - dataSource, - chatThreads, - this.tuningConfig.getHttpTimeout(), - this.tuningConfig.getChatRetries() - ); - log.info( - "Created taskClient with dataSource[%s] chatThreads[%d] httpTimeout[%s] chatRetries[%d]", - dataSource, - chatThreads, - this.tuningConfig.getHttpTimeout(), - this.tuningConfig.getChatRetries() - ); - } - - @Override - public void start() - { - synchronized (stateChangeLock) { - Preconditions.checkState(!lifecycleStarted, "already started"); - Preconditions.checkState(!exec.isShutdown(), "already stopped"); - - // Try normal initialization first, if that fails then schedule periodic initialization retries - try { - tryInit(); - } - catch (Exception e) { - if (!started) { - log.warn("First initialization attempt failed for KafkaSupervisor[%s], starting retries...", dataSource); - - exec.submit( - () -> { - try { - RetryUtils.retry( - () -> { - tryInit(); - return 0; - }, - (throwable) -> { - return !started; - }, - 0, - MAX_INITIALIZATION_RETRIES, - null, - null - ); - } - catch (Exception e2) { - log.makeAlert( - "Failed to initialize after %s retries, aborting. Please resubmit the supervisor spec to restart this supervisor [%s]", - MAX_INITIALIZATION_RETRIES, - supervisorId - ).emit(); - throw new RuntimeException(e2); - } - } - ); - } - } - lifecycleStarted = true; - } - } - - @Override - public void stop(boolean stopGracefully) - { - synchronized (stateChangeLock) { - Preconditions.checkState(lifecycleStarted, "lifecycle not started"); - - log.info("Beginning shutdown of KafkaSupervisor[%s]", dataSource); - - try { - scheduledExec.shutdownNow(); // stop recurring executions - reportingExec.shutdownNow(); - - if (started) { - Optional taskRunner = taskMaster.getTaskRunner(); - if (taskRunner.isPresent()) { - taskRunner.get().unregisterListener(supervisorId); - } - - // Stopping gracefully will synchronize the end offsets of the tasks and signal them to publish, and will block - // until the tasks have acknowledged or timed out. We want this behavior when we're explicitly shut down through - // the API, but if we shut down for other reasons (e.g. we lose leadership) we want to just stop and leave the - // tasks as they are. - synchronized (stopLock) { - if (stopGracefully) { - log.info("Posting GracefulShutdownNotice, signalling managed tasks to complete and publish"); - notices.add(new GracefulShutdownNotice()); - } else { - log.info("Posting ShutdownNotice"); - notices.add(new ShutdownNotice()); - } - - long shutdownTimeoutMillis = tuningConfig.getShutdownTimeout().getMillis(); - long endTime = System.currentTimeMillis() + shutdownTimeoutMillis; - while (!stopped) { - long sleepTime = endTime - System.currentTimeMillis(); - if (sleepTime <= 0) { - log.info("Timed out while waiting for shutdown (timeout [%,dms])", shutdownTimeoutMillis); - stopped = true; - break; - } - stopLock.wait(sleepTime); - } - } - log.info("Shutdown notice handled"); - } - - taskClient.close(); - workerExec.shutdownNow(); - exec.shutdownNow(); - started = false; - - log.info("KafkaSupervisor[%s] has stopped", dataSource); - } - catch (Exception e) { - log.makeAlert(e, "Exception stopping KafkaSupervisor[%s]", dataSource) - .emit(); - } - } - } - - @Override - public SupervisorReport getStatus() - { - return generateReport(true); - } - - @Override - public Map> getStats() - { - try { - return getCurrentTotalStats(); - } - catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - log.error(ie, "getStats() interrupted."); - throw new RuntimeException(ie); - } - catch (ExecutionException | TimeoutException eete) { - throw new RuntimeException(eete); - } } - @Override - public void reset(DataSourceMetadata dataSourceMetadata) - { - log.info("Posting ResetNotice"); - notices.add(new ResetNotice(dataSourceMetadata)); - } @Override public void checkpoint( @@ -511,14 +130,16 @@ public void checkpoint( Preconditions.checkNotNull(previousCheckPoint, "previousCheckpoint"); Preconditions.checkNotNull(currentCheckPoint, "current checkpoint cannot be null"); Preconditions.checkArgument( - ioConfig.getTopic().equals(((KafkaDataSourceMetadata) currentCheckPoint).getKafkaPartitions().getTopic()), + spec.getIoConfig() + .getTopic() + .equals(((KafkaDataSourceMetadata) currentCheckPoint).getKafkaPartitions().getTopic()), "Supervisor topic [%s] and topic in checkpoint [%s] does not match", - ioConfig.getTopic(), + spec.getIoConfig().getTopic(), ((KafkaDataSourceMetadata) currentCheckPoint).getKafkaPartitions().getTopic() ); log.info("Checkpointing [%s] for taskGroup [%s]", currentCheckPoint, taskGroupId); - notices.add( + addNotice( new CheckpointNotice( taskGroupId, baseSequenceName, @@ -528,2053 +149,245 @@ public void checkpoint( ); } - public void possiblyRegisterListener() + + @Override + protected RecordSupplier setupRecordSupplier() { - // getTaskRunner() sometimes fails if the task queue is still being initialized so retry later until we succeed + return new KafkaRecordSupplier(spec.getIoConfig()); - if (listenerRegistered) { - return; - } + } - Optional taskRunner = taskMaster.getTaskRunner(); - if (taskRunner.isPresent()) { - taskRunner.get().registerListener( - new TaskRunnerListener() - { - @Override - public String getListenerId() - { - return supervisorId; - } + @Override + protected void scheduleReporting(ScheduledExecutorService reportingExec) + { + KafkaSupervisorIOConfig ioConfig = spec.getIoConfig(); + KafkaSupervisorTuningConfig tuningConfig = spec.getTuningConfig(); + reportingExec.scheduleAtFixedRate( + updateCurrentAndLatestOffsets(), + ioConfig.getStartDelay().getMillis() + INITIAL_GET_OFFSET_DELAY_MILLIS, // wait for tasks to start up + Math.max( + tuningConfig.getOffsetFetchPeriod().getMillis(), MINIMUM_GET_OFFSET_PERIOD_MILLIS + ), + TimeUnit.MILLISECONDS + ); - @Override - public void locationChanged(final String taskId, final TaskLocation newLocation) - { - // do nothing - } + reportingExec.scheduleAtFixedRate( + emitLag(), + ioConfig.getStartDelay().getMillis() + INITIAL_EMIT_LAG_METRIC_DELAY_MILLIS, // wait for tasks to start up + monitorSchedulerConfig.getEmitterPeriod().getMillis(), + TimeUnit.MILLISECONDS + ); + } - @Override - public void statusChanged(String taskId, TaskStatus status) - { - notices.add(new RunNotice()); - } - }, MoreExecutors.sameThreadExecutor() - ); - listenerRegistered = true; - } + @Override + protected int getTaskGroupIdForPartition(Integer partition) + { + return partition % spec.getIoConfig().getTaskCount(); } - private interface Notice + @Override + protected boolean checkSourceMetaDataMatch(DataSourceMetadata metadata) { - void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException; + return metadata instanceof KafkaDataSourceMetadata; } - private class RunNotice implements Notice + @Override + protected boolean checkTaskInstance(Task task) { - @Override - public void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException - { - long nowTime = System.currentTimeMillis(); - if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { - return; - } - lastRunTime = nowTime; - - runInternal(); - } + return task instanceof KafkaIndexTask; } - private class GracefulShutdownNotice extends ShutdownNotice + @Override + protected SeekableStreamSupervisorReportPayload createReportPayload( + int numPartitions, + boolean includeOffsets + ) { - @Override - public void handle() throws InterruptedException, ExecutionException, TimeoutException - { - gracefulShutdownInternal(); - super.handle(); - } + KafkaSupervisorIOConfig ioConfig = spec.getIoConfig(); + Map partitionLag = getLagPerPartition(getHighestCurrentOffsets()); + return new KafkaSupervisorReportPayload( + spec.getDataSchema().getDataSource(), + ioConfig.getTopic(), + numPartitions, + ioConfig.getReplicas(), + ioConfig.getTaskDuration().getMillis() / 1000, + includeOffsets ? latestOffsetsFromStream : null, + includeOffsets ? partitionLag : null, + includeOffsets ? partitionLag.values().stream().mapToLong(x -> Math.max(x, 0)).sum() : null, + includeOffsets ? offsetsLastUpdated : null + ); } - private class ShutdownNotice implements Notice - { - @Override - public void handle() throws InterruptedException, ExecutionException, TimeoutException - { - consumer.close(); - synchronized (stopLock) { - stopped = true; - stopLock.notifyAll(); - } - } + @Override + protected SeekableStreamIOConfig createIoConfig( + int groupId, + Map startPartitions, + Map endPartitions, + String baseSequenceName, + DateTime minimumMessageTime, + DateTime maximumMessageTime, + Set exclusiveStartSequenceNumberPartitions, + SeekableStreamSupervisorIOConfig ioConfig + ) + { + KafkaSupervisorIOConfig kafkaIoConfig = (KafkaSupervisorIOConfig) ioConfig; + return new KafkaIOConfig( + groupId, + baseSequenceName, + new KafkaPartitions(kafkaIoConfig.getTopic(), startPartitions), + new KafkaPartitions(kafkaIoConfig.getTopic(), endPartitions), + kafkaIoConfig.getConsumerProperties(), + true, + minimumMessageTime, + maximumMessageTime, + kafkaIoConfig.isSkipOffsetGaps() + ); } - private class ResetNotice implements Notice + @Override + protected List> createIndexTask( + int replicas, + String baseSequenceName, + ObjectMapper sortingMapper, + TreeMap> sequenceOffsets, + SeekableStreamIOConfig taskIoConfig, + SeekableStreamTuningConfig taskTuningConfig, + RowIngestionMetersFactory rowIngestionMetersFactory + ) throws JsonProcessingException { - final DataSourceMetadata dataSourceMetadata; - - ResetNotice(DataSourceMetadata dataSourceMetadata) + final String checkpoints = sortingMapper.writerWithType(new TypeReference>>() { - this.dataSourceMetadata = dataSourceMetadata; - } + }).writeValueAsString(sequenceOffsets); + final Map context = spec.getContext() == null + ? ImmutableMap.of( + "checkpoints", + checkpoints, + IS_INCREMENTAL_HANDOFF_SUPPORTED, + true + ) : ImmutableMap.builder() + .put("checkpoints", checkpoints) + .put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true) + .putAll(spec.getContext()) + .build(); - @Override - public void handle() - { - resetInternal(dataSourceMetadata); + List> taskList = new ArrayList<>(); + for (int i = 0; i < replicas; i++) { + String taskId = Joiner.on("_").join(baseSequenceName, getRandomId()); + taskList.add(new KafkaIndexTask( + taskId, + new TaskResource(baseSequenceName, 1), + spec.getDataSchema(), + (KafkaTuningConfig) taskTuningConfig, + (KafkaIOConfig) taskIoConfig, + context, + null, + null, + rowIngestionMetersFactory + )); } + return taskList; } - private class CheckpointNotice implements Notice - { - @Nullable - private final Integer nullableTaskGroupId; - @Deprecated - private final String baseSequenceName; - private final KafkaDataSourceMetadata previousCheckpoint; - private final KafkaDataSourceMetadata currentCheckpoint; - - CheckpointNotice( - @Nullable Integer nullableTaskGroupId, - @Deprecated String baseSequenceName, - KafkaDataSourceMetadata previousCheckpoint, - KafkaDataSourceMetadata currentCheckpoint - ) - { - this.baseSequenceName = baseSequenceName; - this.nullableTaskGroupId = nullableTaskGroupId; - this.previousCheckpoint = previousCheckpoint; - this.currentCheckpoint = currentCheckpoint; - } - @Override - public void handle() throws ExecutionException, InterruptedException - { - // Find taskGroupId using taskId if it's null. It can be null while rolling update. - final int taskGroupId; - if (nullableTaskGroupId == null) { - // We search taskId in taskGroups and pendingCompletionTaskGroups sequentially. This should be fine because - // 1) a taskGroup can be moved from taskGroups to pendingCompletionTaskGroups in RunNotice - // (see checkTaskDuration()). - // 2) Notices are proceesed by a single thread. So, CheckpointNotice and RunNotice cannot be processed at the - // same time. - final java.util.Optional maybeGroupId = taskGroups - .entrySet() - .stream() - .filter(entry -> { - final TaskGroup taskGroup = entry.getValue(); - return taskGroup.baseSequenceName.equals(baseSequenceName); - }) - .findAny() - .map(Entry::getKey); - taskGroupId = maybeGroupId.orElse( - pendingCompletionTaskGroups - .entrySet() - .stream() - .filter(entry -> { - final List taskGroups = entry.getValue(); - return taskGroups.stream().anyMatch(group -> group.baseSequenceName.equals(baseSequenceName)); - }) - .findAny() - .orElseThrow(() -> new ISE("Cannot find taskGroup for baseSequenceName[%s]", baseSequenceName)) - .getKey() + @Override + protected Map getLagPerPartition(Map currentOffsets) + { + return currentOffsets + .entrySet() + .stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + e -> latestOffsetsFromStream != null + && latestOffsetsFromStream.get(e.getKey()) != null + && e.getValue() != null + ? latestOffsetsFromStream.get(e.getKey()) - e.getValue() + : Integer.MIN_VALUE + ) ); - } else { - taskGroupId = nullableTaskGroupId; - } - - // check for consistency - // if already received request for this sequenceName and dataSourceMetadata combination then return - final TaskGroup taskGroup = taskGroups.get(taskGroupId); - - if (isValidTaskGroup(taskGroupId, taskGroup)) { - final TreeMap> checkpoints = taskGroup.sequenceOffsets; - - // check validity of previousCheckpoint - int index = checkpoints.size(); - for (int sequenceId : checkpoints.descendingKeySet()) { - Map checkpoint = checkpoints.get(sequenceId); - // We have already verified the topic of the current checkpoint is same with that in ioConfig. - // See checkpoint(). - if (checkpoint.equals(previousCheckpoint.getKafkaPartitions().getPartitionOffsetMap())) { - break; - } - index--; - } - if (index == 0) { - throw new ISE("No such previous checkpoint [%s] found", previousCheckpoint); - } else if (index < checkpoints.size()) { - // if the found checkpoint is not the latest one then already checkpointed by a replica - Preconditions.checkState(index == checkpoints.size() - 1, "checkpoint consistency failure"); - log.info("Already checkpointed with offsets [%s]", checkpoints.lastEntry().getValue()); - return; - } - final Map newCheckpoint = checkpointTaskGroup(taskGroup, false).get(); - taskGroup.addNewCheckpoint(newCheckpoint); - log.info("Handled checkpoint notice, new checkpoint is [%s] for taskGroup [%s]", newCheckpoint, taskGroupId); - } - } - - private boolean isValidTaskGroup(int taskGroupId, @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); - } - } - - return true; - } } - @VisibleForTesting - void resetInternal(DataSourceMetadata dataSourceMetadata) + private Runnable emitLag() { - if (dataSourceMetadata == null) { - // Reset everything - boolean result = indexerMetadataStorageCoordinator.deleteDataSourceMetadata(dataSource); - log.info("Reset dataSource[%s] - dataSource metadata entry deleted? [%s]", dataSource, result); - taskGroups.values().forEach(this::killTasksInGroup); - taskGroups.clear(); - partitionGroups.clear(); - } else if (!(dataSourceMetadata instanceof KafkaDataSourceMetadata)) { - throw new IAE("Expected KafkaDataSourceMetadata but found instance of [%s]", dataSourceMetadata.getClass()); - } else { - // Reset only the partitions in dataSourceMetadata if it has not been reset yet - final KafkaDataSourceMetadata resetKafkaMetadata = (KafkaDataSourceMetadata) dataSourceMetadata; + return () -> { + try { + Map highestCurrentOffsets = getHighestCurrentOffsets(); - if (resetKafkaMetadata.getKafkaPartitions().getTopic().equals(ioConfig.getTopic())) { - // metadata can be null - final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); - if (metadata != null && !(metadata instanceof KafkaDataSourceMetadata)) { - throw new IAE( - "Expected KafkaDataSourceMetadata from metadata store but found instance of [%s]", - metadata.getClass() - ); + if (latestOffsetsFromStream == null) { + throw new ISE("Latest offsets from Kafka have not been fetched"); } - final KafkaDataSourceMetadata currentMetadata = (KafkaDataSourceMetadata) metadata; - // defend against consecutive reset requests from replicas - // as well as the case where the metadata store do not have an entry for the reset partitions - boolean doReset = false; - for (Entry resetPartitionOffset : resetKafkaMetadata.getKafkaPartitions() - .getPartitionOffsetMap() - .entrySet()) { - final Long partitionOffsetInMetadataStore = currentMetadata == null - ? null - : currentMetadata.getKafkaPartitions() - .getPartitionOffsetMap() - .get(resetPartitionOffset.getKey()); - final TaskGroup partitionTaskGroup = taskGroups.get( - getTaskGroupIdForPartition(resetPartitionOffset.getKey()) + if (!latestOffsetsFromStream.keySet().equals(highestCurrentOffsets.keySet())) { + log.warn( + "Lag metric: Kafka partitions %s do not match task partitions %s", + latestOffsetsFromStream.keySet(), + highestCurrentOffsets.keySet() ); - final boolean isSameOffset = partitionTaskGroup != null - && partitionTaskGroup.partitionOffsets.get(resetPartitionOffset.getKey()) - .equals(resetPartitionOffset.getValue()); - if (partitionOffsetInMetadataStore != null || isSameOffset) { - doReset = true; - break; - } } - if (!doReset) { - log.info("Ignoring duplicate reset request [%s]", dataSourceMetadata); - return; - } + long lag = getLagPerPartition(highestCurrentOffsets) + .values() + .stream() + .mapToLong(x -> Math.max(x, 0)) + .sum(); - boolean metadataUpdateSuccess = false; - if (currentMetadata == null) { - metadataUpdateSuccess = true; - } else { - final DataSourceMetadata newMetadata = currentMetadata.minus(resetKafkaMetadata); - try { - metadataUpdateSuccess = indexerMetadataStorageCoordinator.resetDataSourceMetadata(dataSource, newMetadata); - } - catch (IOException e) { - log.error("Resetting DataSourceMetadata failed [%s]", e.getMessage()); - Throwables.propagate(e); - } - } - if (metadataUpdateSuccess) { - resetKafkaMetadata.getKafkaPartitions().getPartitionOffsetMap().keySet().forEach(partition -> { - final int groupId = getTaskGroupIdForPartition(partition); - killTaskGroupForPartitions(ImmutableSet.of(partition)); - taskGroups.remove(groupId); - partitionGroups.get(groupId).replaceAll((partitionId, offset) -> NOT_SET); - }); - } else { - throw new ISE("Unable to reset metadata"); - } - } else { - log.warn( - "Reset metadata topic [%s] and supervisor's topic [%s] do not match", - resetKafkaMetadata.getKafkaPartitions().getTopic(), - ioConfig.getTopic() + emitter.emit( + ServiceMetricEvent.builder() + .setDimension("dataSource", spec.getDataSchema().getDataSource()) + .build("ingest/kafka/lag", lag) ); } - } + catch (Exception e) { + log.warn(e, "Unable to compute Kafka lag"); + } + }; } - private void killTaskGroupForPartitions(Set partitions) + // the following are for unit testing purposes only + @Override + @VisibleForTesting + protected void runInternal() + throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException { - for (Integer partition : partitions) { - killTasksInGroup(taskGroups.get(getTaskGroupIdForPartition(partition))); - } + super.runInternal(); } - private void killTasksInGroup(TaskGroup taskGroup) + @Override + @VisibleForTesting + protected Runnable updateCurrentAndLatestOffsets() { - if (taskGroup != null) { - for (String taskId : taskGroup.tasks.keySet()) { - log.info("Killing task [%s] in the task group", taskId); - killTask(taskId); - } - } + return super.updateCurrentAndLatestOffsets(); } + @Override @VisibleForTesting - void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException + protected void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException { - // Prepare for shutdown by 1) killing all tasks that haven't been assigned to a worker yet, and 2) causing all - // running tasks to begin publishing by setting their startTime to a very long time ago so that the logic in - // checkTaskDuration() will be triggered. This is better than just telling these tasks to publish whatever they - // have, as replicas that are supposed to publish the same segment may not have read the same set of offsets. - for (TaskGroup taskGroup : taskGroups.values()) { - for (Entry entry : taskGroup.tasks.entrySet()) { - if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) { - killTask(entry.getKey()); - } else { - entry.getValue().startTime = DateTimes.EPOCH; - } - } - } - - checkTaskDuration(); + super.gracefulShutdownInternal(); } + @Override @VisibleForTesting - void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException + protected void resetInternal(DataSourceMetadata dataSourceMetadata) { - possiblyRegisterListener(); - updatePartitionDataFromKafka(); - discoverTasks(); - updateTaskStatus(); - checkTaskDuration(); - checkPendingCompletionTasks(); - checkCurrentTaskState(); - - // if supervisor is not suspended, ensure required tasks are running - // if suspended, ensure tasks have been requested to gracefully stop - if (!spec.isSuspended()) { - log.info("[%s] supervisor is running.", dataSource); - createNewTasks(); - } else { - log.info("[%s] supervisor is suspended.", dataSource); - gracefulShutdownInternal(); - } - - if (log.isDebugEnabled()) { - log.debug(generateReport(true).toString()); - } else { - log.info(generateReport(false).toString()); - } + super.resetInternal(dataSourceMetadata); } - String generateSequenceName( - Map startPartitions, - Optional minimumMessageTime, - Optional maximumMessageTime - ) + @Override + @VisibleForTesting + protected void moveTaskGroupToPendingCompletion(int taskGroupId) { - StringBuilder sb = new StringBuilder(); - - for (Entry entry : startPartitions.entrySet()) { - sb.append(StringUtils.format("+%d(%d)", entry.getKey(), entry.getValue())); - } - String partitionOffsetStr = sb.toString().substring(1); - - String minMsgTimeStr = (minimumMessageTime.isPresent() ? String.valueOf(minimumMessageTime.get().getMillis()) : ""); - String maxMsgTimeStr = (maximumMessageTime.isPresent() ? String.valueOf(maximumMessageTime.get().getMillis()) : ""); - - String dataSchema, tuningConfig; - try { - dataSchema = sortingMapper.writeValueAsString(spec.getDataSchema()); - tuningConfig = sortingMapper.writeValueAsString(taskTuningConfig); - } - catch (JsonProcessingException e) { - throw Throwables.propagate(e); - } - - String hashCode = DigestUtils.sha1Hex(dataSchema - + tuningConfig - + partitionOffsetStr - + minMsgTimeStr - + maxMsgTimeStr) - .substring(0, 15); - - return Joiner.on("_").join("index_kafka", dataSource, hashCode); + super.moveTaskGroupToPendingCompletion(taskGroupId); } + @Override @VisibleForTesting - protected void tryInit() + protected int getNoticesQueueSize() { - synchronized (stateChangeLock) { - if (started) { - log.warn("SUpervisor was already started, skipping init"); - return; - } - - if (stopped) { - log.warn("Supervisor was already stopped, skipping init."); - return; - } - - try { - consumer = getKafkaConsumer(); - - exec.submit( - () -> { - try { - long pollTimeout = Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS); - while (!Thread.currentThread().isInterrupted() && !stopped) { - final Notice notice = notices.poll(pollTimeout, TimeUnit.MILLISECONDS); - if (notice == null) { - continue; - } + return super.getNoticesQueueSize(); + } - try { - notice.handle(); - } - catch (Throwable e) { - log.makeAlert(e, "KafkaSupervisor[%s] failed to handle notice", dataSource) - .addData("noticeClass", notice.getClass().getSimpleName()) - .emit(); - } - } - } - catch (InterruptedException e) { - log.info("KafkaSupervisor[%s] interrupted, exiting", dataSource); - } - } - ); - firstRunTime = DateTimes.nowUtc().plus(ioConfig.getStartDelay()); - scheduledExec.scheduleAtFixedRate( - buildRunTask(), - ioConfig.getStartDelay().getMillis(), - Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS), - TimeUnit.MILLISECONDS - ); - reportingExec.scheduleAtFixedRate( - updateCurrentAndLatestOffsets(), - ioConfig.getStartDelay().getMillis() + INITIAL_GET_OFFSET_DELAY_MILLIS, // wait for tasks to start up - Math.max( - tuningConfig.getOffsetFetchPeriod().getMillis(), MINIMUM_GET_OFFSET_PERIOD_MILLIS - ), - TimeUnit.MILLISECONDS - ); - - reportingExec.scheduleAtFixedRate( - emitLag(), - ioConfig.getStartDelay().getMillis() + INITIAL_EMIT_LAG_METRIC_DELAY_MILLIS, // wait for tasks to start up - monitorSchedulerConfig.getEmitterPeriod().getMillis(), - TimeUnit.MILLISECONDS - ); - - started = true; - log.info( - "Started KafkaSupervisor[%s], first run in [%s], with spec: [%s]", - dataSource, - ioConfig.getStartDelay(), - spec.toString() - ); - } - catch (Exception e) { - if (consumer != null) { - consumer.close(); - } - initRetryCounter++; - log.makeAlert(e, "Exception starting KafkaSupervisor[%s]", dataSource) - .emit(); - - throw new RuntimeException(e); - } - } - } - - private KafkaConsumer getKafkaConsumer() - { - final Properties props = new Properties(); - - props.setProperty("metadata.max.age.ms", "10000"); - props.setProperty("group.id", StringUtils.format("kafka-supervisor-%s", RealtimeIndexTask.makeRandomId())); - - KafkaIndexTask.addConsumerPropertiesFromConfig(props, sortingMapper, ioConfig.getConsumerProperties()); - - props.setProperty("enable.auto.commit", "false"); - - ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); - try { - Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); - return new KafkaConsumer<>(props, new ByteArrayDeserializer(), new ByteArrayDeserializer()); - } - finally { - Thread.currentThread().setContextClassLoader(currCtxCl); - } - } - - private void updatePartitionDataFromKafka() - { - List partitions; - try { - synchronized (consumerLock) { - partitions = consumer.partitionsFor(ioConfig.getTopic()); - } - } - catch (Exception e) { - log.warn( - e, - "Unable to get partition data from Kafka for brokers [%s], are the brokers up?", - ioConfig.getConsumerProperties().get(KafkaSupervisorIOConfig.BOOTSTRAP_SERVERS_KEY) - ); - return; - } - - int numPartitions = (partitions != null ? partitions.size() : 0); - - log.debug("Found [%d] Kafka partitions for topic [%s]", numPartitions, ioConfig.getTopic()); - - for (int partition = 0; partition < numPartitions; partition++) { - int taskGroupId = getTaskGroupIdForPartition(partition); - - ConcurrentHashMap partitionMap = partitionGroups.computeIfAbsent( - taskGroupId, - k -> new ConcurrentHashMap<>() - ); - - // 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 - // offset value from the metadata store, and if it can't find it there, from Kafka. Once a task begins - // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet- - // completed task, which will cause the next set of tasks to begin reading from where the previous task left - // off. If that previous task now fails, we will set the offset in [partitionGroups] back to NOT_SET which will - // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to - // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task - // failures during publishing. - if (partitionMap.putIfAbsent(partition, NOT_SET) == null) { - log.info( - "New partition [%d] discovered for topic [%s], added to task group [%d]", - partition, - ioConfig.getTopic(), - taskGroupId - ); - } - } - } - - private void discoverTasks() throws ExecutionException, InterruptedException, TimeoutException - { - int taskCount = 0; - List futureTaskIds = Lists.newArrayList(); - List> futures = Lists.newArrayList(); - List tasks = taskStorage.getActiveTasks(); - final Map taskGroupsToVerify = new HashMap<>(); - - for (Task task : tasks) { - if (!(task instanceof KafkaIndexTask) || !dataSource.equals(task.getDataSource())) { - continue; - } - - taskCount++; - final KafkaIndexTask kafkaTask = (KafkaIndexTask) task; - final String taskId = task.getId(); - - // Determine which task group this task belongs to based on one of the partitions handled by this task. If we - // later determine that this task is actively reading, we will make sure that it matches our current partition - // allocation (getTaskGroupIdForPartition(partition) should return the same value for every partition being read - // by this task) and kill it if it is not compatible. If the task is instead found to be in the publishing - // state, we will permit it to complete even if it doesn't match our current partition allocation to support - // seamless schema migration. - - Iterator it = kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap().keySet().iterator(); - final Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null); - - if (taskGroupId != null) { - // check to see if we already know about this task, either in [taskGroups] or in [pendingCompletionTaskGroups] - // and if not add it to taskGroups or pendingCompletionTaskGroups (if status = PUBLISHING) - TaskGroup taskGroup = taskGroups.get(taskGroupId); - if (!isTaskInPendingCompletionGroups(taskId) && (taskGroup == null || !taskGroup.tasks.containsKey(taskId))) { - - futureTaskIds.add(taskId); - futures.add( - Futures.transform( - taskClient.getStatusAsync(taskId), new Function() - { - @Override - public Boolean apply(SeekableStreamIndexTask.Status status) - { - try { - log.debug("Task [%s], status [%s]", taskId, status); - if (status == SeekableStreamIndexTask.Status.PUBLISHING) { - kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap().keySet().forEach( - partition -> addDiscoveredTaskToPendingCompletionTaskGroups( - getTaskGroupIdForPartition(partition), - taskId, - kafkaTask.getIOConfig() - .getStartPartitions() - .getPartitionOffsetMap() - ) - ); - - // update partitionGroups with the publishing task's offsets (if they are greater than what is - // existing) so that the next tasks will start reading from where this task left off - Map publishingTaskEndOffsets = taskClient.getEndOffsets(taskId); - - for (Entry entry : publishingTaskEndOffsets.entrySet()) { - Integer partition = entry.getKey(); - Long offset = entry.getValue(); - ConcurrentHashMap partitionOffsets = partitionGroups.get( - getTaskGroupIdForPartition(partition) - ); - - boolean succeeded; - do { - succeeded = true; - Long previousOffset = partitionOffsets.putIfAbsent(partition, offset); - if (previousOffset != null && previousOffset < offset) { - succeeded = partitionOffsets.replace(partition, previousOffset, offset); - } - } while (!succeeded); - } - } else { - for (Integer partition : kafkaTask.getIOConfig() - .getStartPartitions() - .getPartitionOffsetMap() - .keySet()) { - if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { - log.warn( - "Stopping task [%s] which does not match the expected partition allocation", - taskId - ); - try { - stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - } - catch (InterruptedException | ExecutionException | TimeoutException e) { - log.warn(e, "Exception while stopping task"); - } - return false; - } - } - // make sure the task's io and tuning configs match with the supervisor config - // if it is current then only create corresponding taskGroup if it does not exist - if (!isTaskCurrent(taskGroupId, taskId)) { - log.info( - "Stopping task [%s] which does not match the expected parameters and ingestion spec", - taskId - ); - try { - stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - } - catch (InterruptedException | ExecutionException | TimeoutException e) { - log.warn(e, "Exception while stopping task"); - } - return false; - } else { - final TaskGroup taskGroup = taskGroups.computeIfAbsent( - taskGroupId, - k -> { - log.info("Creating a new task group for taskGroupId[%d]", taskGroupId); - return new TaskGroup( - taskGroupId, - ImmutableMap.copyOf( - kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap() - ), - kafkaTask.getIOConfig().getMinimumMessageTime(), - kafkaTask.getIOConfig().getMaximumMessageTime() - ); - } - ); - taskGroupsToVerify.put(taskGroupId, taskGroup); - final TaskData prevTaskData = taskGroup.tasks.putIfAbsent(taskId, new TaskData()); - if (prevTaskData != null) { - throw new ISE( - "WTH? a taskData[%s] already exists for new task[%s]", - prevTaskData, - taskId - ); - } - } - } - return true; - } - catch (Throwable t) { - log.error(t, "Something bad while discovering task [%s]", taskId); - return null; - } - } - }, workerExec - ) - ); - } - } - } - - List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - for (int i = 0; i < results.size(); i++) { - if (results.get(i) == null) { - String taskId = futureTaskIds.get(i); - log.warn("Task [%s] failed to return status, killing task", taskId); - killTask(taskId); - } - } - log.debug("Found [%d] Kafka indexing tasks for dataSource [%s]", taskCount, dataSource); - - // make sure the checkpoints are consistent with each other and with the metadata store - verifyAndMergeCheckpoints(taskGroupsToVerify.values()); - } - - private void verifyAndMergeCheckpoints(final Collection taskGroupsToVerify) - { - final List> futures = new ArrayList<>(); - for (TaskGroup taskGroup : taskGroupsToVerify) { - futures.add(workerExec.submit(() -> verifyAndMergeCheckpoints(taskGroup))); - } - try { - Futures.allAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - } - catch (InterruptedException | ExecutionException | TimeoutException e) { - throw new RuntimeException(e); - } - } - - /** - * This method does two things - - * 1. Makes sure the checkpoints information in the taskGroup is consistent with that of the tasks, if not kill - * inconsistent tasks. - * 2. truncates the checkpoints in the taskGroup corresponding to which segments have been published, so that any newly - * created tasks for the taskGroup start indexing from after the latest published offsets. - */ - private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) - { - final int groupId = taskGroup.groupId; - final List>>> taskSequences = new ArrayList<>(); - final List>>> futures = new ArrayList<>(); - final List taskIds = new ArrayList<>(); - - for (String taskId : taskGroup.taskIds()) { - final ListenableFuture>> checkpointsFuture = taskClient.getCheckpointsAsync( - taskId, - true - ); - taskIds.add(taskId); - futures.add(checkpointsFuture); - } - - try { - List>> futuresResult = Futures.successfulAsList(futures) - .get(futureTimeoutInSeconds, TimeUnit.SECONDS); - - for (int i = 0; i < futuresResult.size(); i++) { - final TreeMap> checkpoints = futuresResult.get(i); - final String taskId = taskIds.get(i); - if (checkpoints == null) { - try { - // catch the exception in failed futures - futures.get(i).get(); - } - catch (Exception e) { - log.error(e, "Problem while getting checkpoints for task [%s], killing the task", taskId); - killTask(taskId); - taskGroup.tasks.remove(taskId); - } - } else if (checkpoints.isEmpty()) { - log.warn("Ignoring task [%s], as probably it is not started running yet", taskId); - } else { - taskSequences.add(new Pair<>(taskId, checkpoints)); - } - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - - final KafkaDataSourceMetadata latestDataSourceMetadata = (KafkaDataSourceMetadata) indexerMetadataStorageCoordinator - .getDataSourceMetadata(dataSource); - final boolean hasValidOffsetsFromDb = latestDataSourceMetadata != null && - latestDataSourceMetadata.getKafkaPartitions() != null && - ioConfig.getTopic().equals( - latestDataSourceMetadata.getKafkaPartitions().getTopic() - ); - final Map latestOffsetsFromDb; - if (hasValidOffsetsFromDb) { - latestOffsetsFromDb = latestDataSourceMetadata.getKafkaPartitions().getPartitionOffsetMap(); - } else { - latestOffsetsFromDb = null; - } - - // order tasks of this taskGroup by the latest sequenceId - taskSequences.sort((o1, o2) -> o2.rhs.firstKey().compareTo(o1.rhs.firstKey())); - - final Set tasksToKill = new HashSet<>(); - final AtomicInteger earliestConsistentSequenceId = new AtomicInteger(-1); - int taskIndex = 0; - - while (taskIndex < taskSequences.size()) { - TreeMap> taskCheckpoints = taskSequences.get(taskIndex).rhs; - String taskId = taskSequences.get(taskIndex).lhs; - if (earliestConsistentSequenceId.get() == -1) { - // find the first replica task with earliest sequenceId consistent with datasource metadata in the metadata - // store - if (taskCheckpoints.entrySet().stream().anyMatch( - sequenceCheckpoint -> sequenceCheckpoint.getValue().entrySet().stream().allMatch( - partitionOffset -> Longs.compare( - partitionOffset.getValue(), - latestOffsetsFromDb == null ? - partitionOffset.getValue() : - latestOffsetsFromDb.getOrDefault(partitionOffset.getKey(), partitionOffset.getValue()) - ) == 0) && earliestConsistentSequenceId.compareAndSet(-1, sequenceCheckpoint.getKey())) || ( - pendingCompletionTaskGroups.getOrDefault(groupId, EMPTY_LIST).size() > 0 - && earliestConsistentSequenceId.compareAndSet(-1, taskCheckpoints.firstKey()))) { - final SortedMap> latestCheckpoints = new TreeMap<>( - taskCheckpoints.tailMap(earliestConsistentSequenceId.get()) - ); - log.info("Setting taskGroup sequences to [%s] for group [%d]", latestCheckpoints, groupId); - taskGroup.sequenceOffsets.clear(); - taskGroup.sequenceOffsets.putAll(latestCheckpoints); - } else { - log.debug( - "Adding task [%s] to kill list, checkpoints[%s], latestoffsets from DB [%s]", - taskId, - taskCheckpoints, - latestOffsetsFromDb - ); - tasksToKill.add(taskId); - } - } else { - // check consistency with taskGroup sequences - if (taskCheckpoints.get(taskGroup.sequenceOffsets.firstKey()) == null - || !(taskCheckpoints.get(taskGroup.sequenceOffsets.firstKey()) - .equals(taskGroup.sequenceOffsets.firstEntry().getValue())) - || taskCheckpoints.tailMap(taskGroup.sequenceOffsets.firstKey()).size() - != taskGroup.sequenceOffsets.size()) { - log.debug( - "Adding task [%s] to kill list, checkpoints[%s], taskgroup checkpoints [%s]", - taskId, - taskCheckpoints, - taskGroup.sequenceOffsets - ); - tasksToKill.add(taskId); - } - } - taskIndex++; - } - - if ((tasksToKill.size() > 0 && tasksToKill.size() == taskGroup.tasks.size()) || - (taskGroup.tasks.size() == 0 && pendingCompletionTaskGroups.getOrDefault(groupId, EMPTY_LIST).size() == 0)) { - // 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 - log.warn("Clearing task group [%d] information as no valid tasks left the group", groupId); - taskGroups.remove(groupId); - partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET); - } - - taskSequences.stream().filter(taskIdSequences -> tasksToKill.contains(taskIdSequences.lhs)).forEach( - sequenceCheckpoint -> { - log.warn( - "Killing task [%s], as its checkpoints [%s] are not consistent with group checkpoints[%s] or latest " - + "persisted offsets in metadata store [%s]", - sequenceCheckpoint.lhs, - sequenceCheckpoint.rhs, - taskGroup.sequenceOffsets, - latestOffsetsFromDb - ); - killTask(sequenceCheckpoint.lhs); - taskGroup.tasks.remove(sequenceCheckpoint.lhs); - } - ); - } - - private void addDiscoveredTaskToPendingCompletionTaskGroups( - int groupId, - String taskId, - Map startingPartitions - ) - { - final CopyOnWriteArrayList taskGroupList = pendingCompletionTaskGroups.computeIfAbsent( - groupId, - k -> new CopyOnWriteArrayList<>() - ); - for (TaskGroup taskGroup : taskGroupList) { - if (taskGroup.partitionOffsets.equals(startingPartitions)) { - if (taskGroup.tasks.putIfAbsent(taskId, new TaskData()) == null) { - log.info("Added discovered task [%s] to existing pending task group [%s]", taskId, groupId); - } - return; - } - } - - log.info("Creating new pending completion task group [%s] for discovered task [%s]", groupId, taskId); - - // reading the minimumMessageTime & maximumMessageTime from the publishing task and setting it here is not necessary as this task cannot - // change to a state where it will read any more events - TaskGroup newTaskGroup = new TaskGroup( - groupId, - ImmutableMap.copyOf(startingPartitions), - Optional.absent(), - Optional.absent() - ); - - newTaskGroup.tasks.put(taskId, new TaskData()); - newTaskGroup.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout()); - - taskGroupList.add(newTaskGroup); - } - - private void updateTaskStatus() throws ExecutionException, InterruptedException, TimeoutException - { - final List> futures = Lists.newArrayList(); - final List futureTaskIds = Lists.newArrayList(); - - // update status (and startTime if unknown) of current tasks in taskGroups - for (TaskGroup group : taskGroups.values()) { - for (Entry entry : group.tasks.entrySet()) { - final String taskId = entry.getKey(); - final TaskData taskData = entry.getValue(); - - if (taskData.startTime == null) { - futureTaskIds.add(taskId); - futures.add( - Futures.transform( - taskClient.getStartTimeAsync(taskId), new Function() - { - @Nullable - @Override - public Boolean apply(@Nullable DateTime startTime) - { - if (startTime == null) { - return false; - } - - taskData.startTime = startTime; - long millisRemaining = ioConfig.getTaskDuration().getMillis() - - (System.currentTimeMillis() - taskData.startTime.getMillis()); - if (millisRemaining > 0) { - scheduledExec.schedule( - buildRunTask(), - millisRemaining + MAX_RUN_FREQUENCY_MILLIS, - TimeUnit.MILLISECONDS - ); - } - - return true; - } - }, workerExec - ) - ); - } - - taskData.status = taskStorage.getStatus(taskId).get(); - } - } - - // update status of pending completion tasks in pendingCompletionTaskGroups - for (List taskGroups : pendingCompletionTaskGroups.values()) { - for (TaskGroup group : taskGroups) { - for (Entry entry : group.tasks.entrySet()) { - entry.getValue().status = taskStorage.getStatus(entry.getKey()).get(); - } - } - } - - List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - for (int i = 0; i < results.size(); i++) { - // false means the task hasn't started running yet and that's okay; null means it should be running but the HTTP - // request threw an exception so kill the task - if (results.get(i) == null) { - String taskId = futureTaskIds.get(i); - log.warn("Task [%s] failed to return start time, killing task", taskId); - killTask(taskId); - } - } - } - - private void checkTaskDuration() throws InterruptedException, ExecutionException, TimeoutException - { - final List>> futures = Lists.newArrayList(); - final List futureGroupIds = Lists.newArrayList(); - - for (Entry entry : taskGroups.entrySet()) { - Integer groupId = entry.getKey(); - TaskGroup group = entry.getValue(); - - // find the longest running task from this group - DateTime earliestTaskStart = DateTimes.nowUtc(); - for (TaskData taskData : group.tasks.values()) { - // startTime can be null if kafkaSupervisor is stopped gracefully before processing any runNotice - if (taskData.startTime != null && earliestTaskStart.isAfter(taskData.startTime)) { - earliestTaskStart = taskData.startTime; - } - } - - // if this task has run longer than the configured duration, signal all tasks in the group to persist - if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { - log.info("Task group [%d] has run for [%s]", groupId, ioConfig.getTaskDuration()); - futureGroupIds.add(groupId); - futures.add(checkpointTaskGroup(group, true)); - } - } - - List> results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - for (int j = 0; j < results.size(); j++) { - Integer groupId = futureGroupIds.get(j); - TaskGroup group = taskGroups.get(groupId); - Map endOffsets = results.get(j); - - if (endOffsets != null) { - // set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion - group.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout()); - pendingCompletionTaskGroups.computeIfAbsent(groupId, k -> new CopyOnWriteArrayList<>()).add(group); - - // set endOffsets as the next startOffsets - for (Entry entry : endOffsets.entrySet()) { - partitionGroups.get(groupId).put(entry.getKey(), entry.getValue()); - } - } else { - log.warn( - "All tasks in group [%s] failed to transition to publishing state, killing tasks [%s]", - groupId, - group.taskIds() - ); - for (String id : group.taskIds()) { - killTask(id); - } - // clear partitionGroups, so that latest offsets from db is used as start offsets not the stale ones - // if tasks did some successful incremental handoffs - partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET); - } - - // remove this task group from the list of current task groups now that it has been handled - taskGroups.remove(groupId); - } - } - - private ListenableFuture> checkpointTaskGroup(final TaskGroup taskGroup, final boolean finalize) - { - if (finalize) { - // 1) Check if any task completed (in which case we're done) and kill unassigned tasks - Iterator> i = taskGroup.tasks.entrySet().iterator(); - while (i.hasNext()) { - Entry taskEntry = i.next(); - String taskId = taskEntry.getKey(); - TaskData task = taskEntry.getValue(); - - // task.status can be null if kafkaSupervisor is stopped gracefully before processing any runNotice. - if (task.status != null) { - if (task.status.isSuccess()) { - // If any task in this group has already completed, stop the rest of the tasks in the group and return. - // This will cause us to create a new set of tasks next cycle that will start from the offsets in - // 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) - return Futures.transform( - stopTasksInGroup(taskGroup), - new Function>() - { - @Nullable - @Override - public Map apply(@Nullable Object input) - { - return null; - } - } - ); - } - - if (task.status.isRunnable()) { - if (taskInfoProvider.getTaskLocation(taskId).equals(TaskLocation.unknown())) { - log.info("Killing task [%s] which hasn't been assigned to a worker", taskId); - killTask(taskId); - i.remove(); - } - } - } - } - } - - // 2) Pause running tasks - final List>> pauseFutures = Lists.newArrayList(); - final List pauseTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); - for (final String taskId : pauseTaskIds) { - pauseFutures.add(taskClient.pauseAsync(taskId)); - } - - return Futures.transform( - Futures.successfulAsList(pauseFutures), new Function>, Map>() - { - @Nullable - @Override - public Map apply(List> input) - { - // 3) Build a map of the highest offset read by any task in the group for each partition - final Map endOffsets = new HashMap<>(); - for (int i = 0; i < input.size(); i++) { - Map result = input.get(i); - - if (result == null || result.isEmpty()) { // kill tasks that didn't return a value - String taskId = pauseTaskIds.get(i); - log.warn("Task [%s] failed to respond to [pause] in a timely manner, killing task", taskId); - killTask(taskId); - taskGroup.tasks.remove(taskId); - - } else { // otherwise build a map of the highest offsets seen - for (Entry offset : result.entrySet()) { - if (!endOffsets.containsKey(offset.getKey()) - || endOffsets.get(offset.getKey()).compareTo(offset.getValue()) < 0) { - endOffsets.put(offset.getKey(), offset.getValue()); - } - } - } - } - - // 4) Set the end offsets for each task to the values from step 3 and resume the tasks. All the tasks should - // finish reading and start publishing within a short period, depending on how in sync the tasks were. - final List> setEndOffsetFutures = Lists.newArrayList(); - final List setEndOffsetTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); - - if (setEndOffsetTaskIds.isEmpty()) { - log.info("All tasks in taskGroup [%d] have failed, tasks will be re-created", taskGroup.groupId); - return null; - } - - try { - - if (endOffsets.equals(taskGroup.sequenceOffsets.lastEntry().getValue())) { - log.warn( - "Checkpoint [%s] is same as the start offsets [%s] of latest sequence for the task group [%d]", - endOffsets, - taskGroup.sequenceOffsets.lastEntry().getValue(), - taskGroup.groupId - ); - } - - log.info( - "Setting endOffsets for tasks in taskGroup [%d] to %s and resuming", - taskGroup.groupId, - endOffsets - ); - for (final String taskId : setEndOffsetTaskIds) { - setEndOffsetFutures.add(taskClient.setEndOffsetsAsync(taskId, endOffsets, finalize)); - } - - List results = Futures.successfulAsList(setEndOffsetFutures) - .get(futureTimeoutInSeconds, TimeUnit.SECONDS); - for (int i = 0; i < results.size(); i++) { - if (results.get(i) == null || !results.get(i)) { - String taskId = setEndOffsetTaskIds.get(i); - log.warn("Task [%s] failed to respond to [set end offsets] in a timely manner, killing task", taskId); - killTask(taskId); - taskGroup.tasks.remove(taskId); - } - } - } - catch (Exception e) { - log.error("Something bad happened [%s]", e.getMessage()); - Throwables.propagate(e); - } - - if (taskGroup.tasks.isEmpty()) { - log.info("All tasks in taskGroup [%d] have failed, tasks will be re-created", taskGroup.groupId); - return null; - } - - return endOffsets; - } - }, workerExec - ); - } - - /** - * Monitors [pendingCompletionTaskGroups] for tasks that have completed. If any task in a task group has completed, we - * can safely stop the rest of the tasks in that group. If a task group has exceeded its publishing timeout, then - * we need to stop all tasks in not only that task group but also 1) any subsequent task group that is also pending - * completion and 2) the current task group that is running, because the assumption that we have handled up to the - * starting offset for subsequent task groups is no longer valid, and subsequent tasks would fail as soon as they - * attempted to publish because of the contiguous range consistency check. - */ - private void checkPendingCompletionTasks() throws ExecutionException, InterruptedException, TimeoutException - { - List> futures = Lists.newArrayList(); - - for (Entry> pendingGroupList : pendingCompletionTaskGroups.entrySet()) { - - boolean stopTasksInTaskGroup = false; - Integer groupId = pendingGroupList.getKey(); - CopyOnWriteArrayList taskGroupList = pendingGroupList.getValue(); - List toRemove = Lists.newArrayList(); - - for (TaskGroup group : taskGroupList) { - boolean foundSuccess = false, entireTaskGroupFailed = false; - - if (stopTasksInTaskGroup) { - // One of the earlier groups that was handling the same partition set timed out before the segments were - // published so stop any additional groups handling the same partition set that are pending completion. - futures.add(stopTasksInGroup(group)); - toRemove.add(group); - continue; - } - - Iterator> iTask = group.tasks.entrySet().iterator(); - while (iTask.hasNext()) { - final Entry entry = iTask.next(); - final String taskId = entry.getKey(); - final TaskData taskData = entry.getValue(); - - Preconditions.checkNotNull(taskData.status, "WTH? task[%s] has a null status", taskId); - - if (taskData.status.isFailure()) { - iTask.remove(); // remove failed task - if (group.tasks.isEmpty()) { - // if all tasks in the group have failed, just nuke all task groups with this partition set and restart - entireTaskGroupFailed = true; - break; - } - } - - if (taskData.status.isSuccess()) { - // If one of the pending completion tasks was successful, stop the rest of the tasks in the group as - // we no longer need them to publish their segment. - log.info("Task [%s] completed successfully, stopping tasks %s", taskId, group.taskIds()); - futures.add(stopTasksInGroup(group)); - foundSuccess = true; - toRemove.add(group); // remove the TaskGroup from the list of pending completion task groups - break; // skip iterating the rest of the tasks in this group as they've all been stopped now - } - } - - if ((!foundSuccess && group.completionTimeout.isBeforeNow()) || entireTaskGroupFailed) { - if (entireTaskGroupFailed) { - log.warn("All tasks in group [%d] failed to publish, killing all tasks for these partitions", groupId); - } else { - log.makeAlert( - "No task in [%s] for taskGroup [%d] succeeded before the completion timeout elapsed [%s]!", - group.taskIds(), - groupId, - ioConfig.getCompletionTimeout() - ).emit(); - } - - // 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); - // kill all the tasks in this pending completion group - killTasksInGroup(group); - // set a flag so the other pending completion groups for this set of partitions will also stop - stopTasksInTaskGroup = true; - - // kill all the tasks in the currently reading task group and remove the bad task group - killTasksInGroup(taskGroups.remove(groupId)); - toRemove.add(group); - } - } - - taskGroupList.removeAll(toRemove); - } - - // wait for all task shutdowns to complete before returning - Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - } - - private void checkCurrentTaskState() throws ExecutionException, InterruptedException, TimeoutException - { - List> futures = Lists.newArrayList(); - Iterator> iTaskGroups = taskGroups.entrySet().iterator(); - while (iTaskGroups.hasNext()) { - Entry taskGroupEntry = iTaskGroups.next(); - Integer groupId = taskGroupEntry.getKey(); - TaskGroup taskGroup = taskGroupEntry.getValue(); - - // Iterate the list of known tasks in this group and: - // 1) Kill any tasks which are not "current" (have the partitions, starting offsets, and minimumMessageTime - // & maximumMessageTime (if applicable) in [taskGroups]) - // 2) Remove any tasks that have failed from the list - // 3) If any task completed successfully, stop all the tasks in this group and move to the next group - - log.debug("Task group [%d] pre-pruning: %s", groupId, taskGroup.taskIds()); - - Iterator> iTasks = taskGroup.tasks.entrySet().iterator(); - while (iTasks.hasNext()) { - Entry task = iTasks.next(); - String taskId = task.getKey(); - TaskData taskData = task.getValue(); - - // stop and remove bad tasks from the task group - if (!isTaskCurrent(groupId, taskId)) { - log.info("Stopping task [%s] which does not match the expected offset range and ingestion spec", taskId); - futures.add(stopTask(taskId, false)); - iTasks.remove(); - continue; - } - - Preconditions.checkNotNull(taskData.status, "WTH? task[%s] has a null status", taskId); - - // remove failed tasks - if (taskData.status.isFailure()) { - iTasks.remove(); - continue; - } - - // check for successful tasks, and if we find one, stop all tasks in the group and remove the group so it can - // be recreated with the next set of offsets - if (taskData.status.isSuccess()) { - futures.add(stopTasksInGroup(taskGroup)); - iTaskGroups.remove(); - break; - } - } - log.debug("Task group [%d] post-pruning: %s", groupId, taskGroup.taskIds()); - } - - // wait for all task shutdowns to complete before returning - Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - } - - void createNewTasks() throws JsonProcessingException - { - // update the checkpoints in the taskGroup to latest ones so that new tasks do not read what is already published - verifyAndMergeCheckpoints( - taskGroups.values() - .stream() - .filter(taskGroup -> taskGroup.tasks.size() < ioConfig.getReplicas()) - .collect(Collectors.toList()) - ); - - // check that there is a current task group for each group of partitions in [partitionGroups] - for (Integer groupId : partitionGroups.keySet()) { - if (!taskGroups.containsKey(groupId)) { - log.info("Creating new task group [%d] for partitions %s", groupId, partitionGroups.get(groupId).keySet()); - - Optional minimumMessageTime = (ioConfig.getLateMessageRejectionPeriod().isPresent() ? Optional.of( - DateTimes.nowUtc().minus(ioConfig.getLateMessageRejectionPeriod().get()) - ) : Optional.absent()); - - Optional maximumMessageTime = (ioConfig.getEarlyMessageRejectionPeriod().isPresent() ? Optional.of( - DateTimes.nowUtc().plus(ioConfig.getTaskDuration()).plus(ioConfig.getEarlyMessageRejectionPeriod().get()) - ) : Optional.absent()); - - final TaskGroup taskGroup = new TaskGroup( - groupId, - generateStartingOffsetsForPartitionGroup(groupId), - minimumMessageTime, - maximumMessageTime - ); - taskGroups.put( - groupId, - taskGroup - ); - } - } - - // iterate through all the current task groups and make sure each one has the desired number of replica tasks - boolean createdTask = false; - for (Entry entry : taskGroups.entrySet()) { - TaskGroup taskGroup = entry.getValue(); - Integer groupId = entry.getKey(); - - if (ioConfig.getReplicas() > taskGroup.tasks.size()) { - log.info( - "Number of tasks [%d] does not match configured numReplicas [%d] in task group [%d], creating more tasks", - taskGroup.tasks.size(), ioConfig.getReplicas(), groupId - ); - createKafkaTasksForGroup(groupId, ioConfig.getReplicas() - taskGroup.tasks.size()); - createdTask = true; - } - } - - if (createdTask && firstRunTime.isBeforeNow()) { - // Schedule a run event after a short delay to update our internal data structures with the new tasks that were - // just created. This is mainly for the benefit of the status API in situations where the run period is lengthy. - scheduledExec.schedule(buildRunTask(), 5000, TimeUnit.MILLISECONDS); - } - } - - private void createKafkaTasksForGroup(int groupId, int replicas) throws JsonProcessingException - { - Map startPartitions = taskGroups.get(groupId).partitionOffsets; - Map endPartitions = new HashMap<>(); - - for (Integer partition : startPartitions.keySet()) { - endPartitions.put(partition, Long.MAX_VALUE); - } - TaskGroup group = taskGroups.get(groupId); - - Map consumerProperties = Maps.newHashMap(ioConfig.getConsumerProperties()); - DateTime minimumMessageTime = taskGroups.get(groupId).minimumMessageTime.orNull(); - DateTime maximumMessageTime = taskGroups.get(groupId).maximumMessageTime.orNull(); - - KafkaIOConfig kafkaIOConfig = new KafkaIOConfig( - groupId, - group.baseSequenceName, - new KafkaPartitions(ioConfig.getTopic(), startPartitions), - new KafkaPartitions(ioConfig.getTopic(), endPartitions), - consumerProperties, - true, - minimumMessageTime, - maximumMessageTime, - ioConfig.isSkipOffsetGaps() - ); - - final String checkpoints = sortingMapper.writerWithType(new TypeReference>>() - { - }).writeValueAsString(taskGroups.get(groupId).sequenceOffsets); - final Map context = spec.getContext() == null - ? ImmutableMap.of( - "checkpoints", - checkpoints, - IS_INCREMENTAL_HANDOFF_SUPPORTED, - true - ) - : ImmutableMap.builder() - .put("checkpoints", checkpoints) - .put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true) - .putAll(spec.getContext()) - .build(); - for (int i = 0; i < replicas; i++) { - String taskId = Joiner.on("_").join(group.baseSequenceName, RealtimeIndexTask.makeRandomId()); - KafkaIndexTask indexTask = new KafkaIndexTask( - taskId, - new TaskResource(group.baseSequenceName, 1), - spec.getDataSchema(), - taskTuningConfig, - kafkaIOConfig, - context, - null, - null, - rowIngestionMetersFactory, - sortingMapper - ); - - Optional taskQueue = taskMaster.getTaskQueue(); - if (taskQueue.isPresent()) { - try { - taskQueue.get().add(indexTask); - } - catch (EntryExistsException e) { - log.error("Tried to add task [%s] but it already exists", indexTask.getId()); - } - } else { - log.error("Failed to get task queue because I'm not the leader!"); - } - } - } - - private ImmutableMap generateStartingOffsetsForPartitionGroup(int groupId) - { - ImmutableMap.Builder builder = ImmutableMap.builder(); - for (Entry entry : partitionGroups.get(groupId).entrySet()) { - Integer partition = entry.getKey(); - Long offset = entry.getValue(); - - if (offset != null && offset != NOT_SET) { - // if we are given a startingOffset (set by a previous task group which is pending completion) then use it - builder.put(partition, offset); - } else { - // if we don't have a startingOffset (first run or we had some previous failures and reset the offsets) then - // get the offset from metadata storage (if available) or Kafka (otherwise) - builder.put(partition, getOffsetFromStorageForPartition(partition)); - } - } - return builder.build(); - } - - /** - * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't - * find any data, it will retrieve the latest or earliest Kafka offset depending on the useEarliestOffset config. - */ - private long getOffsetFromStorageForPartition(int partition) - { - long offset; - final Map metadataOffsets = getOffsetsFromMetadataStorage(); - if (metadataOffsets.get(partition) != null) { - offset = metadataOffsets.get(partition); - log.debug("Getting offset [%,d] from metadata storage for partition [%d]", offset, partition); - - long latestKafkaOffset = getOffsetFromKafkaForPartition(partition, false); - if (offset > latestKafkaOffset) { - throw new ISE( - "Offset in metadata storage [%,d] > latest Kafka offset [%,d] for partition[%d] dataSource[%s]. If these " - + "messages are no longer available (perhaps you deleted and re-created your Kafka topic) you can use the " - + "supervisor reset API to restart ingestion.", - offset, - latestKafkaOffset, - partition, - dataSource - ); - } - - } else { - offset = getOffsetFromKafkaForPartition(partition, ioConfig.isUseEarliestOffset()); - log.debug("Getting offset [%,d] from Kafka for partition [%d]", offset, partition); - } - - return offset; - } - - private Map getOffsetsFromMetadataStorage() - { - final DataSourceMetadata dataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); - if (dataSourceMetadata instanceof KafkaDataSourceMetadata) { - KafkaPartitions partitions = ((KafkaDataSourceMetadata) dataSourceMetadata).getKafkaPartitions(); - if (partitions != null) { - if (!ioConfig.getTopic().equals(partitions.getTopic())) { - log.warn( - "Topic in metadata storage [%s] doesn't match spec topic [%s], ignoring stored offsets", - partitions.getTopic(), - ioConfig.getTopic() - ); - return Collections.emptyMap(); - } else if (partitions.getPartitionOffsetMap() != null) { - return partitions.getPartitionOffsetMap(); - } - } - } - - return Collections.emptyMap(); - } - - private long getOffsetFromKafkaForPartition(int partition, boolean useEarliestOffset) - { - synchronized (consumerLock) { - TopicPartition topicPartition = new TopicPartition(ioConfig.getTopic(), partition); - if (!consumer.assignment().contains(topicPartition)) { - consumer.assign(Collections.singletonList(topicPartition)); - } - - if (useEarliestOffset) { - consumer.seekToBeginning(Collections.singletonList(topicPartition)); - } else { - consumer.seekToEnd(Collections.singletonList(topicPartition)); - } - - return consumer.position(topicPartition); - } - } - - /** - * Compares the sequence name from the task with one generated for the task's group ID and returns false if they do - * not match. The sequence name is generated from a hash of the dataSchema, tuningConfig, starting offsets, and the - * minimumMessageTime or maximumMessageTime if set. - */ - private boolean isTaskCurrent(int taskGroupId, String taskId) - { - Optional taskOptional = taskStorage.getTask(taskId); - if (!taskOptional.isPresent() || !(taskOptional.get() instanceof KafkaIndexTask)) { - return false; - } - - String taskSequenceName = ((KafkaIndexTask) taskOptional.get()).getIOConfig().getBaseSequenceName(); - if (taskGroups.get(taskGroupId) != null) { - return Preconditions - .checkNotNull(taskGroups.get(taskGroupId), "null taskGroup for taskId[%s]", taskGroupId) - .baseSequenceName - .equals(taskSequenceName); - } else { - return generateSequenceName( - ((KafkaIndexTask) taskOptional.get()).getIOConfig() - .getStartPartitions() - .getPartitionOffsetMap(), - ((KafkaIndexTask) taskOptional.get()).getIOConfig().getMinimumMessageTime(), - ((KafkaIndexTask) taskOptional.get()).getIOConfig().getMaximumMessageTime() - ).equals(taskSequenceName); - } - } - - private ListenableFuture stopTasksInGroup(@Nullable TaskGroup taskGroup) - { - if (taskGroup == null) { - return Futures.immediateFuture(null); - } - - final List> futures = Lists.newArrayList(); - for (Entry entry : taskGroup.tasks.entrySet()) { - final String taskId = entry.getKey(); - final TaskData taskData = entry.getValue(); - if (taskData.status == null) { - killTask(taskId); - } else if (!taskData.status.isComplete()) { - futures.add(stopTask(taskId, false)); - } - } - - return Futures.successfulAsList(futures); - } - - private ListenableFuture stopTask(final String id, final boolean publish) - { - return Futures.transform( - taskClient.stopAsync(id, publish), new Function() - { - @Nullable - @Override - public Void apply(@Nullable Boolean result) - { - if (result == null || !result) { - log.info("Task [%s] failed to stop in a timely manner, killing task", id); - killTask(id); - } - return null; - } - } - ); - } - - private void killTask(final String id) - { - Optional taskQueue = taskMaster.getTaskQueue(); - if (taskQueue.isPresent()) { - taskQueue.get().shutdown(id); - } else { - log.error("Failed to get task queue because I'm not the leader!"); - } - } - - protected int getTaskGroupIdForPartition(int partition) - { - return partition % ioConfig.getTaskCount(); - } - - private boolean isTaskInPendingCompletionGroups(String taskId) - { - for (List taskGroups : pendingCompletionTaskGroups.values()) { - for (TaskGroup taskGroup : taskGroups) { - if (taskGroup.tasks.containsKey(taskId)) { - return true; - } - } - } - return false; - } - - private SupervisorReport generateReport(boolean includeOffsets) - { - int numPartitions = partitionGroups.values().stream().mapToInt(Map::size).sum(); - - Map partitionLag = getLagPerPartition(getHighestCurrentOffsets()); - final KafkaSupervisorReportPayload payload = new KafkaSupervisorReportPayload( - dataSource, - ioConfig.getTopic(), - numPartitions, - ioConfig.getReplicas(), - ioConfig.getTaskDuration().getMillis() / 1000, - includeOffsets ? latestOffsetsFromKafka : null, - includeOffsets ? partitionLag : null, - includeOffsets ? partitionLag.values().stream().mapToLong(x -> Math.max(x, 0)).sum() : null, - includeOffsets ? offsetsLastUpdated : null, - spec.isSuspended() - ); - SupervisorReport report = new SupervisorReport<>( - dataSource, - DateTimes.nowUtc(), - payload - ); - - List taskReports = Lists.newArrayList(); - - try { - for (TaskGroup taskGroup : taskGroups.values()) { - for (Entry entry : taskGroup.tasks.entrySet()) { - String taskId = entry.getKey(); - @Nullable - DateTime startTime = entry.getValue().startTime; - Map currentOffsets = entry.getValue().currentOffsets; - Long remainingSeconds = null; - if (startTime != null) { - long elapsedMillis = System.currentTimeMillis() - startTime.getMillis(); - long remainingMillis = Math.max(0, ioConfig.getTaskDuration().getMillis() - elapsedMillis); - remainingSeconds = TimeUnit.MILLISECONDS.toSeconds(remainingMillis); - } - - taskReports.add( - new TaskReportData( - taskId, - includeOffsets ? taskGroup.partitionOffsets : null, - includeOffsets ? currentOffsets : null, - startTime, - remainingSeconds, - SeekableStreamTaskReportData.TaskType.ACTIVE, - includeOffsets ? getLagPerPartition(currentOffsets) : null - ) - ); - } - } - - for (List taskGroups : pendingCompletionTaskGroups.values()) { - for (TaskGroup taskGroup : taskGroups) { - for (Entry entry : taskGroup.tasks.entrySet()) { - String taskId = entry.getKey(); - @Nullable - DateTime startTime = entry.getValue().startTime; - Map currentOffsets = entry.getValue().currentOffsets; - Long remainingSeconds = null; - if (taskGroup.completionTimeout != null) { - remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - System.currentTimeMillis()) - / 1000; - } - - taskReports.add( - new TaskReportData( - taskId, - includeOffsets ? taskGroup.partitionOffsets : null, - includeOffsets ? currentOffsets : null, - startTime, - remainingSeconds, - SeekableStreamTaskReportData.TaskType.PUBLISHING, - null - ) - ); - } - } - } - - taskReports.forEach(payload::addTask); - } - catch (Exception e) { - log.warn(e, "Failed to generate status report"); - } - - return report; - } - - private Runnable buildRunTask() - { - return () -> notices.add(new RunNotice()); - } - - private void updateLatestOffsetsFromKafka() - { - synchronized (consumerLock) { - final List partitionInfoList = consumer.partitionsFor(ioConfig.getTopic()); - - if (partitionInfoList == null || partitionInfoList.size() == 0) { - throw new ISE("Could not retrieve partitions for topic [%s]", ioConfig.getTopic()); - } - - final Set topicPartitions = partitionInfoList - .stream() - .map(x -> new TopicPartition(x.topic(), x.partition())) - .collect(Collectors.toSet()); - - consumer.assign(topicPartitions); - consumer.seekToEnd(topicPartitions); - - latestOffsetsFromKafka = topicPartitions - .stream() - .collect(Collectors.toMap(TopicPartition::partition, consumer::position)); - } - } - - private Map getHighestCurrentOffsets() - { - return taskGroups - .values() - .stream() - .flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()) - .flatMap(taskData -> taskData.getValue().currentOffsets.entrySet().stream()) - .collect(Collectors.toMap(Entry::getKey, Entry::getValue, Long::max)); - } - - private Map getLagPerPartition(Map currentOffsets) - { - return currentOffsets - .entrySet() - .stream() - .collect( - Collectors.toMap( - Entry::getKey, - e -> latestOffsetsFromKafka != null - && latestOffsetsFromKafka.get(e.getKey()) != null - && e.getValue() != null - ? latestOffsetsFromKafka.get(e.getKey()) - e.getValue() - : Integer.MIN_VALUE - ) - ); - } - - private Runnable emitLag() - { - return () -> { - try { - Map highestCurrentOffsets = getHighestCurrentOffsets(); - - if (latestOffsetsFromKafka == null) { - throw new ISE("Latest offsets from Kafka have not been fetched"); - } - - if (!latestOffsetsFromKafka.keySet().equals(highestCurrentOffsets.keySet())) { - log.warn( - "Lag metric: Kafka partitions %s do not match task partitions %s", - latestOffsetsFromKafka.keySet(), - highestCurrentOffsets.keySet() - ); - } - - long lag = getLagPerPartition(highestCurrentOffsets) - .values() - .stream() - .mapToLong(x -> Math.max(x, 0)) - .sum(); - - emitter.emit( - ServiceMetricEvent.builder().setDimension("dataSource", dataSource).build("ingest/kafka/lag", lag) - ); - } - catch (Exception e) { - log.warn(e, "Unable to compute Kafka lag"); - } - }; - } - - private void updateCurrentOffsets() throws InterruptedException, ExecutionException, TimeoutException - { - final List> futures = Stream.concat( - taskGroups.values().stream().flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()), - pendingCompletionTaskGroups.values() - .stream() - .flatMap(List::stream) - .flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()) - ).map( - task -> Futures.transform( - taskClient.getCurrentOffsetsAsync(task.getKey(), false), - (Function, Void>) (currentOffsets) -> { - - if (currentOffsets != null && !currentOffsets.isEmpty()) { - task.getValue().currentOffsets = currentOffsets; - } - - return null; - } - ) - ).collect(Collectors.toList()); - - Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - } - - @VisibleForTesting - Runnable updateCurrentAndLatestOffsets() - { - return () -> { - try { - updateCurrentOffsets(); - updateLatestOffsetsFromKafka(); - offsetsLastUpdated = DateTimes.nowUtc(); - } - catch (Exception e) { - log.warn(e, "Exception while getting current/latest offsets"); - } - }; - } - - /** - * Collect row ingestion stats from all tasks managed by this supervisor. - * - * @return A map of groupId->taskId->task row stats - * - * @throws InterruptedException - * @throws ExecutionException - * @throws TimeoutException - */ - private Map> getCurrentTotalStats() - throws InterruptedException, ExecutionException, TimeoutException - { - Map> allStats = Maps.newHashMap(); - final List> futures = new ArrayList<>(); - final List> groupAndTaskIds = new ArrayList<>(); - - for (int groupId : taskGroups.keySet()) { - TaskGroup group = taskGroups.get(groupId); - for (String taskId : group.taskIds()) { - futures.add( - Futures.transform( - taskClient.getMovingAveragesAsync(taskId), - (Function, StatsFromTaskResult>) (currentStats) -> { - return new StatsFromTaskResult( - groupId, - taskId, - currentStats - ); - } - ) - ); - groupAndTaskIds.add(new Pair<>(groupId, taskId)); - } - } - - for (int groupId : pendingCompletionTaskGroups.keySet()) { - List pendingGroups = pendingCompletionTaskGroups.get(groupId); - for (TaskGroup pendingGroup : pendingGroups) { - for (String taskId : pendingGroup.taskIds()) { - futures.add( - Futures.transform( - taskClient.getMovingAveragesAsync(taskId), - (Function, StatsFromTaskResult>) (currentStats) -> new StatsFromTaskResult( - groupId, - taskId, - currentStats - ) - ) - ); - groupAndTaskIds.add(new Pair<>(groupId, taskId)); - } - } - } - - List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - for (int i = 0; i < results.size(); i++) { - StatsFromTaskResult result = results.get(i); - if (result != null) { - Map groupMap = allStats.computeIfAbsent(result.getGroupId(), k -> Maps.newHashMap()); - groupMap.put(result.getTaskId(), result.getStats()); - } else { - Pair groupAndTaskId = groupAndTaskIds.get(i); - log.error("Failed to get stats for group[%d]-task[%s]", groupAndTaskId.lhs, groupAndTaskId.rhs); - } - } - - return allStats; - } - - @VisibleForTesting - void addTaskGroupToActivelyReadingTaskGroup( - int taskGroupId, - ImmutableMap partitionOffsets, - Optional minMsgTime, - Optional maxMsgTime, - Set tasks - ) - { - TaskGroup group = new TaskGroup( - taskGroupId, - partitionOffsets, - minMsgTime, - maxMsgTime - ); - group.tasks.putAll(tasks.stream().collect(Collectors.toMap(x -> x, x -> new TaskData()))); - if (taskGroups.putIfAbsent(taskGroupId, group) != null) { - throw new ISE( - "trying to add taskGroup with ID [%s] to actively reading task groups, but group already exists.", - taskGroupId - ); - } - } - - @VisibleForTesting - void addTaskGroupToPendingCompletionTaskGroup( - int taskGroupId, - ImmutableMap partitionOffsets, - Optional minMsgTime, - Optional maxMsgTime, - Set tasks - ) - { - TaskGroup group = new TaskGroup( - taskGroupId, - partitionOffsets, - minMsgTime, - maxMsgTime - ); - group.tasks.putAll(tasks.stream().collect(Collectors.toMap(x -> x, x -> new TaskData()))); - pendingCompletionTaskGroups.computeIfAbsent(taskGroupId, x -> new CopyOnWriteArrayList<>()) - .add(group); - } - - @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 final String groupId; - private final String taskId; - private final Map stats; - - public StatsFromTaskResult( - int groupId, - String taskId, - Map stats - ) - { - this.groupId = String.valueOf(groupId); - this.taskId = taskId; - this.stats = stats; - } - - public String getGroupId() - { - return groupId; - } - - public String getTaskId() - { - return taskId; - } - - public Map getStats() - { - return stats; - } - } - - // exposed for testing for visibility into initialization state - @VisibleForTesting - public boolean isStarted() - { - return started; - } - - // exposed for testing for visibility into initialization state - @VisibleForTesting - public boolean isLifecycleStarted() - { - return lifecycleStarted; - } - - // exposed for testing for visibility into initialization state - @VisibleForTesting - public int getInitRetryCounter() - { - return initRetryCounter; - } - - // exposed for testing to allow "bootstrap.servers" to be changed after supervisor is created - @VisibleForTesting - public KafkaSupervisorIOConfig getIoConfig() - { - return ioConfig; - } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java index b1647d182620..2b29e6e5be68 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; +import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index bc058ba20eb5..6c88559ec3fe 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -25,13 +25,13 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import org.apache.druid.guice.annotations.Json; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; @@ -118,6 +118,20 @@ public List getDataSources() return ImmutableList.of(getDataSchema().getDataSource()); } + @Override + @JsonProperty + public KafkaSupervisorTuningConfig getTuningConfig() + { + return (KafkaSupervisorTuningConfig) super.getTuningConfig(); + } + + @Override + @JsonProperty + public KafkaSupervisorIOConfig getIoConfig() + { + return (KafkaSupervisorIOConfig) super.getIoConfig(); + } + @Override public String toString() { diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index 2892c5a544a2..5f624e886e5b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -20,8 +20,8 @@ package org.apache.druid.indexing.kafka.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; import org.apache.druid.indexing.kafka.KafkaTuningConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.TuningConfigs; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -130,6 +130,7 @@ public Duration getShutdownTimeout() return shutdownTimeout; } + @Override @JsonProperty public Duration getOffsetFetchPeriod() { diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/TaskReportData.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/TaskReportData.java deleted file mode 100644 index 5f011d79ab0f..000000000000 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/TaskReportData.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.kafka.supervisor; - -import com.fasterxml.jackson.annotation.JsonInclude; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamTaskReportData; -import org.joda.time.DateTime; - -import javax.annotation.Nullable; -import java.util.Map; - -public class TaskReportData extends SeekableStreamTaskReportData -{ - - private final Map lag; - - public TaskReportData( - String id, - @Nullable Map startingOffsets, - @Nullable Map currentOffsets, - @Nullable DateTime startTime, - Long remainingSeconds, - TaskType type, - @Nullable Map lag - ) - { - super( - id, - startingOffsets, - currentOffsets, - startTime, - remainingSeconds, - type - ); - - this.lag = lag; - } - - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getLag() - { - return lag; - } - - @Override - public String toString() - { - return "{" + - "id='" + getId() + '\'' + - (getStartingOffsets() != null ? ", startingOffsets=" + getStartingOffsets() : "") + - (getCurrentOffsets() != null ? ", currentOffsets=" + getCurrentOffsets() : "") + - ", startTime=" + getStartTime() + - ", remainingSeconds=" + getRemainingSeconds() + - (lag != null ? ", lag=" + lag : "") + - '}'; - } -} diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 81aad11f69ad..117f7703891f 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -72,13 +72,14 @@ import org.apache.druid.indexing.common.task.IndexTaskTest; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor; -import org.apache.druid.indexing.kafka.test.TestBroker; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.MetadataTaskStorage; import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.test.TestBroker; import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; import org.apache.druid.indexing.test.TestDataSegmentKiller; import org.apache.druid.java.util.common.CompressionUtils; @@ -437,7 +438,7 @@ public void testRunBeforeDataInserted() throws Exception final ListenableFuture future = runTask(task); // Wait for the task to start reading - while (task.getRunner().getStatus() != KafkaIndexTask.Status.READING) { + while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { Thread.sleep(10); } @@ -510,7 +511,7 @@ public void testIncrementalHandOff() throws Exception ) ); final ListenableFuture future = runTask(task); - while (task.getRunner().getStatus() != KafkaIndexTask.Status.PAUSED) { + while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { Thread.sleep(10); } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); @@ -722,7 +723,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception final ListenableFuture future = runTask(task); // task will pause for checkpointing - while (task.getRunner().getStatus() != KafkaIndexTask.Status.PAUSED) { + while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { Thread.sleep(10); } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); @@ -782,7 +783,7 @@ public void testRunWithMinimumMessageTime() throws Exception final ListenableFuture future = runTask(task); // Wait for the task to start reading - while (task.getRunner().getStatus() != KafkaIndexTask.Status.READING) { + while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { Thread.sleep(10); } @@ -836,7 +837,7 @@ public void testRunWithMaximumMessageTime() throws Exception final ListenableFuture future = runTask(task); // Wait for the task to start reading - while (task.getRunner().getStatus() != KafkaIndexTask.Status.READING) { + while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { Thread.sleep(10); } @@ -900,7 +901,7 @@ public void testRunWithTransformSpec() throws Exception final ListenableFuture future = runTask(task); // Wait for the task to start reading - while (task.getRunner().getStatus() != KafkaIndexTask.Status.READING) { + while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { Thread.sleep(10); } @@ -1741,7 +1742,7 @@ public void testRunWithPauseAndResume() throws Exception } Assert.assertEquals(2, countEvents(task)); - Assert.assertEquals(KafkaIndexTask.Status.READING, task.getRunner().getStatus()); + Assert.assertEquals(SeekableStreamIndexTask.Status.READING, task.getRunner().getStatus()); Map currentOffsets = objectMapper.readValue( task.getRunner().pause().getEntity().toString(), @@ -1749,7 +1750,7 @@ public void testRunWithPauseAndResume() throws Exception { } ); - Assert.assertEquals(KafkaIndexTask.Status.PAUSED, task.getRunner().getStatus()); + Assert.assertEquals(SeekableStreamIndexTask.Status.PAUSED, task.getRunner().getStatus()); // Insert remaining data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { @@ -1812,13 +1813,13 @@ public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception runTask(task); - while (!task.getRunner().getStatus().equals(KafkaIndexTask.Status.READING)) { + while (!task.getRunner().getStatus().equals(SeekableStreamIndexTask.Status.READING)) { Thread.sleep(2000); } task.getRunner().pause(); - while (!task.getRunner().getStatus().equals(KafkaIndexTask.Status.PAUSED)) { + while (!task.getRunner().getStatus().equals(SeekableStreamIndexTask.Status.PAUSED)) { Thread.sleep(25); } } @@ -1851,12 +1852,12 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva runTask(task); - while (!task.getRunner().getStatus().equals(KafkaIndexTask.Status.READING)) { + while (!task.getRunner().getStatus().equals(SeekableStreamIndexTask.Status.READING)) { Thread.sleep(20); } for (int i = 0; i < 5; i++) { - Assert.assertEquals(task.getRunner().getStatus(), KafkaIndexTask.Status.READING); + Assert.assertEquals(task.getRunner().getStatus(), SeekableStreamIndexTask.Status.READING); // Offset should not be reset Assert.assertTrue(task.getRunner().getCurrentOffsets().get(0) == 200L); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 0ef426381508..188a1ea6534c 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -39,7 +39,6 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -51,7 +50,6 @@ import org.apache.druid.indexing.kafka.KafkaIndexTaskClient; import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; import org.apache.druid.indexing.kafka.KafkaPartitions; -import org.apache.druid.indexing.kafka.test.TestBroker; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; @@ -61,6 +59,9 @@ import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; +import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; +import org.apache.druid.indexing.seekablestream.test.TestBroker; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java index c1a06716a3ce..16ae12a43fa6 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.kafka.test; +package org.apache.druid.indexing.seekablestream.test; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java index 92042a26aa28..ad2cfa905017 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.google.common.annotations.VisibleForTesting; @@ -140,6 +141,11 @@ protected RetryPolicy newRetryPolicy() return retryPolicyFactory.makeRetryPolicy(); } + protected T deserialize(String content, JavaType type) throws IOException + { + return objectMapper.readValue(content, type); + } + protected T deserialize(String content, TypeReference typeReference) throws IOException { return objectMapper.readValue(content, typeReference); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java index 244429e15a3c..fd094b26375e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java @@ -16,6 +16,24 @@ * specific language governing permissions and limitations * under the License. */ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ package org.apache.druid.indexing.seekablestream; @@ -25,7 +43,7 @@ import java.util.Objects; -abstract public class SeekableStreamDataSourceMetadata implements DataSourceMetadata +public abstract class SeekableStreamDataSourceMetadata implements DataSourceMetadata { private final SeekableStreamPartitions seekableStreamPartitions; @@ -60,13 +78,13 @@ public boolean matches(DataSourceMetadata other) } @Override - abstract public DataSourceMetadata plus(DataSourceMetadata other); + public abstract DataSourceMetadata plus(DataSourceMetadata other); @Override - abstract public DataSourceMetadata minus(DataSourceMetadata other); + public abstract DataSourceMetadata minus(DataSourceMetadata other); @Override - abstract public boolean equals(Object o); + public abstract boolean equals(Object o); @Override public int hashCode() @@ -75,5 +93,5 @@ public int hashCode() } @Override - abstract public String toString(); + public abstract String toString(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java index 4b132d7c36a8..3745746c35d6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java @@ -27,8 +27,9 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; +import java.util.Set; -abstract public class SeekableStreamIOConfig implements IOConfig +public abstract class SeekableStreamIOConfig implements IOConfig { private static final boolean DEFAULT_USE_TRANSACTION = true; @@ -104,6 +105,9 @@ public Optional getMinimumMessageTime() return minimumMessageTime; } + @JsonProperty + public abstract Set getExclusiveStartSequenceNumberPartitions(); + @Override - abstract public String toString(); + public abstract String toString(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index a077a4e3d7e2..176305177daa 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -48,22 +48,11 @@ import java.util.Random; //TODO: need more refactoring for run() -abstract public class SeekableStreamIndexTask extends AbstractTask implements ChatHandler +public abstract class SeekableStreamIndexTask extends AbstractTask implements ChatHandler { - public enum Status - { - NOT_STARTED, - STARTING, - READING, - PAUSED, - PUBLISHING - } - private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTask.class); private static final Random RANDOM = new Random(); private static final String TYPE = "index_seekable_stream"; - - protected final DataSchema dataSchema; protected final InputRowParser parser; protected final SeekableStreamTuningConfig tuningConfig; @@ -149,17 +138,25 @@ public SeekableStreamIOConfig getIOConfig() return ioConfig; } - @Override - abstract public TaskStatus run(final TaskToolbox toolbox); + public abstract TaskStatus run(TaskToolbox toolbox); @Override - abstract public boolean canRestore(); + public abstract boolean canRestore(); @Override - abstract public void stopGracefully(); + public abstract void stopGracefully(); @Override - abstract public QueryRunner getQueryRunner(Query query); + public abstract QueryRunner getQueryRunner(Query query); + + public enum Status + { + NOT_STARTED, + STARTING, + READING, + PAUSED, + PUBLISHING + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 66b4e73404c4..3d359643f480 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -42,7 +42,7 @@ import java.util.Map; import java.util.TreeMap; -abstract public class SeekableStreamIndexTaskClient extends IndexTaskClient +public abstract class SeekableStreamIndexTaskClient extends IndexTaskClient { private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTaskClient.class); @@ -349,7 +349,9 @@ public ListenableFuture getStatusAsync(final Str return doAsync(() -> getStatus(id)); } - abstract protected JavaType constructMapType(Class mapType); + protected abstract JavaType constructMapType(Class mapType); + + public abstract Class getTaskClassType(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientFactory.java index 15cf078b6dcc..6a909704bdaa 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientFactory.java @@ -28,7 +28,7 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; -abstract public class SeekableStreamIndexTaskClientFactory +public abstract class SeekableStreamIndexTaskClientFactory implements IndexTaskClientFactory { private HttpClient httpClient; @@ -45,7 +45,7 @@ public SeekableStreamIndexTaskClientFactory( } @Override - abstract public T build( + public abstract T build( TaskInfoProvider taskInfoProvider, String dataSource, int numThreads, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 7e2d7ea99af1..b0fc2c478e39 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -18,6 +18,7 @@ */ package org.apache.druid.indexing.seekablestream; + import com.google.common.annotations.VisibleForTesting; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; @@ -54,11 +55,11 @@ public interface SeekableStreamIndexTaskRunner extends ChatHandler Response setEndOffsets( Map offsets, boolean finish // this field is only for internal purposes, shouldn't be usually set by users - ) throws InterruptedException; + ); @VisibleForTesting - Response pause() throws InterruptedException; + Response pause(); @VisibleForTesting - void resume() throws InterruptedException; + void resume(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index 313fc5c2853b..e8f9dc73fa92 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -26,7 +26,7 @@ import java.util.Map; import java.util.Objects; -abstract public class SeekableStreamPartitions +public abstract class SeekableStreamPartitions { private final String id; private final Map partitionOffsetMap; @@ -72,6 +72,6 @@ public int hashCode() } @Override - abstract public String toString(); + public abstract String toString(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java index 768e04639ac7..78fb9cdd6942 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java @@ -32,7 +32,7 @@ import java.io.File; import java.util.Objects; -abstract public class SeekableStreamTuningConfig implements TuningConfig, AppenderatorConfig +public abstract class SeekableStreamTuningConfig implements TuningConfig, AppenderatorConfig { private static final int DEFAULT_MAX_ROWS_PER_SEGMENT = 5_000_000; private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false; @@ -121,7 +121,7 @@ public SeekableStreamTuningConfig( : logParseExceptions; } - abstract public SeekableStreamTuningConfig copyOf(); + public abstract SeekableStreamTuningConfig copyOf(); @Override @JsonProperty @@ -233,7 +233,7 @@ public int getMaxSavedParseExceptions() return maxSavedParseExceptions; } - abstract public SeekableStreamTuningConfig withBasePersistDirectory(File dir); + public abstract SeekableStreamTuningConfig withBasePersistDirectory(File dir); @Override public boolean equals(Object o) @@ -285,6 +285,6 @@ public int hashCode() } @Override - abstract public String toString(); + public abstract String toString(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java index 7192a12a7d4b..85dee67f72f2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.indexing.seekablestream.common; import java.util.List; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java index ecdb251d11de..42e3b96e626f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java @@ -1,6 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.indexing.seekablestream.common; import java.io.Closeable; +import java.util.Collection; import java.util.Set; import java.util.concurrent.TimeoutException; @@ -18,9 +38,11 @@ public interface RecordSupplier extends Closeable void seekAfter(StreamPartition partition, T2 sequenceNumber); - void seekToEarliest(StreamPartition partition); + void seekToEarliest(Set> partition); + + void seekToLatest(Set> partition); - void seekToLatest(StreamPartition partition); + Collection> getAssignment(); Record poll(long timeout); @@ -28,8 +50,11 @@ public interface RecordSupplier extends Closeable T2 getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException; + T2 position(StreamPartition partition); + Set getPartitionIds(String streamName); + @Override void close(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java index 49cd782ccaad..57163560430a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.indexing.seekablestream.common; @@ -12,6 +31,11 @@ private SequenceNumberPlus(String sequenceNumber, boolean exclusive) this.exclusive = exclusive; } + public static SequenceNumberPlus of(String sequenceNumber, boolean exclusive) + { + return new SequenceNumberPlus(sequenceNumber, exclusive); + } + public String get() { return sequenceNumber; @@ -21,9 +45,4 @@ public boolean isExclusive() { return exclusive; } - - public static SequenceNumberPlus of(String sequenceNumber, boolean exclusive) - { - return new SequenceNumberPlus(sequenceNumber, exclusive); - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java index 1bece8f3c2d1..7a6db473aa8d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.indexing.seekablestream.common; public class StreamPartition @@ -11,6 +30,11 @@ public StreamPartition(String streamName, T1 partitionId) this.partitionId = partitionId; } + public static StreamPartition of(String streamName, T1 partitionId) + { + return new StreamPartition<>(streamName, partitionId); + } + public String getStreamName() { return streamName; @@ -55,10 +79,5 @@ public String toString() ", partitionId='" + partitionId + '\'' + '}'; } - - public static StreamPartition of(String streamName, T1 partitionId) - { - return new StreamPartition<>(streamName, partitionId); - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 386934b4a5be..6c3171d0a548 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.indexing.seekablestream.supervisor; import com.fasterxml.jackson.core.JsonProcessingException; @@ -15,6 +34,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; @@ -22,12 +42,6 @@ import org.apache.commons.codec.digest.DigestUtils; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; -import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; -import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.common.IndexTaskClient; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -42,6 +56,15 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; +import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -49,14 +72,14 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; +import org.apache.druid.metadata.EntryExistsException; import org.joda.time.DateTime; import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -77,98 +100,31 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import java.util.stream.Stream; //TODO: rename offset -> sequence //TODO: prune 'kafka' and 'kinesis' //TODO: inheritance //TODO: make classes abstract //TODO: resolve warnings -abstract public class SeekableStreamSupervisor implements Supervisor +public abstract class SeekableStreamSupervisor, T2 extends Comparable> + implements Supervisor { - //---------------------------------------GOOD--------------------------------------------------------- + public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED"; private static final EmittingLogger log = new EmittingLogger(SeekableStreamSupervisor.class); private static final Random RANDOM = new Random(); private static final long MAX_RUN_FREQUENCY_MILLIS = 1000; private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120; private static final CopyOnWriteArrayList EMPTY_LIST = Lists.newCopyOnWriteArrayList(); - //---------------------------------------GOOD--------------------------------------------------------- - - - public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED"; - - private class TaskGroup - { - final int groupId; - - final ImmutableMap partitionOffsets; - final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); - final Optional minimumMessageTime; - final Optional maximumMessageTime; - final Set exclusiveStartSequenceNumberPartitions; //TODO: exclusiveSequence - final TreeMap> sequenceOffsets = new TreeMap<>(); - final String baseSequenceName; - DateTime completionTimeout; - - public TaskGroup( - int groupId, - ImmutableMap partitionOffsets, - Optional minimumMessageTime, - Optional maximumMessageTime, - Set exclusiveStartSequenceNumberPartitions - ) - { - this.groupId = groupId; - this.partitionOffsets = partitionOffsets; - this.minimumMessageTime = minimumMessageTime; - this.maximumMessageTime = maximumMessageTime; - this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions != null - ? exclusiveStartSequenceNumberPartitions - : new HashSet<>(); - this.baseSequenceName = generateSequenceName(partitionOffsets, minimumMessageTime, maximumMessageTime); - } - - int addNewCheckpoint(Map checkpoint) - { - sequenceOffsets.put(sequenceOffsets.lastKey() + 1, checkpoint); - return sequenceOffsets.lastKey(); - } - - public Set taskIds() - { - return tasks.keySet(); - } - } - - private class TaskData - { - volatile TaskStatus status; - volatile DateTime startTime; - volatile Map currentOffsets = new HashMap<>(); - - @Override - public String toString() - { - return "TaskData{" + - "status=" + status + - ", startTime=" + startTime + - ", currentOffsets=" + currentOffsets + - '}'; - } - } - - //TODO: doc - private final ConcurrentHashMap taskGroups = new ConcurrentHashMap<>(); - private final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); - private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); - + protected final ConcurrentHashMap taskGroups = new ConcurrentHashMap<>(); + protected final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); + protected final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); private final TaskStorage taskStorage; private final TaskMaster taskMaster; private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; private final SeekableStreamIndexTaskClient taskClient; private final ObjectMapper sortingMapper; private final SeekableStreamSupervisorSpec spec; - private final ServiceEmitter emitter; - private final DruidMonitorSchedulerConfig monitorSchedulerConfig; private final String dataSource; private final SeekableStreamSupervisorIOConfig ioConfig; private final SeekableStreamSupervisorTuningConfig tuningConfig; @@ -177,7 +133,6 @@ public String toString() private final TaskInfoProvider taskInfoProvider; private final long futureTimeoutInSeconds; // how long to wait for async operations to complete private final RowIngestionMetersFactory rowIngestionMetersFactory; - private final ExecutorService exec; private final ScheduledExecutorService scheduledExec; private final ScheduledExecutorService reportingExec; @@ -185,84 +140,21 @@ public String toString() private final BlockingQueue notices = new LinkedBlockingDeque<>(); private final Object stopLock = new Object(); private final Object stateChangeLock = new Object(); - private final Object consumerLock = new Object(); //TODO: prob not needed - + private final Object recordSupplierLock = new Object(); + private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); + private final boolean checkpointSupported; + private final T2 NOT_SET; + private final T2 MAX_SEQUENCE_NUMBER; + protected volatile Map latestOffsetsFromStream; + protected volatile DateTime offsetsLastUpdated; private boolean listenerRegistered = false; private long lastRunTime; - private volatile DateTime firstRunTime; private volatile DateTime earlyPublishTime = null; private volatile RecordSupplier recordSupplier; private volatile boolean started = false; private volatile boolean stopped = false; - private final T2 NOT_SET; - - //-------------------------------GOOD---------------------------------------- - - /** - * Notice is used to queue tasks that are internal to the supervisor - */ - private interface Notice - { - void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException; - } - - private class RunNotice implements Notice - { - @Override - public void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException - { - long nowTime = System.currentTimeMillis(); - if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { - return; - } - lastRunTime = nowTime; - - runInternal(); - } - } - - private class GracefulShutdownNotice extends ShutdownNotice - { - @Override - public void handle() throws InterruptedException, ExecutionException, TimeoutException - { - gracefulShutdownInternal(); - super.handle(); - } - } - - private class ShutdownNotice implements Notice - { - @Override - public void handle() throws InterruptedException, ExecutionException, TimeoutException - { - recordSupplier.close(); - - synchronized (stopLock) { - stopped = true; - stopLock.notifyAll(); - } - } - } - - private class ResetNotice implements Notice - { - final DataSourceMetadata dataSourceMetadata; - - ResetNotice(DataSourceMetadata dataSourceMetadata) - { - this.dataSourceMetadata = dataSourceMetadata; - } - - @Override - public void handle() - { - resetInternal(dataSourceMetadata); - } - } - //---------------------------GOOD----------------------------------------- public SeekableStreamSupervisor( final String supervisorId, @@ -273,7 +165,9 @@ public SeekableStreamSupervisor( final ObjectMapper mapper, final SeekableStreamSupervisorSpec spec, final RowIngestionMetersFactory rowIngestionMetersFactory, - final T2 NOT_SET + final T2 NOT_SET, // TODO: fix these flags and extra variables + final T2 MAX_SEQUENCE_NUMBER, + final boolean checkpointSupported ) { this.taskStorage = taskStorage; @@ -281,10 +175,10 @@ public SeekableStreamSupervisor( this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; this.sortingMapper = mapper.copy().configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); this.spec = spec; - this.emitter = spec.getEmitter(); - this.monitorSchedulerConfig = spec.getMonitorSchedulerConfig(); this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.NOT_SET = NOT_SET; + this.MAX_SEQUENCE_NUMBER = MAX_SEQUENCE_NUMBER; //TODO: placeholder till fix + this.checkpointSupported = checkpointSupported; this.dataSource = spec.getDataSchema().getDataSource(); this.ioConfig = spec.getIoConfig(); @@ -358,6 +252,17 @@ public Optional getTaskStatus(String id) ); } + protected static String getRandomId() + { + final StringBuilder suffix = new StringBuilder(8); + for (int i = 0; i < Integer.BYTES * 2; ++i) { + suffix.append((char) ('a' + ((RANDOM.nextInt() >>> (i * 4)) & 0x0F))); + } + return suffix.toString(); + } + + //-------------------------------GOOD---------------------------------------- + //-------------------------------GOOD---------------------------------------- @Override public void start() @@ -367,7 +272,10 @@ public void start() Preconditions.checkState(!exec.isShutdown(), "already stopped"); try { - recordSupplier = setupRecordSupplier(); + if (recordSupplier == null) { + recordSupplier = setupRecordSupplier(); + } + exec.submit( () -> { @@ -398,9 +306,7 @@ public void start() TimeUnit.MILLISECONDS ); - - //TODO: this logic is different from kafka and kinesis, need to look into - scheduleReporting(); + scheduleReporting(reportingExec); started = true; log.info( @@ -422,7 +328,6 @@ public void start() } } - //---------------------------GOOOD------------------------------ @Override public void stop(boolean stopGracefully) @@ -483,15 +388,28 @@ public void stop(boolean stopGracefully) } } + @Override + public Map> getStats() + { + try { + return getCurrentTotalStats(); + } + catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + log.error(ie, "getStats() interrupted."); + throw new RuntimeException(ie); + } + catch (ExecutionException | TimeoutException eete) { + throw new RuntimeException(eete); + } + } - //-------------------------------GOOD---------------------------------------- @Override public SupervisorReport getStatus() { return generateReport(true); } - //-------------------------------GOOD---------------------------------------- @Override public void reset(DataSourceMetadata dataSourceMetadata) { @@ -501,16 +419,15 @@ public void reset(DataSourceMetadata dataSourceMetadata) //TODO: checkpoints @Override - abstract public void checkpoint( + public abstract void checkpoint( @Nullable Integer taskGroupId, String baseSequenceName, DataSourceMetadata previousCheckPoint, DataSourceMetadata currentCheckPoint ); - //-------------------------------GOOD---------------------------------------- @VisibleForTesting - void runInternal() + protected void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException { possiblyRegisterListener(); @@ -567,7 +484,7 @@ public void statusChanged(String taskId, TaskStatus status) //-------------------------------GOOD---------------------------------------- @VisibleForTesting - void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException + protected void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException { for (TaskGroup taskGroup : taskGroups.values()) { for (Map.Entry entry : @@ -585,7 +502,7 @@ void gracefulShutdownInternal() throws ExecutionException, InterruptedException, //-------------------------------GOOD---------------------------------------- @VisibleForTesting - void resetInternal(DataSourceMetadata dataSourceMetadata) + protected void resetInternal(DataSourceMetadata dataSourceMetadata) { if (dataSourceMetadata == null) { // Reset everything @@ -596,15 +513,18 @@ void resetInternal(DataSourceMetadata dataSourceMetadata) partitionGroups.clear(); } else { - checkSourceMetadataInstanceMatch(dataSourceMetadata); - + if (!checkSourceMetaDataMatch(dataSourceMetadata)) { + throw new IAE("Expected KafkaDataSourceMetadata but found instance of [%s]", dataSourceMetadata.getClass()); + } // Reset only the partitions in dataSourceMetadata if it has not been reset yet final SeekableStreamDataSourceMetadata resetMetadata = (SeekableStreamDataSourceMetadata) dataSourceMetadata; if (resetMetadata.getSeekableStreamPartitions().getId().equals(ioConfig.getId())) { // metadata can be null final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); - checkSourceMetadataInstanceMatch(metadata); + if (metadata != null && !checkSourceMetaDataMatch(metadata)) { + throw new IAE("Expected KafkaDataSourceMetadata but found instance of [%s]", metadata.getClass()); + } final SeekableStreamDataSourceMetadata currentMetadata = (SeekableStreamDataSourceMetadata) metadata; @@ -623,8 +543,8 @@ void resetInternal(DataSourceMetadata dataSourceMetadata) getTaskGroupIdForPartition(resetPartitionOffset.getKey()) ); final boolean isSameOffset = partitionTaskGroup != null - && partitionTaskGroup.partitionOffsets.get(resetPartitionOffset.getKey()) - .equals(resetPartitionOffset.getValue()); + && partitionTaskGroup.startingSequences.get(resetPartitionOffset.getKey()) + .equals(resetPartitionOffset.getValue()); if (partitionOffsetInMetadataStore != null || isSameOffset) { doReset = true; break; @@ -671,7 +591,6 @@ void resetInternal(DataSourceMetadata dataSourceMetadata) } - private void killTask(final String id) { Optional taskQueue = taskMaster.getTaskQueue(); @@ -711,7 +630,6 @@ private boolean isTaskInPendingCompletionGroups(String taskId) return false; } - private void discoverTasks() throws ExecutionException, InterruptedException, TimeoutException { int taskCount = 0; @@ -721,7 +639,7 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti final Map taskGroupsToVerify = new HashMap<>(); for (Task task : tasks) { - if (!isTaskInstanceOfThis(task) || !dataSource.equals(task.getDataSource())) { + if (!checkTaskInstance(task) || !dataSource.equals(task.getDataSource())) { continue; } @@ -841,7 +759,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) ), seekableStreamIndexTask.getIOConfig().getMinimumMessageTime(), seekableStreamIndexTask.getIOConfig().getMaximumMessageTime(), - null //TODO: fix + seekableStreamIndexTask.getIOConfig().getExclusiveStartSequenceNumberPartitions() ); } ); @@ -880,7 +798,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) killTask(taskId); } } - log.debug("Found [%d] Kafka indexing tasks for dataSource [%s]", taskCount, dataSource); + log.debug("Found [%d] seekable stream indexing tasks for dataSource [%s]", taskCount, dataSource); // make sure the checkpoints are consistent with each other and with the metadata store verifyAndMergeCheckpoints(taskGroupsToVerify.values()); @@ -904,8 +822,6 @@ private void verifyAndMergeCheckpoints(final Collection taskGroupsToV } } - //TODO: prob wanna refactor this - /** * This method does two things - * 1. Makes sure the checkpoints information in the taskGroup is consistent with that of the tasks, if not kill @@ -999,8 +915,8 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) taskCheckpoints.tailMap(earliestConsistentSequenceId.get()) ); log.info("Setting taskGroup sequences to [%s] for group [%d]", latestCheckpoints, groupId); - taskGroup.sequenceOffsets.clear(); - taskGroup.sequenceOffsets.putAll(latestCheckpoints); + taskGroup.currentSequences.clear(); + taskGroup.currentSequences.putAll(latestCheckpoints); } else { log.debug( "Adding task [%s] to kill list, checkpoints[%s], latestoffsets from DB [%s]", @@ -1012,16 +928,16 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) } } else { // check consistency with taskGroup sequences - if (taskCheckpoints.get(taskGroup.sequenceOffsets.firstKey()) == null - || !(taskCheckpoints.get(taskGroup.sequenceOffsets.firstKey()) - .equals(taskGroup.sequenceOffsets.firstEntry().getValue())) - || taskCheckpoints.tailMap(taskGroup.sequenceOffsets.firstKey()).size() - != taskGroup.sequenceOffsets.size()) { + if (taskCheckpoints.get(taskGroup.currentSequences.firstKey()) == null + || !(taskCheckpoints.get(taskGroup.currentSequences.firstKey()) + .equals(taskGroup.currentSequences.firstEntry().getValue())) + || taskCheckpoints.tailMap(taskGroup.currentSequences.firstKey()).size() + != taskGroup.currentSequences.size()) { log.debug( "Adding task [%s] to kill list, checkpoints[%s], taskgroup checkpoints [%s]", taskId, taskCheckpoints, - taskGroup.sequenceOffsets + taskGroup.currentSequences ); tasksToKill.add(taskId); } @@ -1045,7 +961,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) + "persisted offsets in metadata store [%s]", sequenceCheckpoint.lhs, sequenceCheckpoint.rhs, - taskGroup.sequenceOffsets, + taskGroup.currentSequences, latestOffsetsFromDb ); killTask(sequenceCheckpoint.lhs); @@ -1065,7 +981,7 @@ private void addDiscoveredTaskToPendingCompletionTaskGroups( k -> new CopyOnWriteArrayList<>() ); for (TaskGroup taskGroup : taskGroupList) { - if (taskGroup.partitionOffsets.equals(startingPartitions)) { + if (taskGroup.startingSequences.equals(startingPartitions)) { if (taskGroup.tasks.putIfAbsent(taskId, new TaskData()) == null) { log.info("Added discovered task [%s] to existing pending task group [%s]", taskId, groupId); } @@ -1113,7 +1029,7 @@ public Void apply(@Nullable Boolean result) private boolean isTaskCurrent(int taskGroupId, String taskId) { Optional taskOptional = taskStorage.getTask(taskId); - if (!taskOptional.isPresent() || !(isTaskInstanceOfThis(taskOptional.get()))) { + if (!taskOptional.isPresent() || !checkTaskInstance(taskOptional.get())) { return false; } @@ -1134,7 +1050,7 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) } } - String generateSequenceName( + protected String generateSequenceName( Map startPartitions, Optional minimumMessageTime, Optional maximumMessageTime @@ -1169,8 +1085,41 @@ String generateSequenceName( return Joiner.on("_").join("index_seekable_streaming", dataSource, hashCode); } - // TODO: refactor this with recordSupplier - abstract protected void updatePartitionDataFromStream(); + private void updatePartitionDataFromStream() + { + Set partitionIds = null; + try { + synchronized (recordSupplierLock) { + partitionIds = recordSupplier.getPartitionIds(ioConfig.getId()); + } + } + catch (Exception e) { + log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getId()); + return; + } + + log.debug("Found [%d] partitions for topic [%s]", partitionIds.size(), ioConfig.getId()); + + for (T1 partitionId : partitionIds) { + int taskGroupId = getTaskGroupIdForPartition(partitionId); + // TODO: check for closed partitions (not sure if can be done in Kafka) + // TODO: earlyPublishTime + + ConcurrentHashMap partitionMap = partitionGroups.computeIfAbsent( + taskGroupId, + k -> new ConcurrentHashMap<>() + ); + + if (partitionMap.putIfAbsent(partitionId, NOT_SET) == null) { + log.info( + "New partition [%s] discovered for topic [%s], added to task group [%d]", + partitionId, + ioConfig.getId(), + taskGroupId + ); + } + } + } private void updateTaskStatus() throws ExecutionException, InterruptedException, TimeoutException { @@ -1240,6 +1189,8 @@ public Boolean apply(@Nullable DateTime startTime) } } + //TODO: prob wanna refactor this + private Runnable buildRunTask() { return () -> notices.add(new RunNotice()); @@ -1257,13 +1208,19 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException // find the longest running task from this group DateTime earliestTaskStart = DateTimes.nowUtc(); for (TaskData taskData : group.tasks.values()) { - // startTime can be null if kafkaSupervisor is stopped gracefully before processing any runNotice if (taskData.startTime != null && earliestTaskStart.isAfter(taskData.startTime)) { earliestTaskStart = taskData.startTime; } } // TODO: early publish time +// boolean doEarlyPublish = false; +// if (earlyPublishTime != null && (earlyPublishTime.isBeforeNow() || earlyPublishTime.isEqualNow())) { +// log.info("Early publish requested - signalling tasks to publish"); +// +// earlyPublishTime = null; +// doEarlyPublish = true; +// } // if this task has run longer than the configured duration, signal all tasks in the group to persist if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { @@ -1317,7 +1274,6 @@ private ListenableFuture> checkpointTaskGroup(final TaskGroup taskGr String taskId = taskEntry.getKey(); TaskData task = taskEntry.getValue(); - // task.status can be null if kafkaSupervisor is stopped gracefully before processing any runNotice. if (task.status != null) { if (task.status.isSuccess()) { // If any task in this group has already completed, stop the rest of the tasks in the group and return. @@ -1396,11 +1352,11 @@ public Map apply(List> input) try { - if (endOffsets.equals(taskGroup.sequenceOffsets.lastEntry().getValue())) { + if (endOffsets.equals(taskGroup.currentSequences.lastEntry().getValue())) { log.warn( "Checkpoint [%s] is same as the start offsets [%s] of latest sequence for the task group [%d]", endOffsets, - taskGroup.sequenceOffsets.lastEntry().getValue(), + taskGroup.currentSequences.lastEntry().getValue(), taskGroup.groupId ); } @@ -1626,7 +1582,7 @@ private void createNewTasks() throws JsonProcessingException groupId, new TaskGroup( groupId, - generateStartingSequence(groupId), + generateStartingSequencesForPartitionGroup(groupId), minimumMessageTime, maximumMessageTime, null //TODO: exclusive sequence @@ -1649,7 +1605,7 @@ private void createNewTasks() throws JsonProcessingException Integer groupId = entry.getKey(); //TODO: kinesis -// if (taskGroup.partitionOffsets == null || taskGroup.partitionOffsets +// if (taskGroup.startingSequences == null || taskGroup.startingSequences // .values().stream().allMatch(x -> x == null || Record.END_OF_SHARD_MARKER.equals(x))) { // log.debug("Nothing to read in any partition for taskGroup [%d], skipping task creation", groupId); // continue; @@ -1673,24 +1629,736 @@ private void createNewTasks() throws JsonProcessingException } - abstract protected void createTasksForGroup(int group, int replicas) throws JsonProcessingException; + protected void addNotice(Notice notice) + { + notices.add(notice); + } + + @VisibleForTesting + @Nullable + protected TaskGroup removeTaskGroup(int taskGroupId) + { + return taskGroups.remove(taskGroupId); + } + + @VisibleForTesting + protected void moveTaskGroupToPendingCompletion(int taskGroupId) + { + final TaskGroup taskGroup = taskGroups.remove(taskGroupId); + if (taskGroup != null) { + pendingCompletionTaskGroups.computeIfAbsent(taskGroupId, k -> new CopyOnWriteArrayList<>()).add(taskGroup); + } + } + + @VisibleForTesting + protected int getNoticesQueueSize() + { + return notices.size(); + } + + /** + * Collect row ingestion stats from all tasks managed by this supervisor. + * + * @return A map of groupId->taskId->task row stats + * + * @throws InterruptedException + * @throws ExecutionException + * @throws TimeoutException + */ + private Map> getCurrentTotalStats() + throws InterruptedException, ExecutionException, TimeoutException + { + Map> allStats = Maps.newHashMap(); + final List> futures = new ArrayList<>(); + final List> groupAndTaskIds = new ArrayList<>(); + + for (int groupId : taskGroups.keySet()) { + TaskGroup group = taskGroups.get(groupId); + for (String taskId : group.taskIds()) { + futures.add( + Futures.transform( + taskClient.getMovingAveragesAsync(taskId), + (Function, StatsFromTaskResult>) (currentStats) -> { + return new StatsFromTaskResult( + groupId, + taskId, + currentStats + ); + } + ) + ); + groupAndTaskIds.add(new Pair<>(groupId, taskId)); + } + } + + for (int groupId : pendingCompletionTaskGroups.keySet()) { + TaskGroup group = taskGroups.get(groupId); + for (String taskId : group.taskIds()) { + futures.add( + Futures.transform( + taskClient.getMovingAveragesAsync(taskId), + (Function, StatsFromTaskResult>) (currentStats) -> { + return new StatsFromTaskResult( + groupId, + taskId, + currentStats + ); + } + ) + ); + groupAndTaskIds.add(new Pair<>(groupId, taskId)); + } + } + + List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + for (int i = 0; i < results.size(); i++) { + StatsFromTaskResult result = results.get(i); + if (result != null) { + Map groupMap = allStats.computeIfAbsent(result.getGroupId(), k -> Maps.newHashMap()); + groupMap.put(result.getTaskId(), result.getStats()); + } else { + Pair groupAndTaskId = groupAndTaskIds.get(i); + log.error("Failed to get stats for group[%d]-task[%s]", groupAndTaskId.lhs, groupAndTaskId.rhs); + } + } + + return allStats; + } + + private ImmutableMap generateStartingSequencesForPartitionGroup(int groupId) throws TimeoutException + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (Map.Entry entry : partitionGroups.get(groupId).entrySet()) { + T1 partition = entry.getKey(); + T2 offset = entry.getValue(); + + if (offset != null && !offset.equals(NOT_SET)) { + // if we are given a startingOffset (set by a previous task group which is pending completion) then use it + builder.put(partition, offset); + } else { + // if we don't have a startingOffset (first run or we had some previous failures and reset the offsets) then + // get the offset from metadata storage (if available) or Kafka (otherwise) + builder.put(partition, getOffsetFromStorageForPartition(partition)); + } + } + return builder.build(); + } + + /** + * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't + * find any data, it will retrieve the latest or earliest Kafka offset depending on the useEarliestOffset config. + */ + private T2 getOffsetFromStorageForPartition(T1 partition) throws TimeoutException + { + final Map metadataOffsets = getOffsetsFromMetadataStorage(); + T2 offset = metadataOffsets.get(partition); + if (offset != null) { + log.debug("Getting offset [%s] from metadata storage for partition [%s]", offset, partition); - abstract protected ImmutableMap generateStartingSequence(int groupId) - throws TimeoutException; + // TODO: tuningConfig.isSkipSequenceNumberAvailabilityCheck() - abstract protected RecordSupplier setupRecordSupplier(); + try { + T2 latestSequence = getOffsetFromStreamForPartition(partition, false); + if (latestSequence == null || offset.compareTo(latestSequence) > 0) { + if (taskTuningConfig.isResetOffsetAutomatically()) { + // TODO: reset internal + throw new ISE( + "Previous sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) - automatically resetting offset", + offset, + partition, + latestSequence + ); - abstract protected SeekableStreamSupervisorReportPayload createSeekableStreamSupervisorReportPayload(); + } else { + throw new ISE( + "Previous sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]). You can clear the previous sequenceNumber and start reading from a valid message by using the supervisor's reset API.", + offset, + partition, + latestSequence + ); + } + } + } + catch (TimeoutException e) { + throw new ISE(e, "Timeout while fetching earliest sequence number for partition [%s]", partition); + } - abstract protected void scheduleReporting(); + return offset; + } else { + boolean useEarliestSequenceNumber = ioConfig.isUseEarliestSequenceNumber(); + if (subsequentlyDiscoveredPartitions.contains(partition)) { + log.info( + "Overriding useEarliestSequenceNumber and starting from beginning of newly discovered partition [%s] (which is probably from a split or merge)", + partition + ); + useEarliestSequenceNumber = true; + } - abstract protected int getTaskGroupIdForPartition(T1 partition); + offset = getOffsetFromStreamForPartition(partition, useEarliestSequenceNumber); + log.info("Getting sequence number [%s] for partition [%s]", offset, partition); + return offset; + } + } - abstract protected void checkSourceMetadataInstanceMatch(DataSourceMetadata metadata) throws IAE; + private Map getOffsetsFromMetadataStorage() + { + final DataSourceMetadata dataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); + if (dataSourceMetadata instanceof SeekableStreamDataSourceMetadata + && checkSourceMetaDataMatch(dataSourceMetadata)) { + SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata).getSeekableStreamPartitions(); + if (partitions != null) { + if (!ioConfig.getId().equals(partitions.getId())) { + log.warn( + "Topic in metadata storage [%s] doesn't match spec topic [%s], ignoring stored offsets", + partitions.getId(), + ioConfig.getId() + ); + return Collections.emptyMap(); + } else if (partitions.getPartitionSequenceMap() != null) { + return partitions.getPartitionSequenceMap(); + } + } + } - // TODO: may want to put more logic in the base class - abstract protected SupervisorReport generateReport(boolean includeOffsets); + return Collections.emptyMap(); + } - abstract protected boolean isTaskInstanceOfThis(Task task); + private T2 getOffsetFromStreamForPartition(T1 partition, boolean useEarliestOffset) throws TimeoutException + { + synchronized (recordSupplierLock) { + StreamPartition topicPartition = new StreamPartition<>(ioConfig.getId(), partition); + if (!recordSupplier.getAssignment().contains(topicPartition)) { + recordSupplier.assign(Collections.singleton(topicPartition)); + } + + return useEarliestOffset + ? recordSupplier.getEarliestSequenceNumber(topicPartition) + : recordSupplier.getLatestSequenceNumber(topicPartition); + } + } + + private void createTasksForGroup(int groupId, int replicas) throws JsonProcessingException + { + TaskGroup group = taskGroups.get(groupId); + Map startPartitions = group.startingSequences; + Map endPartitions = new HashMap<>(); + Set exclusiveStartSequenceNumberPartitions = taskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; + for (T1 partition : startPartitions.keySet()) { + endPartitions.put(partition, MAX_SEQUENCE_NUMBER); + } + + DateTime minimumMessageTime = taskGroups.get(groupId).minimumMessageTime.orNull(); + DateTime maximumMessageTime = taskGroups.get(groupId).maximumMessageTime.orNull(); + + SeekableStreamIOConfig newIoConfig = createIoConfig( + groupId, + startPartitions, + endPartitions, + group.baseSequenceName, + minimumMessageTime, + maximumMessageTime, + exclusiveStartSequenceNumberPartitions, + ioConfig + ); + + + List> taskList = createIndexTask( + replicas, + group.baseSequenceName, + sortingMapper, + group.currentSequences, + newIoConfig, + taskTuningConfig, + rowIngestionMetersFactory + ); + + for (int i = 0; i < replicas; i++) { + Optional taskQueue = taskMaster.getTaskQueue(); + SeekableStreamIndexTask indexTask = taskList.get(i); + if (taskQueue.isPresent()) { + try { + taskQueue.get().add(indexTask); + } + catch (EntryExistsException e) { + log.error("Tried to add task [%s] but it already exists", indexTask.getId()); + } + } else { + log.error("Failed to get task queue because I'm not the leader!"); + } + } + } + + private void updateCurrentOffsets() throws InterruptedException, ExecutionException, TimeoutException + { + final List> futures = Stream.concat( + taskGroups.values().stream().flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()), + pendingCompletionTaskGroups.values() + .stream() + .flatMap(List::stream) + .flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()) + ).map( + task -> Futures.transform( + taskClient.getCurrentOffsetsAsync(task.getKey(), false), + (Function, Void>) (currentSequences) -> { + + if (currentSequences != null && !currentSequences.isEmpty()) { + task.getValue().currentSequences = currentSequences; + } + + return null; + } + ) + ).collect(Collectors.toList()); + + Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + + private void updateLatestOffsetsFromStream() + { + synchronized (recordSupplierLock) { + Set partitionIds = null; + try { + partitionIds = recordSupplier.getPartitionIds(ioConfig.getId()); + } + catch (Exception e) { + log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getId()); + Throwables.propagate(e); + } + + Set> partitions = partitionIds + .stream() + .map(e -> new StreamPartition<>(ioConfig.getId(), e)) + .collect(Collectors.toSet()); + + recordSupplier.assign(partitions); + recordSupplier.seekToLatest(partitions); + + latestOffsetsFromStream = partitions.stream() + .collect(Collectors.toMap( + StreamPartition::getPartitionId, + x -> recordSupplier.position(x) + )); + } + + } + + @VisibleForTesting + protected Runnable updateCurrentAndLatestOffsets() + { + return () -> { + try { + updateCurrentOffsets(); + updateLatestOffsetsFromStream(); + offsetsLastUpdated = DateTimes.nowUtc(); + } + catch (Exception e) { + log.warn(e, "Exception while getting current/latest offsets"); + } + }; + } + + protected Map getHighestCurrentOffsets() + { + return taskGroups + .values() + .stream() + .flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()) + .flatMap(taskData -> taskData.getValue().currentSequences.entrySet().stream()) + .collect(Collectors.toMap( + Map.Entry::getKey, + Map.Entry::getValue, + (v1, v2) -> v1.compareTo(v2) > 0 ? v1 : v2 + )); + } + + private SupervisorReport> generateReport( + boolean includeOffsets + ) + { + int numPartitions = partitionGroups.values().stream().mapToInt(Map::size).sum(); + + final SeekableStreamSupervisorReportPayload payload = createReportPayload(numPartitions, includeOffsets); + + SupervisorReport> report = new SupervisorReport<>( + dataSource, + DateTimes.nowUtc(), + payload + ); + + List> taskReports = Lists.newArrayList(); + + try { + for (TaskGroup taskGroup : taskGroups.values()) { + for (Map.Entry entry : taskGroup.tasks.entrySet()) { + String taskId = entry.getKey(); + @Nullable + DateTime startTime = entry.getValue().startTime; + Map currentOffsets = entry.getValue().currentSequences; + Long remainingSeconds = null; + if (startTime != null) { + remainingSeconds = Math.max( + 0, ioConfig.getTaskDuration().getMillis() - (System.currentTimeMillis() - startTime.getMillis()) + ) / 1000; + } + + taskReports.add( + new TaskReportData( + taskId, + includeOffsets ? taskGroup.startingSequences : null, + includeOffsets ? currentOffsets : null, + startTime, + remainingSeconds, + TaskReportData.TaskType.ACTIVE, + includeOffsets ? getLagPerPartition(currentOffsets) : null + ) + ); + } + } + + for (List taskGroups : pendingCompletionTaskGroups.values()) { + for (TaskGroup taskGroup : taskGroups) { + for (Map.Entry entry : taskGroup.tasks.entrySet()) { + String taskId = entry.getKey(); + @Nullable + DateTime startTime = entry.getValue().startTime; + Map currentOffsets = entry.getValue().currentSequences; + Long remainingSeconds = null; + if (taskGroup.completionTimeout != null) { + remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - System.currentTimeMillis()) + / 1000; + } + + taskReports.add( + new TaskReportData( + taskId, + includeOffsets ? taskGroup.startingSequences : null, + includeOffsets ? currentOffsets : null, + startTime, + remainingSeconds, + TaskReportData.TaskType.PUBLISHING, + null + ) + ); + } + } + } + + taskReports.forEach(payload::addTask); + } + catch (Exception e) { + log.warn(e, "Failed to generate status report"); + } + + return report; + } + + protected abstract SeekableStreamIOConfig createIoConfig( + int groupId, + Map startPartitions, + Map endPartitions, + String baseSequenceName, + DateTime minimumMessageTime, + DateTime maximumMessageTime, + Set exclusiveStartSequenceNumberPartitions, + SeekableStreamSupervisorIOConfig ioConfig + ); + + protected abstract List> createIndexTask( + int replicas, + String baseSequenceName, + ObjectMapper sortingMapper, + TreeMap> sequenceOffsets, + SeekableStreamIOConfig taskIoConfig, + SeekableStreamTuningConfig taskTuningConfig, + RowIngestionMetersFactory rowIngestionMetersFactory + ) throws JsonProcessingException; + + protected abstract RecordSupplier setupRecordSupplier(); + + protected abstract void scheduleReporting(ScheduledExecutorService reportingExec); + + protected abstract int getTaskGroupIdForPartition(T1 partition); + + protected abstract boolean checkSourceMetaDataMatch(DataSourceMetadata metadata); + + protected abstract boolean checkTaskInstance(Task task); + + protected abstract SeekableStreamSupervisorReportPayload createReportPayload( + int numPartitions, + boolean includeOffsets + ); + + protected abstract Map getLagPerPartition(Map currentOffsets); + + /** + * Notice is used to queue tasks that are internal to the supervisor + */ + private interface Notice + { + void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException; + } + + private static class StatsFromTaskResult + { + private final String groupId; + private final String taskId; + private final Map stats; + + public StatsFromTaskResult( + int groupId, + String taskId, + Map stats + ) + { + this.groupId = String.valueOf(groupId); + this.taskId = taskId; + this.stats = stats; + } + + public String getGroupId() + { + return groupId; + } + + public String getTaskId() + { + return taskId; + } + + public Map getStats() + { + return stats; + } + } + + private class TaskGroup + { + final int groupId; + + final ImmutableMap startingSequences; + final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); + final Optional minimumMessageTime; + final Optional maximumMessageTime; + final Set exclusiveStartSequenceNumberPartitions; //TODO: exclusiveSequence + final TreeMap> currentSequences = new TreeMap<>(); + final String baseSequenceName; + DateTime completionTimeout; + + public TaskGroup( + int groupId, + ImmutableMap startingSequences, + Optional minimumMessageTime, + Optional maximumMessageTime, + Set exclusiveStartSequenceNumberPartitions + ) + { + this.groupId = groupId; + this.startingSequences = startingSequences; + this.minimumMessageTime = minimumMessageTime; + this.maximumMessageTime = maximumMessageTime; + this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions != null + ? exclusiveStartSequenceNumberPartitions + : new HashSet<>(); + this.baseSequenceName = generateSequenceName(startingSequences, minimumMessageTime, maximumMessageTime); + } + + int addNewCheckpoint(Map checkpoint) + { + currentSequences.put(currentSequences.lastKey() + 1, checkpoint); + return currentSequences.lastKey(); + } + + public Set taskIds() + { + return tasks.keySet(); + } + } + + private class TaskData + { + volatile TaskStatus status; + volatile DateTime startTime; + volatile Map currentSequences = new HashMap<>(); + + @Override + public String toString() + { + return "TaskData{" + + "status=" + status + + ", startTime=" + startTime + + ", currentSequences=" + currentSequences + + '}'; + } + } + + private class RunNotice implements Notice + { + @Override + public void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException + { + long nowTime = System.currentTimeMillis(); + if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { + return; + } + lastRunTime = nowTime; + + runInternal(); + } + } + + private class GracefulShutdownNotice extends ShutdownNotice + { + @Override + public void handle() throws InterruptedException, ExecutionException, TimeoutException + { + gracefulShutdownInternal(); + super.handle(); + } + } + + private class ShutdownNotice implements Notice + { + @Override + public void handle() throws InterruptedException, ExecutionException, TimeoutException + { + recordSupplier.close(); + + synchronized (stopLock) { + stopped = true; + stopLock.notifyAll(); + } + } + } + + private class ResetNotice implements Notice + { + final DataSourceMetadata dataSourceMetadata; + + ResetNotice(DataSourceMetadata dataSourceMetadata) + { + this.dataSourceMetadata = dataSourceMetadata; + } + + @Override + public void handle() + { + resetInternal(dataSourceMetadata); + } + } + + protected class CheckpointNotice implements Notice + { + @Nullable + private final Integer nullableTaskGroupId; + @Deprecated + private final String baseSequenceName; + private final SeekableStreamDataSourceMetadata previousCheckpoint; + private final SeekableStreamDataSourceMetadata currentCheckpoint; + + public CheckpointNotice( + @Nullable Integer nullableTaskGroupId, + @Deprecated String baseSequenceName, + SeekableStreamDataSourceMetadata previousCheckpoint, + SeekableStreamDataSourceMetadata currentCheckpoint + ) + { + this.baseSequenceName = baseSequenceName; + this.nullableTaskGroupId = nullableTaskGroupId; + this.previousCheckpoint = previousCheckpoint; + this.currentCheckpoint = currentCheckpoint; + } + + @Override + public void handle() throws ExecutionException, InterruptedException + { + // Find taskGroupId using taskId if it's null. It can be null while rolling update. + final int taskGroupId; + if (nullableTaskGroupId == null) { + // We search taskId in taskGroups and pendingCompletionTaskGroups sequentially. This should be fine because + // 1) a taskGroup can be moved from taskGroups to pendingCompletionTaskGroups in RunNotice + // (see checkTaskDuration()). + // 2) Notices are proceesed by a single thread. So, CheckpointNotice and RunNotice cannot be processed at the + // same time. + final java.util.Optional maybeGroupId = taskGroups + .entrySet() + .stream() + .filter(entry -> { + final TaskGroup taskGroup = entry.getValue(); + return taskGroup.baseSequenceName.equals(baseSequenceName); + }) + .findAny() + .map(Map.Entry::getKey); + taskGroupId = maybeGroupId.orElse( + pendingCompletionTaskGroups + .entrySet() + .stream() + .filter(entry -> { + final List taskGroups = entry.getValue(); + return taskGroups.stream().anyMatch(group -> group.baseSequenceName.equals(baseSequenceName)); + }) + .findAny() + .orElseThrow(() -> new ISE("Cannot find taskGroup for baseSequenceName[%s]", baseSequenceName)) + .getKey() + ); + } else { + taskGroupId = nullableTaskGroupId; + } + + // check for consistency + // if already received request for this sequenceName and dataSourceMetadata combination then return + final TaskGroup taskGroup = taskGroups.get(taskGroupId); + + if (isValidTaskGroup(taskGroupId, taskGroup)) { + final TreeMap> checkpoints = taskGroup.currentSequences; + + // check validity of previousCheckpoint + int index = checkpoints.size(); + for (int sequenceId : checkpoints.descendingKeySet()) { + Map checkpoint = checkpoints.get(sequenceId); + // We have already verified the topic of the current checkpoint is same with that in ioConfig. + // See checkpoint(). + if (checkpoint.equals(previousCheckpoint.getSeekableStreamPartitions() + .getPartitionSequenceMap() + )) { + break; + } + index--; + } + if (index == 0) { + throw new ISE("No such previous checkpoint [%s] found", previousCheckpoint); + } else if (index < checkpoints.size()) { + // if the found checkpoint is not the latest one then already checkpointed by a replica + Preconditions.checkState(index == checkpoints.size() - 1, "checkpoint consistency failure"); + log.info("Already checkpointed with offsets [%s]", checkpoints.lastEntry().getValue()); + return; + } + final Map newCheckpoint = checkpointTaskGroup(taskGroup, false).get(); + taskGroup.addNewCheckpoint(newCheckpoint); + log.info("Handled checkpoint notice, new checkpoint is [%s] for taskGroup [%s]", newCheckpoint, taskGroupId); + } + } + + protected boolean isValidTaskGroup(int taskGroupId, @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); + } + } + + return true; + } + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java index a2c38c7a0cfe..5be635b4dec2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java @@ -25,7 +25,7 @@ import org.joda.time.Duration; import org.joda.time.Period; -abstract public class SeekableStreamSupervisorIOConfig +public abstract class SeekableStreamSupervisorIOConfig { private final String id; private final Integer replicas; @@ -67,6 +67,10 @@ public SeekableStreamSupervisorIOConfig( : Optional.of(earlyMessageRejectionPeriod.toStandardDuration()); } + private static Duration defaultDuration(final Period period, final String theDefault) + { + return (period == null ? new Period(theDefault) : period).toStandardDuration(); + } public String getId() { @@ -103,7 +107,6 @@ public Duration getPeriod() return period; } - public boolean isUseEarliestSequenceNumber() { return useEarliestSequenceNumber; @@ -128,10 +131,5 @@ public Optional getLateMessageRejectionPeriod() } @Override - abstract public String toString(); - - private static Duration defaultDuration(final Period period, final String theDefault) - { - return (period == null ? new Period(theDefault) : period).toStandardDuration(); - } + public abstract String toString(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java index 00db07fbae26..51236ef647b7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -29,15 +29,15 @@ import java.util.List; import java.util.Map; -abstract public class SeekableStreamSupervisorReportPayload +public abstract class SeekableStreamSupervisorReportPayload { private final String dataSource; private final String id; private final int partitions; private final int replicas; private final long durationSeconds; - private final List activeTasks; - private final List publishingTasks; + private final List activeTasks; + private final List publishingTasks; private final Map latestOffsets; private final Map minimumLag; private final Long aggregateLag; @@ -68,11 +68,11 @@ public SeekableStreamSupervisorReportPayload( this.offsetsLastUpdated = offsetsLastUpdated; } - public void addTask(SeekableStreamTaskReportData data) + public void addTask(TaskReportData data) { - if (data.getType().equals(SeekableStreamTaskReportData.TaskType.ACTIVE)) { + if (data.getType().equals(TaskReportData.TaskType.ACTIVE)) { activeTasks.add(data); - } else if (data.getType().equals(SeekableStreamTaskReportData.TaskType.PUBLISHING)) { + } else if (data.getType().equals(TaskReportData.TaskType.PUBLISHING)) { publishingTasks.add(data); } else { throw new IAE("Unknown task type [%s]", data.getType().name()); @@ -109,13 +109,13 @@ public long getDurationSeconds() } @JsonProperty - public List getActiveTasks() + public List getActiveTasks() { return activeTasks; } @JsonProperty - public List getPublishingTasks() + public List getPublishingTasks() { return publishingTasks; } @@ -148,5 +148,5 @@ public DateTime getOffsetsLastUpdated() } @Override - abstract public String toString(); + public abstract String toString(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java index ea89a08c2869..dd4d03a71773 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java @@ -26,13 +26,13 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import org.apache.druid.guice.annotations.Json; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; @@ -40,21 +40,20 @@ import java.util.List; import java.util.Map; -abstract public class SeekableStreamSupervisorSpec implements SupervisorSpec +public abstract class SeekableStreamSupervisorSpec implements SupervisorSpec { - private final DataSchema dataSchema; - private final SeekableStreamSupervisorTuningConfig tuningConfig; - private final SeekableStreamSupervisorIOConfig ioConfig; - private final Map context; - private final ServiceEmitter emitter; - private final DruidMonitorSchedulerConfig monitorSchedulerConfig; - protected final TaskStorage taskStorage; protected final TaskMaster taskMaster; protected final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; protected final SeekableStreamIndexTaskClientFactory indexTaskClientFactory; protected final ObjectMapper mapper; protected final RowIngestionMetersFactory rowIngestionMetersFactory; + private final DataSchema dataSchema; + private final SeekableStreamSupervisorTuningConfig tuningConfig; + private final SeekableStreamSupervisorIOConfig ioConfig; + private final Map context; + private final ServiceEmitter emitter; + private final DruidMonitorSchedulerConfig monitorSchedulerConfig; @JsonCreator public SeekableStreamSupervisorSpec( @@ -128,7 +127,7 @@ public DruidMonitorSchedulerConfig getMonitorSchedulerConfig() } @Override - abstract public Supervisor createSupervisor(); + public abstract Supervisor createSupervisor(); @Override public List getDataSources() @@ -137,7 +136,7 @@ public List getDataSources() } @Override - abstract public String toString(); + public abstract String toString(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java index da4bf33a6350..bb4738719877 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java @@ -27,6 +27,11 @@ public interface SeekableStreamSupervisorTuningConfig { + static Duration defaultDuration(final Period period, final String theDefault) + { + return (period == null ? new Period(theDefault) : period).toStandardDuration(); + } + @JsonProperty Integer getWorkerThreads(); @@ -42,13 +47,11 @@ public interface SeekableStreamSupervisorTuningConfig @JsonProperty Duration getShutdownTimeout(); + @JsonProperty + Duration getOffsetFetchPeriod(); + @Override String toString(); SeekableStreamTuningConfig copyOf(); - - static Duration defaultDuration(final Period period, final String theDefault) - { - return (period == null ? new Period(theDefault) : period).toStandardDuration(); - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamTaskReportData.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java similarity index 77% rename from indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamTaskReportData.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java index 7a84bd447ccd..8e32fa9e73a3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamTaskReportData.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java @@ -26,27 +26,23 @@ import javax.annotation.Nullable; import java.util.Map; -abstract public class SeekableStreamTaskReportData +public class TaskReportData { - public enum TaskType - { - ACTIVE, PUBLISHING, UNKNOWN - } - private final String id; private final Map startingOffsets; private final DateTime startTime; private final Long remainingSeconds; private final TaskType type; private final Map currentOffsets; - - public SeekableStreamTaskReportData( + private final Map lag; + public TaskReportData( String id, @Nullable Map startingOffsets, @Nullable Map currentOffsets, @Nullable DateTime startTime, Long remainingSeconds, - TaskType type + TaskType type, + @Nullable Map lag ) { this.id = id; @@ -55,6 +51,7 @@ public SeekableStreamTaskReportData( this.startTime = startTime; this.remainingSeconds = remainingSeconds; this.type = type; + this.lag = lag; } @JsonProperty @@ -95,7 +92,28 @@ public TaskType getType() return type; } + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Map getLag() + { + return lag; + } + @Override - abstract public String toString(); + public String toString() + { + return "{" + + "id='" + id + '\'' + + (startingOffsets != null ? ", startingOffsets=" + startingOffsets : "") + + (currentOffsets != null ? ", currentOffsets=" + currentOffsets : "") + + ", startTime=" + startTime + + ", remainingSeconds=" + remainingSeconds + + (lag != null ? ", lag=" + lag : "") + + '}'; + } + public enum TaskType + { + ACTIVE, PUBLISHING, UNKNOWN + } } From 22675c5e97479e8bd1f2a4ff9d1e435e783126c7 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 18 Sep 2018 16:34:59 -0700 Subject: [PATCH 07/87] implemented resume/suspend and refactored some package names --- .../indexing/kafka/KafkaTuningConfig.java | 4 +++ .../kafka/supervisor/KafkaSupervisor.java | 3 +- .../KafkaSupervisorReportPayload.java | 7 +++-- .../kafka/supervisor/KafkaSupervisorSpec.java | 24 +++++++++++++++ .../kafka/KafkaIndexTaskClientTest.java | 13 +++++---- .../kafka/supervisor/KafkaSupervisorTest.java | 7 +++-- .../SeekableStreamTuningConfig.java | 13 +++++++++ .../supervisor/SeekableStreamSupervisor.java | 14 ++++++--- ...SeekableStreamSupervisorReportPayload.java | 11 ++++++- .../SeekableStreamSupervisorSpec.java | 29 +++++++++++++++++-- 10 files changed, 106 insertions(+), 19 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java index 4a0321ec8f08..279d7a5ae84b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java @@ -36,6 +36,7 @@ public KafkaTuningConfig( @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, + @JsonProperty("maxTotalRows") Long maxTotalRows, @JsonProperty("intermediatePersistPeriod") @Nullable Period intermediatePersistPeriod, @JsonProperty("basePersistDirectory") @Nullable File basePersistDirectory, @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, @@ -56,6 +57,7 @@ public KafkaTuningConfig( maxRowsInMemory, maxBytesInMemory, maxRowsPerSegment, + maxTotalRows, intermediatePersistPeriod, basePersistDirectory, maxPendingPersists, @@ -79,6 +81,7 @@ public KafkaTuningConfig copyOf() getMaxRowsInMemory(), getMaxBytesInMemory(), getMaxRowsPerSegment(), + getMaxTotalRows(), getIntermediatePersistPeriod(), getBasePersistDirectory(), getMaxPendingPersists(), @@ -102,6 +105,7 @@ public KafkaTuningConfig withBasePersistDirectory(File dir) getMaxRowsInMemory(), getMaxBytesInMemory(), getMaxRowsPerSegment(), + getMaxTotalRows(), getIntermediatePersistPeriod(), dir, getMaxPendingPersists(), diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 1f90e86322a7..533e791798fc 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -215,7 +215,8 @@ protected SeekableStreamSupervisorReportPayload createReportPaylo includeOffsets ? latestOffsetsFromStream : null, includeOffsets ? partitionLag : null, includeOffsets ? partitionLag.values().stream().mapToLong(x -> Math.max(x, 0)).sum() : null, - includeOffsets ? offsetsLastUpdated : null + includeOffsets ? offsetsLastUpdated : null, + spec.isSuspended() ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java index 2b29e6e5be68..fed5f5e021b0 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java @@ -41,7 +41,8 @@ public KafkaSupervisorReportPayload( @Nullable Map latestOffsets, @Nullable Map minimumLag, @Nullable Long aggregateLag, - @Nullable DateTime offsetsLastUpdated + @Nullable DateTime offsetsLastUpdated, + boolean suspended ) { super( @@ -53,7 +54,8 @@ public KafkaSupervisorReportPayload( latestOffsets, minimumLag, aggregateLag, - offsetsLastUpdated + offsetsLastUpdated, + suspended ); } @@ -93,6 +95,7 @@ public String toString() (getMinimumLag() != null ? ", minimumLag=" + getMinimumLag() : "") + (getAggregateLag() != null ? ", aggregateLag=" + getAggregateLag() : "") + (getOffsetsLastUpdated() != null ? ", offsetsLastUpdated=" + getOffsetsLastUpdated() : "") + + ", suspended=" + getSuspended() + '}'; } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index 6c88559ec3fe..2df5c6d6d6b5 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -41,12 +41,14 @@ public class KafkaSupervisorSpec extends SeekableStreamSupervisorSpec { + @JsonCreator public KafkaSupervisorSpec( @JsonProperty("dataSchema") DataSchema dataSchema, @JsonProperty("tuningConfig") KafkaSupervisorTuningConfig tuningConfig, @JsonProperty("ioConfig") KafkaSupervisorIOConfig ioConfig, @JsonProperty("context") Map context, + @JsonProperty("suspended") Boolean suspended, @JacksonInject TaskStorage taskStorage, @JacksonInject TaskMaster taskMaster, @JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, @@ -83,10 +85,12 @@ public KafkaSupervisorSpec( null, null, null, + null, null ), ioConfig, context, + suspended, taskStorage, taskMaster, indexerMetadataStorageCoordinator, @@ -132,6 +136,26 @@ public KafkaSupervisorIOConfig getIoConfig() return (KafkaSupervisorIOConfig) super.getIoConfig(); } + @Override + protected KafkaSupervisorSpec toggleSuspend(boolean suspend) + { + return new KafkaSupervisorSpec( + getDataSchema(), + getTuningConfig(), + getIoConfig(), + getContext(), + suspend, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + (KafkaIndexTaskClientFactory) indexTaskClientFactory, + mapper, + emitter, + monitorSchedulerConfig, + rowIngestionMetersFactory + ); + } + @Override public String toString() { diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java index 4940ec8a9c71..faa51a7e0ca6 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -31,6 +31,7 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IndexTaskClient; import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; @@ -147,7 +148,7 @@ public void testNoTaskLocation() throws IOException Assert.assertEquals(false, client.resume(TEST_ID)); Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID)); Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID)); - Assert.assertEquals(KafkaIndexTask.Status.NOT_STARTED, client.getStatus(TEST_ID)); + Assert.assertEquals(SeekableStreamIndexTask.Status.NOT_STARTED, client.getStatus(TEST_ID)); Assert.assertEquals(null, client.getStartTime(TEST_ID)); Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsets(TEST_ID, true)); Assert.assertEquals(ImmutableMap.of(), client.getEndOffsets(TEST_ID)); @@ -399,7 +400,7 @@ public void testGetStartTime() throws Exception @Test public void testGetStatus() throws Exception { - KafkaIndexTask.Status status = KafkaIndexTask.Status.READING; + SeekableStreamIndexTask.Status status = SeekableStreamIndexTask.Status.READING; Capture captured = Capture.newInstance(); expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); @@ -409,7 +410,7 @@ public void testGetStatus() throws Exception ); replayAll(); - KafkaIndexTask.Status results = client.getStatus(TEST_ID); + SeekableStreamIndexTask.Status results = client.getStatus(TEST_ID); verifyAll(); Request request = captured.getValue(); @@ -728,13 +729,13 @@ public void testGetStatusAsync() throws Exception replayAll(); List expectedUrls = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List> futures = Lists.newArrayList(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "status"))); futures.add(client.getStatusAsync(testId)); } - List responses = Futures.allAsList(futures).get(); + List responses = Futures.allAsList(futures).get(); verifyAll(); List requests = captured.getValues(); @@ -744,7 +745,7 @@ public void testGetStatusAsync() throws Exception for (int i = 0; i < numRequests; i++) { Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); - Assert.assertEquals(KafkaIndexTask.Status.READING, responses.get(i)); + Assert.assertEquals(SeekableStreamIndexTask.Status.READING, responses.get(i)); } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 188a1ea6534c..6a625e6b4387 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -59,6 +59,7 @@ import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.indexing.seekablestream.test.TestBroker; @@ -2402,9 +2403,9 @@ public void testSuspendedRunningTasks() throws Exception null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.PUBLISHING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); - expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java index 78fb9cdd6942..667a8f481940 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java @@ -40,6 +40,8 @@ public abstract class SeekableStreamTuningConfig implements TuningConfig, Append private final int maxRowsInMemory; private final long maxBytesInMemory; private final int maxRowsPerSegment; + @Nullable + private final Long maxTotalRows; private final Period intermediatePersistPeriod; private final File basePersistDirectory; @Deprecated @@ -61,6 +63,7 @@ public SeekableStreamTuningConfig( @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, + @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows, @JsonProperty("intermediatePersistPeriod") @Nullable Period intermediatePersistPeriod, @JsonProperty("basePersistDirectory") @Nullable File basePersistDirectory, @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, @@ -82,6 +85,7 @@ public SeekableStreamTuningConfig( this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; this.maxRowsPerSegment = maxRowsPerSegment == null ? DEFAULT_MAX_ROWS_PER_SEGMENT : maxRowsPerSegment; + this.maxTotalRows = maxTotalRows; // initializing this to 0, it will be lazily initialized to a value // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; @@ -137,12 +141,21 @@ public long getMaxBytesInMemory() return maxBytesInMemory; } + @Override @JsonProperty public int getMaxRowsPerSegment() { return maxRowsPerSegment; } + @JsonProperty + @Override + @Nullable + public Long getMaxTotalRows() + { + return maxTotalRows; + } + @Override @JsonProperty public Period getIntermediatePersistPeriod() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 6c3171d0a548..b92e149088af 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -104,9 +104,7 @@ //TODO: rename offset -> sequence //TODO: prune 'kafka' and 'kinesis' -//TODO: inheritance -//TODO: make classes abstract -//TODO: resolve warnings +//TODO: resolve warnings + inspect code public abstract class SeekableStreamSupervisor, T2 extends Comparable> implements Supervisor { @@ -437,7 +435,15 @@ protected void runInternal() checkTaskDuration(); checkPendingCompletionTasks(); checkCurrentTaskState(); - createNewTasks(); + // if supervisor is not suspended, ensure required tasks are running + // if suspended, ensure tasks have been requested to gracefully stop + if (!spec.isSuspended()) { + log.info("[%s] supervisor is running.", dataSource); + createNewTasks(); + } else { + log.info("[%s] supervisor is suspended.", dataSource); + gracefulShutdownInternal(); + } if (log.isDebugEnabled()) { log.debug(generateReport(true).toString()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java index 51236ef647b7..991509829707 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -42,6 +42,7 @@ public abstract class SeekableStreamSupervisorReportPayload private final Map minimumLag; private final Long aggregateLag; private final DateTime offsetsLastUpdated; + private final boolean suspended; public SeekableStreamSupervisorReportPayload( String dataSource, @@ -52,7 +53,8 @@ public SeekableStreamSupervisorReportPayload( @Nullable Map latestOffsets, @Nullable Map minimumLag, @Nullable Long aggregateLag, - @Nullable DateTime offsetsLastUpdated + @Nullable DateTime offsetsLastUpdated, + boolean suspended ) { this.dataSource = dataSource; @@ -66,6 +68,7 @@ public SeekableStreamSupervisorReportPayload( this.minimumLag = minimumLag; this.aggregateLag = aggregateLag; this.offsetsLastUpdated = offsetsLastUpdated; + this.suspended = suspended; } public void addTask(TaskReportData data) @@ -102,6 +105,12 @@ public int getReplicas() return replicas; } + @JsonProperty + public boolean getSuspended() + { + return suspended; + } + @JsonProperty public long getDurationSeconds() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java index dd4d03a71773..eee44b67e847 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java @@ -52,8 +52,9 @@ public abstract class SeekableStreamSupervisorSpec implements SupervisorSpec private final SeekableStreamSupervisorTuningConfig tuningConfig; private final SeekableStreamSupervisorIOConfig ioConfig; private final Map context; - private final ServiceEmitter emitter; - private final DruidMonitorSchedulerConfig monitorSchedulerConfig; + protected final ServiceEmitter emitter; + protected final DruidMonitorSchedulerConfig monitorSchedulerConfig; + private final boolean suspended; @JsonCreator public SeekableStreamSupervisorSpec( @@ -61,6 +62,7 @@ public SeekableStreamSupervisorSpec( @JsonProperty("tuningConfig") SeekableStreamSupervisorTuningConfig tuningConfig, @JsonProperty("ioConfig") SeekableStreamSupervisorIOConfig ioConfig, @JsonProperty("context") Map context, + @JsonProperty("suspended") Boolean suspended, @JacksonInject TaskStorage taskStorage, @JacksonInject TaskMaster taskMaster, @JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, @@ -84,6 +86,7 @@ public SeekableStreamSupervisorSpec( this.emitter = emitter; this.monitorSchedulerConfig = monitorSchedulerConfig; this.rowIngestionMetersFactory = rowIngestionMetersFactory; + this.suspended = suspended != null ? suspended : false; } @JsonProperty @@ -135,8 +138,30 @@ public List getDataSources() return ImmutableList.of(getDataSchema().getDataSource()); } + @Override + public SeekableStreamSupervisorSpec createSuspendedSpec() + { + return toggleSuspend(true); + } + + @Override + public SeekableStreamSupervisorSpec createRunningSpec() + { + return toggleSuspend(false); + } + + @Override + @JsonProperty("suspended") + public boolean isSuspended() + { + return suspended; + } + @Override public abstract String toString(); + protected abstract SeekableStreamSupervisorSpec toggleSuspend(boolean suspend); + + } From 05279ffe57794e3c471a5b015f9eafa81ec42113 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 18 Sep 2018 16:56:35 -0700 Subject: [PATCH 08/87] moved kinesis indexing service into core druid extensions --- distribution/pom.xml | 22 +- .../kinesis-indexing-service/README.md | 163 ++ .../imply-kinesis-lib/pom.xml | 16 + .../kinesis/KinesisRecordSupplier.java | 519 ++++ .../druid/indexing/kinesis/KinesisRegion.java | 79 + .../ConstructibleAWSCredentialsConfig.java | 39 + .../druid/indexing/kinesis/common/Record.java | 46 + .../kinesis/common/RecordSupplier.java | 28 + .../kinesis/common/StreamPartition.java | 63 + .../kinesis-indexing-service/pom.xml | 23 + .../kinesis/KinesisDataSourceMetadata.java | 153 ++ .../indexing/kinesis/KinesisIOConfig.java | 223 ++ .../indexing/kinesis/KinesisIndexTask.java | 1370 +++++++++++ .../kinesis/KinesisIndexTaskClient.java | 636 +++++ .../KinesisIndexTaskClientFactory.java | 60 + .../kinesis/KinesisIndexingServiceModule.java | 54 + .../indexing/kinesis/KinesisPartitions.java | 99 + .../indexing/kinesis/KinesisTuningConfig.java | 451 ++++ .../kinesis/common/SequenceNumberPlus.java | 28 + .../kinesis/supervisor/KinesisSupervisor.java | 2107 +++++++++++++++++ .../supervisor/KinesisSupervisorIOConfig.java | 249 ++ .../KinesisSupervisorReportPayload.java | 112 + .../supervisor/KinesisSupervisorSpec.java | 230 ++ .../KinesisSupervisorTuningConfig.java | 163 ++ .../kinesis/supervisor/TaskReportData.java | 110 + ...rg.apache.druid.initialization.DruidModule | 1 + .../KinesisDataSourceMetadataTest.java | 130 + .../indexing/kinesis/KinesisIOConfigTest.java | 251 ++ .../kinesis/KinesisIndexTaskClientTest.java | 985 ++++++++ .../kinesis/KinesisIndexTaskTest.java | 1665 +++++++++++++ .../kinesis/KinesisTuningConfigTest.java | 120 + .../KinesisSupervisorIOConfigTest.java | 161 ++ .../supervisor/KinesisSupervisorTest.java | 1863 +++++++++++++++ .../KinesisSupervisorTuningConfigTest.java | 120 + .../indexing/kinesis/test/TestBroker.java | 121 + .../kinesis-indexing-service/pom.xml | 172 ++ pom.xml | 1 + 37 files changed, 12623 insertions(+), 10 deletions(-) create mode 100644 extensions-core/kinesis-indexing-service/README.md create mode 100644 extensions-core/kinesis-indexing-service/imply-kinesis-lib/pom.xml create mode 100644 extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java create mode 100644 extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java create mode 100644 extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java create mode 100644 extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/Record.java create mode 100644 extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/RecordSupplier.java create mode 100644 extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/StreamPartition.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/common/SequenceNumberPlus.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/TaskReportData.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestBroker.java create mode 100644 extensions-core/kinesis-indexing-service/pom.xml diff --git a/distribution/pom.xml b/distribution/pom.xml index c7706cc5da83..5e1106b8983a 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -20,19 +20,19 @@ - 4.0.0 + 4.0.0 - pom + pom - distribution - distribution - distribution + distribution + distribution + distribution - - druid - org.apache.druid - 0.13.0-incubating-SNAPSHOT - + + druid + org.apache.druid + 0.13.0-SNAPSHOT + @@ -139,6 +139,8 @@ -c org.apache.druid.extensions:druid-kafka-indexing-service -c + org.apache.druid.extensions:druid-kinesis-indexing-service + -c org.apache.druid.extensions:druid-lookups-cached-global -c org.apache.druid.extensions:druid-lookups-cached-single diff --git a/extensions-core/kinesis-indexing-service/README.md b/extensions-core/kinesis-indexing-service/README.md new file mode 100644 index 000000000000..32db6263a098 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/README.md @@ -0,0 +1,163 @@ +# Kinesis Indexing Service + +Similar to the [Kafka indexing service](http://druid.io/docs/0.10.0/development/extensions-core/kafka-ingestion.html), +the Kinesis indexing service uses supervisors which run on the overlord and manage the creation and lifetime of Kinesis +indexing tasks. This indexing service can handle non-recent events and provides exactly-once ingestion semantics. + +The Kinesis indexing service is provided as the `druid-kinesis-indexing-service` core extension (see +[Including Extensions](http://druid.io/docs/0.10.0/operations/including-extensions.html)). Please note that this is +currently designated as an *experimental feature* and is subject to the usual +[experimental caveats](http://druid.io/docs/0.10.0/development/experimental.html). + +## Submitting a Supervisor Spec + +The Kinesis indexing service requires that the `druid-kinesis-indexing-service` extension be loaded on both the overlord +and the middle managers. A supervisor for a dataSource is started by submitting a supervisor spec via HTTP POST to +`http://:/druid/indexer/v1/supervisor`, for example: + +``` +curl -X POST -H 'Content-Type: application/json' -d @supervisor-spec.json http://localhost:8090/druid/indexer/v1/supervisor +``` + +A sample supervisor spec is shown below: + +```json +{ + "type": "kinesis", + "dataSchema": { + "dataSource": "metrics-kinesis", + "parser": { + "type": "string", + "parseSpec": { + "format": "json", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [], + "dimensionExclusions": [ + "timestamp", + "value" + ] + } + } + }, + "metricsSpec": [ + { + "name": "count", + "type": "count" + }, + { + "name": "value_sum", + "fieldName": "value", + "type": "doubleSum" + }, + { + "name": "value_min", + "fieldName": "value", + "type": "doubleMin" + }, + { + "name": "value_max", + "fieldName": "value", + "type": "doubleMax" + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "HOUR", + "queryGranularity": "NONE" + } + }, + "tuningConfig": { + "type": "kinesis", + "maxRowsPerSegment": 5000000 + }, + "ioConfig": { + "stream": "metrics", + "endpoint": "kinesis.us-east-1.amazonaws.com", + "taskCount": 1, + "replicas": 1, + "taskDuration": "PT1H", + "recordsPerFetch": 2000, + "fetchDelayMillis": 1000 + } +} +``` + +## Supervisor Configuration + +|Field|Description|Required| +|--------|-----------|---------| +|`type`|The supervisor type, this should always be `kinesis`.|yes| +|`dataSchema`|The schema that will be used by the Kinesis indexing task during ingestion, see [Ingestion Spec](http://druid.io/docs/0.10.0/ingestion/index.html).|yes| +|`tuningConfig`|A KinesisSupervisorTuningConfig to configure the supervisor and indexing tasks, see below.|no| +|`ioConfig`|A KinesisSupervisorIOConfig to configure the supervisor and indexing tasks, see below.|yes| + +### KinesisSupervisorTuningConfig + +The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`type`|String|The indexing task type, this should always be `kinesis`.|yes| +|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 75000)| +|`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number is post-aggregation rows.|no (default == 5000000)| +|`intermediatePersistPeriod`|ISO8601 Period|The period that determines the rate at which intermediate persists occur.|no (default == PT10M)| +|`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)| +|`indexSpec`|Object|Tune how data is indexed, see 'IndexSpec' [here](http://druid.io/docs/0.10.0/development/extensions-core/kafka-ingestion.html#indexspec).|no| +|`buildV9Directly`|Boolean|Whether to build a v9 index directly instead of first building a v8 index and then converting it to v9 format.|no (default == true)| +|`reportParseExceptions`|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|no (default == false)| +|`workerThreads`|Integer|The number of threads that will be used by the supervisor for asynchronous operations.|no (default == min(10, taskCount))| +|`chatThreads`|Integer|The number of threads that will be used for communicating with indexing tasks.|no (default == min(10, taskCount * replicas))| +|`chatRetries`|Integer|The number of times HTTP requests to indexing tasks will be retried before considering tasks unresponsive.|no (default == 8)| +|`httpTimeout`|ISO8601 Period|How long to wait for a HTTP response from an indexing task.|no (default == PT10S)| +|`shutdownTimeout`|ISO8601 Period|How long to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|no (default == PT80S)| +|`recordBufferSize`|Integer|Size of the buffer (number of events) used between the Kinesis fetch threads and the main ingestion thread.|no (default == 10000)| +|`recordBufferOfferTimeout`|Integer|Length of time in milliseconds to wait for space to become available in the buffer before timing out.|no (default == 10000)| +|`recordBufferFullWait`|Integer|Length of time in milliseconds to wait for the buffer to drain before attempting to fetch records from Kinesis again.|no (default == 10000)| +|`fetchSequenceNumberTimeout`|Integer|Length of time in milliseconds to wait for Kinesis to return the earliest or latest sequence number for a partition. Kinesis will not return the latest sequence number if no data is actively being written to that partition. In this case, this fetch call will repeatedly timeout and retry until fresh data is written to the stream.|no (default == 60000)| +|`fetchThreads`|Integer|Size of the pool of threads fetching data from Kinesis. There is no benefit in having more threads than Kinesis partitions.|no (default == max(1, {numProcessors} - 1))| + +### KinesisSupervisorIOConfig + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`stream`|String|The Kinesis stream to read.|yes| +|`endpoint`|String|The AWS Kinesis stream endpoint for a region. You can find a list of endpoints [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region).|no (default == kinesis.us-east-1.amazonaws.com)| +|`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against node failure.|no (default == 1)| +|`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. See 'Capacity Planning' below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKinesisPartitions}`.|no (default == 1)| +|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment. Note that segments are only pushed to deep storage and loadable by historical nodes when the indexing task completes.|no (default == PT1H)| +|`startDelay`|ISO8601 Period|The period to wait before the supervisor starts managing tasks.|no (default == PT5S)| +|`period`|ISO8601 Period|How often the supervisor will execute its management logic. Note that the supervisor will also run in response to certain events (such as tasks succeeding, failing, and reaching their taskDuration) so this value specifies the maximum time between iterations.|no (default == PT30S)| +|`useEarliestSequenceNumber`|Boolean|If a supervisor is managing a dataSource for the first time, it will obtain a set of starting sequence numbers from Kinesis. This flag determines whether it retrieves the earliest or latest offsets in Kinesis. Under normal circumstances, subsequent tasks will start from where the previous segments ended so this flag will only be used on first run.|no (default == false)| +|`completionTimeout`|ISO8601 Period|The length of time to wait before declaring a publishing task as failed and terminating it. If this is set too low, your tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|no (default == PT6H)| +|`lateMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps earlier than this period before the task was created; for example if this is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps earlier than *2016-01-01T11:00Z* will be dropped. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments (e.g. a realtime and a nightly batch ingestion pipeline).|no (default == none)| +|`recordsPerFetch`|Integer|The number of records to request per GetRecords call to Kinesis. See 'Determining Fetch Settings' below.|no (default == 2000)| +|`fetchDelayMillis`|Integer|Time in milliseconds to wait between subsequent GetRecords calls to Kinesis. See 'Determining Fetch Settings' below.|no (default == 1000)| +|`awsAccessKeyId`|String|The AWS access key ID to use for Kinesis API requests. If this is not provided, the service will look for credentials set in the environment variables, system properties, in the default profile configuration file, and from the EC2 instance profile provider (in this order).|no| +|`awsSecretAccessKey`|String|The AWS secret access key to use for Kinesis API requests. Only used if `awsAccessKeyId` is also provided.|no| + +## Determining Fetch Settings + +Kinesis places the following restrictions on calls to fetch records: + +- Each data record can be up to 1 MB in size. +- Each shard can support up to 5 transactions per second for reads. +- Each shard can read up to 2 MB per second. +- The maximum size of data that GetRecords can return is 10 MB. + +Values for `recordsPerFetch` and `fetchDelayMillis` should be chosen to maximize throughput under the above constraints. +The values that you choose will depend on the average size of a record and the number of consumers you have reading from +a given shard (which will be `replicas` unless you have other consumers also reading from this Kinesis stream). + +If the above limits are violated, AWS will throw ProvisionedThroughputExceededException errors on subsequent calls to +read data. When this happens, the Kinesis indexing service will pause by `fetchDelayMillis` and then attempt the call +again. + +## Supervisor API, Capacity Planning, Persistence, and Schema Changes + +The Kinesis indexing service uses the same supervisor API and has the same considerations for capacity planning, +persistence, and schema changes as the Kafka indexing service. For documentation on these topics, see the relevant +sections of the Kafka indexing service [documentation](http://druid.io/docs/0.10.0/development/extensions-core/kafka-ingestion.html#supervisor-api). diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/pom.xml b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/pom.xml new file mode 100644 index 000000000000..f23c1ba65c56 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/pom.xml @@ -0,0 +1,16 @@ + + + 4.0.0 + + druid-imply-kinesis-lib + druid-imply-kinesis-lib + druid-imply-kinesis-lib + + + org.apache.druid.extensions + druid-kinesis-indexing-service + 0.13.0-SNAPSHOT + ../pom.xml + + diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java new file mode 100644 index 000000000000..f21ddb92200b --- /dev/null +++ b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -0,0 +1,519 @@ +package org.apache.druid.indexing.kinesis; + +import com.amazonaws.ClientConfiguration; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider; +import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; +import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxyFactory; +import com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisProxyFactory; +import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; +import com.amazonaws.services.kinesis.model.ResourceNotFoundException; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.common.aws.AWSCredentialsUtils; +import org.apache.druid.indexing.kinesis.aws.ConstructibleAWSCredentialsConfig; +import org.apache.druid.indexing.kinesis.common.Record; +import org.apache.druid.indexing.kinesis.common.RecordSupplier; +import org.apache.druid.indexing.kinesis.common.StreamPartition; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.emitter.EmittingLogger; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +public class KinesisRecordSupplier implements RecordSupplier +{ + private static final EmittingLogger log = new EmittingLogger(KinesisRecordSupplier.class); + private static final long PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS = 3000; + private static final long EXCEPTION_RETRY_DELAY_MS = 10000; + + private class PartitionResource + { + private final StreamPartition streamPartition; + private final IKinesisProxy kinesisProxy; + private final ScheduledExecutorService scheduledExec; + private final Object startLock = new Object(); + + private volatile String shardIterator; + private volatile boolean started; + private volatile boolean stopRequested; + + public PartitionResource( + StreamPartition streamPartition, + IKinesisProxy kinesisProxy, + ScheduledExecutorService scheduledExec + ) + { + this.streamPartition = streamPartition; + this.kinesisProxy = kinesisProxy; + this.scheduledExec = scheduledExec; + } + + public void start() + { + synchronized (startLock) { + if (started) { + return; + } + + log.info( + "Starting scheduled fetch runnable for stream[%s] partition[%s]", + streamPartition.getStreamName(), + streamPartition.getPartitionId() + ); + + stopRequested = false; + started = true; + + rescheduleRunnable(fetchDelayMillis); + } + } + + public void stop() + { + log.info( + "Stopping scheduled fetch runnable for stream[%s] partition[%s]", + streamPartition.getStreamName(), + streamPartition.getPartitionId() + ); + + stopRequested = true; + } + + private Runnable getRecordRunnable() + { + return () -> { + if (stopRequested) { + started = false; + stopRequested = false; + + log.info("Worker for partition[%s] has been stopped", streamPartition.getPartitionId()); + return; + } + + try { + + if (shardIterator == null) { + log.info("shardIterator[%s] has been closed and has no more records", streamPartition.getPartitionId()); + + // add an end-of-shard marker so caller knows this shard is closed + Record endOfShardRecord = new Record( + streamPartition.getStreamName(), streamPartition.getPartitionId(), Record.END_OF_SHARD_MARKER, null + ); + + if (!records.offer(endOfShardRecord, recordBufferOfferTimeout, TimeUnit.MILLISECONDS)) { + log.warn("Record buffer full, retrying in [%,dms]", recordBufferFullWait); + rescheduleRunnable(recordBufferFullWait); + } + + return; + } + + GetRecordsResult recordsResult = kinesisProxy.get(shardIterator, recordsPerFetch); + + // list will come back empty if there are no records + for (com.amazonaws.services.kinesis.model.Record kinesisRecord : recordsResult.getRecords()) { + final List data; + + if (deaggregate) { + data = new ArrayList<>(); + + final List userRecords = UserRecord.deaggregate(Collections.singletonList(kinesisRecord)); + for (UserRecord userRecord : userRecords) { + data.add(toByteArray(userRecord.getData())); + } + } else { + data = Collections.singletonList(toByteArray(kinesisRecord.getData())); + } + + final Record record = new Record( + streamPartition.getStreamName(), + streamPartition.getPartitionId(), + kinesisRecord.getSequenceNumber(), + data + ); + + if (log.isTraceEnabled()) { + log.trace( + "Stream[%s] / partition[%s] / sequenceNum[%s] / bufferRemainingCapacity[%d]: %s", + record.getStreamName(), + record.getPartitionId(), + record.getSequenceNumber(), + records.remainingCapacity(), + record.getData().stream().map(String::new).collect(Collectors.toList()) + ); + } + + // If the buffer was full and we weren't able to add the message, grab a new stream iterator starting + // from this message and back off for a bit to let the buffer drain before retrying. + if (!records.offer(record, recordBufferOfferTimeout, TimeUnit.MILLISECONDS)) { + log.warn("Record buffer full, storing iterator and retrying in [%,dms]", recordBufferFullWait); + + shardIterator = kinesisProxy.getIterator( + record.getPartitionId(), ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), record.getSequenceNumber() + ); + + rescheduleRunnable(recordBufferFullWait); + return; + } + } + + shardIterator = recordsResult.getNextShardIterator(); // will be null if the shard has been closed + + rescheduleRunnable(fetchDelayMillis); + } + catch (ProvisionedThroughputExceededException e) { + long retryMs = Math.max(PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS, fetchDelayMillis); + log.warn("Exceeded provisioned throughput, retrying in [%,dms]", retryMs); + rescheduleRunnable(retryMs); + } + catch (Throwable e) { + log.error(e, "getRecordRunnable exception, retrying in [%,dms]", EXCEPTION_RETRY_DELAY_MS); + rescheduleRunnable(EXCEPTION_RETRY_DELAY_MS); + } + }; + } + + private void rescheduleRunnable(long delayMillis) + { + if (started && !stopRequested) { + scheduledExec.schedule(getRecordRunnable(), delayMillis, TimeUnit.MILLISECONDS); + } else { + log.info("Worker for partition[%s] has been stopped", streamPartition.getPartitionId()); + } + } + } + + private final int recordsPerFetch; + private final int fetchDelayMillis; + private final boolean deaggregate; + private final int recordBufferOfferTimeout; + private final int recordBufferFullWait; + private final int fetchSequenceNumberTimeout; + + private final IKinesisProxyFactory kinesisProxyFactory; + private final ScheduledExecutorService scheduledExec; + + private final Map kinesisProxies = new ConcurrentHashMap<>(); + private final Map partitionResources = new ConcurrentHashMap<>(); + private final BlockingQueue records; + + private volatile boolean checkPartitionsStarted = false; + private volatile boolean closed = false; + + public KinesisRecordSupplier( + String endpoint, + String awsAccessKeyId, + String awsSecretAccessKey, + int recordsPerFetch, + int fetchDelayMillis, + int fetchThreads, + String awsAssumedRoleArn, + String awsExternalId, + boolean deaggregate, + int recordBufferSize, + int recordBufferOfferTimeout, + int recordBufferFullWait, + int fetchSequenceNumberTimeout + ) + { + this.recordsPerFetch = recordsPerFetch; + this.fetchDelayMillis = fetchDelayMillis; + this.deaggregate = deaggregate; + this.recordBufferOfferTimeout = recordBufferOfferTimeout; + this.recordBufferFullWait = recordBufferFullWait; + this.fetchSequenceNumberTimeout = fetchSequenceNumberTimeout; + + AWSCredentialsProvider awsCredentialsProvider = AWSCredentialsUtils.defaultAWSCredentialsProviderChain( + new ConstructibleAWSCredentialsConfig(awsAccessKeyId, awsSecretAccessKey) + ); + + if (awsAssumedRoleArn != null) { + log.info("Assuming role [%s] with externalId [%s]", awsAssumedRoleArn, awsExternalId); + + STSAssumeRoleSessionCredentialsProvider.Builder builder = new STSAssumeRoleSessionCredentialsProvider + .Builder(awsAssumedRoleArn, String.format("druid-kinesis-%s", UUID.randomUUID().toString())) + .withStsClient(new AWSSecurityTokenServiceClient(awsCredentialsProvider)); + + if (awsExternalId != null) { + builder.withExternalId(awsExternalId); + } + + awsCredentialsProvider = builder.build(); + } + + AmazonKinesisClient kinesisClient = new AmazonKinesisClient(awsCredentialsProvider, new ClientConfiguration()); + kinesisClient.setEndpoint(endpoint); + + kinesisProxyFactory = new KinesisProxyFactory(awsCredentialsProvider, kinesisClient); + records = new LinkedBlockingQueue<>(recordBufferSize); + + log.info( + "Creating fetch thread pool of size [%d] (Runtime.availableProcessors=%d)", + fetchThreads, + Runtime.getRuntime().availableProcessors() + ); + + scheduledExec = Executors.newScheduledThreadPool( + fetchThreads, Execs.makeThreadFactory("KinesisRecordSupplier-Worker-%d") + ); + } + + @Override + public void assign(Set collection) + { + checkIfClosed(); + + collection.stream().forEach( + streamPartition -> partitionResources.putIfAbsent( + streamPartition, + new PartitionResource(streamPartition, getKinesisProxy(streamPartition.getStreamName()), scheduledExec) + ) + ); + + for (Iterator> i = partitionResources.entrySet() + .iterator(); i.hasNext(); ) { + Map.Entry entry = i.next(); + if (!collection.contains(entry.getKey())) { + i.remove(); + entry.getValue().stop(); + } + } + } + + @Override + public void seek(StreamPartition partition, String sequenceNumber) + { + checkIfClosed(); + seekInternal(partition, sequenceNumber, ShardIteratorType.AT_SEQUENCE_NUMBER); + } + + @Override + public void seekAfter(StreamPartition partition, String sequenceNumber) + { + checkIfClosed(); + seekInternal(partition, sequenceNumber, ShardIteratorType.AFTER_SEQUENCE_NUMBER); + } + + @Override + public void seekToEarliest(StreamPartition partition) + { + checkIfClosed(); + seekInternal(partition, null, ShardIteratorType.TRIM_HORIZON); + } + + @Override + public void seekToLatest(StreamPartition partition) + { + checkIfClosed(); + seekInternal(partition, null, ShardIteratorType.LATEST); + } + + @Override + public Record poll(long timeout) + { + checkIfClosed(); + if (checkPartitionsStarted) { + partitionResources.values().stream().forEach(PartitionResource::start); + checkPartitionsStarted = false; + } + + try { + while (true) { + Record record = records.poll(timeout, TimeUnit.MILLISECONDS); + if (record == null || partitionResources.containsKey(record.getStreamPartition())) { + return record; + } else if (log.isTraceEnabled()) { + log.trace( + "Skipping stream[%s] / partition[%s] / sequenceNum[%s] because it is not in current assignment", + record.getStreamName(), + record.getPartitionId(), + record.getSequenceNumber() + ); + } + } + } + catch (InterruptedException e) { + log.warn(e, "InterruptedException"); + return null; + } + } + + @Override + public String getLatestSequenceNumber(StreamPartition partition) throws TimeoutException + { + checkIfClosed(); + return getSequenceNumberInternal(partition, ShardIteratorType.LATEST); + } + + @Override + public String getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException + { + checkIfClosed(); + return getSequenceNumberInternal(partition, ShardIteratorType.TRIM_HORIZON); + } + + @Override + public Set getPartitionIds(String streamName) + { + checkIfClosed(); + Set shardList = getKinesisProxy(streamName).getAllShardIds(); + return shardList != null ? shardList : ImmutableSet.of(); + } + + @Override + public void close() + { + if (this.closed) { + return; + } + + assign(ImmutableSet.of()); + + scheduledExec.shutdown(); + + try { + if (!scheduledExec.awaitTermination(EXCEPTION_RETRY_DELAY_MS, TimeUnit.MILLISECONDS)) { + scheduledExec.shutdownNow(); + } + } + catch (InterruptedException e) { + log.info(e, "InterruptedException while shutting down"); + } + + this.closed = true; + } + + private IKinesisProxy getKinesisProxy(String streamName) + { + if (!kinesisProxies.containsKey(streamName)) { + kinesisProxies.put(streamName, kinesisProxyFactory.getProxy(streamName)); + } + + return kinesisProxies.get(streamName); + } + + private void seekInternal(StreamPartition partition, String sequenceNumber, ShardIteratorType iteratorEnum) + { + PartitionResource resource = partitionResources.get(partition); + if (resource == null) { + throw new ISE("Partition [%s] has not been assigned", partition); + } + + log.debug( + "Seeking partition [%s] to [%s]", + partition.getPartitionId(), + sequenceNumber != null ? sequenceNumber : iteratorEnum.toString() + ); + + resource.shardIterator = getKinesisProxy(partition.getStreamName()).getIterator( + partition.getPartitionId(), iteratorEnum.toString(), sequenceNumber + ); + + checkPartitionsStarted = true; + } + + private String getSequenceNumberInternal(StreamPartition partition, ShardIteratorType iteratorEnum) + throws TimeoutException + { + long timeoutMillis = System.currentTimeMillis() + fetchSequenceNumberTimeout; + IKinesisProxy kinesisProxy = getKinesisProxy(partition.getStreamName()); + String shardIterator = null; + + try { + shardIterator = kinesisProxy.getIterator(partition.getPartitionId(), iteratorEnum.toString(), null); + } + catch (ResourceNotFoundException e) { + log.warn("Caught ResourceNotFoundException: %s", e.getMessage()); + } + + while (shardIterator != null && System.currentTimeMillis() < timeoutMillis) { + + if (closed) { + log.info("KinesisRecordSupplier closed while fetching sequenceNumber"); + return null; + } + + GetRecordsResult recordsResult; + try { + recordsResult = kinesisProxy.get(shardIterator, 1); + } + catch (ProvisionedThroughputExceededException e) { + log.warn("Exceeded provisioned throughput, retrying in [%,dms]", PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS); + try { + Thread.sleep(PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS); + continue; + } + catch (InterruptedException e1) { + log.warn(e1, "Thread interrupted!"); + Thread.currentThread().interrupt(); + break; + } + } + + List records = recordsResult.getRecords(); + + if (!records.isEmpty()) { + return records.get(0).getSequenceNumber(); + } + + shardIterator = recordsResult.getNextShardIterator(); + } + + if (shardIterator == null) { + log.info("Partition[%s] returned a null shard iterator", partition.getPartitionId()); + return null; + } + + throw new TimeoutException( + String.format( + "Timeout while retrieving sequence number for partition[%s]", + partition.getPartitionId() + ) + ); + } + + private void checkIfClosed() + { + if (closed) { + throw new ISE("Invalid operation - KinesisRecordSupplier has already been closed"); + } + } + + /** + * Returns an array with the content between the position and limit of "buffer". This may be the buffer's backing + * array itself. Does not modify position or limit of the buffer. + */ + private static byte[] toByteArray(final ByteBuffer buffer) + { + if (buffer.hasArray() + && buffer.arrayOffset() == 0 + && buffer.position() == 0 + && buffer.array().length == buffer.limit()) { + return buffer.array(); + } else { + final byte[] retVal = new byte[buffer.remaining()]; + buffer.duplicate().get(retVal); + return retVal; + } + } +} diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java new file mode 100644 index 000000000000..dec9bce2f006 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java @@ -0,0 +1,79 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; + +import java.util.EnumSet; +import java.util.List; +import java.util.stream.Collectors; + +public enum KinesisRegion +{ + US_EAST_2, + US_EAST_1, + US_WEST_1, + US_WEST_2, + AP_NORTHEAST_1, + AP_NORTHEAST_2, + AP_NORTHEAST_3, + AP_SOUTH_1, + AP_SOUTHEAST_1, + AP_SOUTHEAST_2, + CA_CENTRAL_1, + CN_NORTH_1, + CN_NORTHWEST_1, + EU_CENTRAL_1, + EU_WEST_1, + EU_WEST_2, + EU_WEST_3, + SA_EAST_1, + US_GOV_WEST_1; + + public String getEndpoint() + { + return StringUtils.format("kinesis.%s.amazonaws.com%s", toString(), toString().startsWith("cn-") ? ".cn" : ""); + } + + @JsonCreator + public static KinesisRegion fromString(String value) + { + return EnumSet.allOf(KinesisRegion.class) + .stream() + .filter(x -> x.toString().equals(value)) + .findFirst() + .orElseThrow(() -> new IAE("Region must be one of: %s", getNames())); + } + + @Override + @JsonValue + public String toString() + { + return StringUtils.toLowerCase(name()).replace('_', '-'); + } + + private static List getNames() + { + return EnumSet.allOf(KinesisRegion.class).stream().map(KinesisRegion::toString).collect(Collectors.toList()); + } +} diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java new file mode 100644 index 000000000000..7b1f1b370ee7 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java @@ -0,0 +1,39 @@ +package org.apache.druid.indexing.kinesis.aws; + +import org.apache.druid.common.aws.AWSCredentialsConfig; +import org.apache.druid.metadata.DefaultPasswordProvider; +import org.apache.druid.metadata.PasswordProvider; + +public class ConstructibleAWSCredentialsConfig extends AWSCredentialsConfig +{ + final private String accessKey; + final private String secretKey; + final private String fileSessionCredentials; + + public ConstructibleAWSCredentialsConfig(String accessKey, String secretKey) + { + this(accessKey, secretKey, null); + } + + public ConstructibleAWSCredentialsConfig(String accessKey, String secretKey, String fileSessionCredentials) + { + this.accessKey = accessKey != null ? accessKey : ""; + this.secretKey = secretKey != null ? secretKey : ""; + this.fileSessionCredentials = fileSessionCredentials != null ? fileSessionCredentials : ""; + } + + @Override + public PasswordProvider getAccessKey() + { + return DefaultPasswordProvider.fromString(accessKey); + } + + @Override + public PasswordProvider getSecretKey() + { + return DefaultPasswordProvider.fromString(secretKey); + } + + @Override + public String getFileSessionCredentials() { return fileSessionCredentials; } +} diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/Record.java b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/Record.java new file mode 100644 index 000000000000..729a97ec4546 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/Record.java @@ -0,0 +1,46 @@ +package org.apache.druid.indexing.kinesis.common; + +import java.util.List; + +public class Record +{ + public static final String END_OF_SHARD_MARKER = "EOS"; + + private final String streamName; + private final String partitionId; + private final String sequenceNumber; + private final List data; + + public Record(String streamName, String partitionId, String sequenceNumber, List data) + { + this.streamName = streamName; + this.partitionId = partitionId; + this.sequenceNumber = sequenceNumber; + this.data = data; + } + + public String getStreamName() + { + return streamName; + } + + public String getPartitionId() + { + return partitionId; + } + + public String getSequenceNumber() + { + return sequenceNumber; + } + + public List getData() + { + return data; + } + + public StreamPartition getStreamPartition() + { + return StreamPartition.of(streamName, partitionId); + } +} diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/RecordSupplier.java b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/RecordSupplier.java new file mode 100644 index 000000000000..de112dc26c0d --- /dev/null +++ b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/RecordSupplier.java @@ -0,0 +1,28 @@ +package org.apache.druid.indexing.kinesis.common; + +import java.io.Closeable; +import java.util.Set; +import java.util.concurrent.TimeoutException; + +public interface RecordSupplier extends Closeable +{ + void assign(Set partitions); + + void seek(StreamPartition partition, String sequenceNumber); + + void seekAfter(StreamPartition partition, String sequenceNumber); + + void seekToEarliest(StreamPartition partition); + + void seekToLatest(StreamPartition partition); + + Record poll(long timeout); + + String getLatestSequenceNumber(StreamPartition partition) throws TimeoutException; + + String getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException; + + Set getPartitionIds(String streamName); + + void close(); +} diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/StreamPartition.java b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/StreamPartition.java new file mode 100644 index 000000000000..cc40709697fa --- /dev/null +++ b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/StreamPartition.java @@ -0,0 +1,63 @@ +package org.apache.druid.indexing.kinesis.common; + +public class StreamPartition +{ + private final String streamName; + private final String partitionId; + + public StreamPartition(String streamName, String partitionId) + { + this.streamName = streamName; + this.partitionId = partitionId; + } + + public String getStreamName() + { + return streamName; + } + + public String getPartitionId() + { + return partitionId; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + StreamPartition that = (StreamPartition) o; + + if (streamName != null ? !streamName.equals(that.streamName) : that.streamName != null) { + return false; + } + return !(partitionId != null ? !partitionId.equals(that.partitionId) : that.partitionId != null); + } + + @Override + public int hashCode() + { + int result = streamName != null ? streamName.hashCode() : 0; + result = 31 * result + (partitionId != null ? partitionId.hashCode() : 0); + return result; + } + + @Override + public String toString() + { + return "StreamPartition{" + + "streamName='" + streamName + '\'' + + ", partitionId='" + partitionId + '\'' + + '}'; + } + + public static StreamPartition of(String streamName, String partitionId) + { + return new StreamPartition(streamName, partitionId); + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml new file mode 100644 index 000000000000..b2e62f212212 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml @@ -0,0 +1,23 @@ + + + 4.0.0 + + kinesis-indexing-service + kinesis-indexing-service + kinesis-indexing-service + + + org.apache.druid.extensions + druid-kinesis-indexing-service + 0.13.0-SNAPSHOT + + + + + org.apache.druid.extensions + druid-imply-kinesis-lib + ${project.parent.version} + + + diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java new file mode 100644 index 000000000000..4a58f9d2bf0e --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java @@ -0,0 +1,153 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Maps; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.java.util.common.IAE; + +import java.util.Map; +import java.util.Objects; + +public class KinesisDataSourceMetadata implements DataSourceMetadata +{ + private final KinesisPartitions kinesisPartitions; + + @JsonCreator + public KinesisDataSourceMetadata( + @JsonProperty("partitions") KinesisPartitions kinesisPartitions + ) + { + this.kinesisPartitions = kinesisPartitions; + } + + @JsonProperty("partitions") + public KinesisPartitions getKinesisPartitions() + { + return kinesisPartitions; + } + + @Override + public boolean isValidStart() + { + return true; + } + + @Override + public boolean matches(DataSourceMetadata other) + { + if (getClass() != other.getClass()) { + return false; + } + + return plus(other).equals(other.plus(this)); + } + + @Override + public DataSourceMetadata plus(DataSourceMetadata other) + { + if (!(other instanceof KinesisDataSourceMetadata)) { + throw new IAE( + "Expected instance of %s, got %s", + KinesisDataSourceMetadata.class.getCanonicalName(), + other.getClass().getCanonicalName() + ); + } + + final KinesisDataSourceMetadata that = (KinesisDataSourceMetadata) other; + + if (that.getKinesisPartitions().getStream().equals(kinesisPartitions.getStream())) { + // Same topic, merge sequence numbers. + final Map newMap = Maps.newHashMap(); + + for (Map.Entry entry : kinesisPartitions.getPartitionSequenceNumberMap().entrySet()) { + newMap.put(entry.getKey(), entry.getValue()); + } + + for (Map.Entry entry : that.getKinesisPartitions().getPartitionSequenceNumberMap().entrySet()) { + newMap.put(entry.getKey(), entry.getValue()); + } + + return new KinesisDataSourceMetadata(new KinesisPartitions(kinesisPartitions.getStream(), newMap)); + } else { + // Different topic, prefer "other". + return other; + } + } + + @Override + public DataSourceMetadata minus(DataSourceMetadata other) + { + if (!(other instanceof KinesisDataSourceMetadata)) { + throw new IAE( + "Expected instance of %s, got %s", + KinesisDataSourceMetadata.class.getCanonicalName(), + other.getClass().getCanonicalName() + ); + } + + final KinesisDataSourceMetadata that = (KinesisDataSourceMetadata) other; + + if (that.getKinesisPartitions().getStream().equals(kinesisPartitions.getStream())) { + // Same stream, remove partitions present in "that" from "this" + final Map newMap = Maps.newHashMap(); + + for (Map.Entry entry : kinesisPartitions.getPartitionSequenceNumberMap().entrySet()) { + if (!that.getKinesisPartitions().getPartitionSequenceNumberMap().containsKey(entry.getKey())) { + newMap.put(entry.getKey(), entry.getValue()); + } + } + + return new KinesisDataSourceMetadata(new KinesisPartitions(kinesisPartitions.getStream(), newMap)); + } else { + // Different stream, prefer "this". + return this; + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KinesisDataSourceMetadata that = (KinesisDataSourceMetadata) o; + return Objects.equals(kinesisPartitions, that.kinesisPartitions); + } + + @Override + public int hashCode() + { + return Objects.hash(kinesisPartitions); + } + + @Override + public String toString() + { + return "KinesisDataSourceMetadata{" + + "kinesisPartitions=" + kinesisPartitions + + '}'; + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java new file mode 100644 index 000000000000..fc2d9a127d5f --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java @@ -0,0 +1,223 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import org.apache.druid.segment.indexing.IOConfig; +import org.joda.time.DateTime; + +import java.util.Set; + +public class KinesisIOConfig implements IOConfig +{ + private static final boolean DEFAULT_USE_TRANSACTION = true; + private static final boolean DEFAULT_PAUSE_AFTER_READ = true; + private static final int DEFAULT_RECORDS_PER_FETCH = 4000; + private static final int DEFAULT_FETCH_DELAY_MILLIS = 0; + + private final String baseSequenceName; + private final KinesisPartitions startPartitions; + private final KinesisPartitions endPartitions; + private final boolean useTransaction; + private final boolean pauseAfterRead; + private final Optional minimumMessageTime; + private final Optional maximumMessageTime; + private final String endpoint; + private final Integer recordsPerFetch; + private final Integer fetchDelayMillis; + private final String awsAccessKeyId; + private final String awsSecretAccessKey; + private final Set exclusiveStartSequenceNumberPartitions; + private final String awsAssumedRoleArn; + private final String awsExternalId; + private final boolean deaggregate; + + @JsonCreator + public KinesisIOConfig( + @JsonProperty("baseSequenceName") String baseSequenceName, + @JsonProperty("startPartitions") KinesisPartitions startPartitions, + @JsonProperty("endPartitions") KinesisPartitions endPartitions, + @JsonProperty("useTransaction") Boolean useTransaction, + @JsonProperty("pauseAfterRead") Boolean pauseAfterRead, + @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, + @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, + @JsonProperty("endpoint") String endpoint, + @JsonProperty("recordsPerFetch") Integer recordsPerFetch, + @JsonProperty("fetchDelayMillis") Integer fetchDelayMillis, + @JsonProperty("awsAccessKeyId") String awsAccessKeyId, + @JsonProperty("awsSecretAccessKey") String awsSecretAccessKey, + @JsonProperty("exclusiveStartSequenceNumberPartitions") Set exclusiveStartSequenceNumberPartitions, + @JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn, + @JsonProperty("awsExternalId") String awsExternalId, + @JsonProperty("deaggregate") boolean deaggregate + ) + { + this.baseSequenceName = Preconditions.checkNotNull(baseSequenceName, "baseSequenceName"); + this.startPartitions = Preconditions.checkNotNull(startPartitions, "startPartitions"); + this.endPartitions = Preconditions.checkNotNull(endPartitions, "endPartitions"); + this.useTransaction = useTransaction != null ? useTransaction : DEFAULT_USE_TRANSACTION; + this.pauseAfterRead = pauseAfterRead != null ? pauseAfterRead : DEFAULT_PAUSE_AFTER_READ; + this.minimumMessageTime = Optional.fromNullable(minimumMessageTime); + this.maximumMessageTime = Optional.fromNullable(maximumMessageTime); + this.endpoint = Preconditions.checkNotNull(endpoint, "endpoint"); + this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : DEFAULT_RECORDS_PER_FETCH; + this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : DEFAULT_FETCH_DELAY_MILLIS; + this.awsAccessKeyId = awsAccessKeyId; + this.awsSecretAccessKey = awsSecretAccessKey; + this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions; + this.awsAssumedRoleArn = awsAssumedRoleArn; + this.awsExternalId = awsExternalId; + this.deaggregate = deaggregate; + + Preconditions.checkArgument( + startPartitions.getStream().equals(endPartitions.getStream()), + "start stream and end stream must match" + ); + + Preconditions.checkArgument( + startPartitions.getPartitionSequenceNumberMap() + .keySet() + .equals(endPartitions.getPartitionSequenceNumberMap().keySet()), + "start partition set and end partition set must match" + ); + } + + @JsonProperty + public String getBaseSequenceName() + { + return baseSequenceName; + } + + @JsonProperty + public KinesisPartitions getStartPartitions() + { + return startPartitions; + } + + @JsonProperty + public KinesisPartitions getEndPartitions() + { + return endPartitions; + } + + @JsonProperty + public boolean isUseTransaction() + { + return useTransaction; + } + + @JsonProperty + public boolean isPauseAfterRead() + { + return pauseAfterRead; + } + + @JsonProperty + public Optional getMinimumMessageTime() + { + return minimumMessageTime; + } + + @JsonProperty + public Optional getMaximumMessageTime() + { + return maximumMessageTime; + } + + @JsonProperty + public String getEndpoint() + { + return endpoint; + } + + @JsonProperty + public int getRecordsPerFetch() + { + return recordsPerFetch; + } + + @JsonProperty + public int getFetchDelayMillis() + { + return fetchDelayMillis; + } + + @JsonProperty + public String getAwsAccessKeyId() + { + return awsAccessKeyId; + } + + @JsonProperty + public String getAwsSecretAccessKey() + { + return awsSecretAccessKey; + } + + @JsonProperty + public Set getExclusiveStartSequenceNumberPartitions() + { + return exclusiveStartSequenceNumberPartitions; + } + + @JsonProperty + public String getAwsAssumedRoleArn() + { + return awsAssumedRoleArn; + } + + @JsonProperty + public String getAwsExternalId() + { + return awsExternalId; + } + + @JsonProperty + public boolean isDeaggregate() + { + return deaggregate; + } + + @Override + public String toString() + { + return "KinesisIOConfig{" + + "baseSequenceName='" + baseSequenceName + '\'' + + ", startPartitions=" + startPartitions + + ", endPartitions=" + endPartitions + + ", useTransaction=" + useTransaction + + ", pauseAfterRead=" + pauseAfterRead + + ", minimumMessageTime=" + minimumMessageTime + + ", maximumMessageTime=" + maximumMessageTime + + ", endpoint='" + endpoint + '\'' + + ", recordsPerFetch=" + recordsPerFetch + + ", fetchDelayMillis=" + fetchDelayMillis + + ", awsAccessKeyId='" + awsAccessKeyId + '\'' + + ", awsSecretAccessKey=" + "************************" + + ", exclusiveStartSequenceNumberPartitions=" + exclusiveStartSequenceNumberPartitions + + ", awsAssumedRoleArn='" + awsAssumedRoleArn + '\'' + + ", awsExternalId='" + awsExternalId + '\'' + + ", deaggregate=" + deaggregate + + '}'; + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java new file mode 100644 index 000000000000..d0295290f3df --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -0,0 +1,1370 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.google.common.primitives.Ints; +import org.apache.druid.data.input.Committer; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.discovery.DiscoveryDruidNode; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.LookupNodeService; +import org.apache.druid.indexer.IngestionState; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; +import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; +import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.ResetDataSourceMetadataAction; +import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.stats.RowIngestionMeters; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.AbstractTask; +import org.apache.druid.indexing.common.task.IndexTaskUtils; +import org.apache.druid.indexing.common.task.RealtimeIndexTask; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.kinesis.common.Record; +import org.apache.druid.indexing.kinesis.common.RecordSupplier; +import org.apache.druid.indexing.kinesis.common.StreamPartition; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.collect.Utils; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.query.NoopQueryRunner; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.RealtimeIOConfig; +import org.apache.druid.segment.realtime.FireDepartment; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.appenderator.Appenderator; +import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; +import org.apache.druid.segment.realtime.appenderator.Appenderators; +import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier; +import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata; +import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; +import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; +import org.apache.druid.segment.realtime.firehose.ChatHandler; +import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceAction; +import org.apache.druid.server.security.ResourceType; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.utils.CircularBuffer; +import org.joda.time.DateTime; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; + +public class KinesisIndexTask extends AbstractTask implements ChatHandler +{ + public static final long PAUSE_FOREVER = -1L; + + public enum Status + { + NOT_STARTED, + STARTING, + READING, + PAUSED, + PUBLISHING + } + + private static final EmittingLogger log = new EmittingLogger(KinesisIndexTask.class); + private static final String TYPE = "index_kinesis"; + private static final Random RANDOM = new Random(); + private static final long POLL_TIMEOUT = 100; + private static final long POLL_RETRY_MS = 30000; + private static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15; + private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; + + private final DataSchema dataSchema; + private final InputRowParser parser; + private final KinesisTuningConfig tuningConfig; + private final KinesisIOConfig ioConfig; + private final AuthorizerMapper authorizerMapper; + private final Optional chatHandlerProvider; + + private final Map endOffsets = new ConcurrentHashMap<>(); + private final Map lastOffsets = new ConcurrentHashMap<>(); + + private ObjectMapper mapper; + + private volatile Appenderator appenderator = null; + private volatile FireDepartmentMetrics fireDepartmentMetrics = null; + private volatile DateTime startTime; + private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread) + private volatile Thread runThread = null; + private volatile boolean stopRequested = false; + private volatile boolean publishOnStop = false; + + // The pause lock and associated conditions are to support coordination between the Jetty threads and the main + // ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully + // the ingestion loop has been stopped at the returned offsets and will not ingest any more data until resumed. The + // fields are used as follows (every step requires acquiring [pauseLock]): + // Pausing: + // - In pause(), [pauseRequested] is set to true and then execution waits for [status] to change to PAUSED, with the + // condition checked when [hasPaused] is signalled. + // - In possiblyPause() called from the main loop, if [pauseRequested] is true, [status] is set to PAUSED, + // [hasPaused] is signalled, and execution pauses until [pauseRequested] becomes false, either by being set or by + // the [pauseMillis] timeout elapsing. [pauseRequested] is checked when [shouldResume] is signalled. + // Resuming: + // - In resume(), [pauseRequested] is set to false, [shouldResume] is signalled, and execution waits for [status] to + // change to something other than PAUSED, with the condition checked when [shouldResume] is signalled. + // - In possiblyPause(), when [shouldResume] is signalled, if [pauseRequested] has become false the pause loop ends, + // [status] is changed to STARTING and [shouldResume] is signalled. + + private final Lock pauseLock = new ReentrantLock(); + private final Condition hasPaused = pauseLock.newCondition(); + private final Condition shouldResume = pauseLock.newCondition(); + + // [pollRetryLock] and [isAwaitingRetry] is used when the Kafka consumer returns an OffsetOutOfRangeException and we + // pause polling from Kafka for POLL_RETRY_MS before trying again. This allows us to signal the sleeping thread and + // resume the main run loop in the case of a pause or stop request from a Jetty thread. + private final Lock pollRetryLock = new ReentrantLock(); + private final Condition isAwaitingRetry = pollRetryLock.newCondition(); + + // [statusLock] is used to synchronize the Jetty thread calling stopGracefully() with the main run thread. It prevents + // the main run thread from switching into a publishing state while the stopGracefully() thread thinks it's still in + // a pre-publishing state. This is important because stopGracefully() will try to use the [stopRequested] flag to stop + // the main thread where possible, but this flag is not honored once publishing has begun so in this case we must + // interrupt the thread. The lock ensures that if the run thread is about to transition into publishing state, it + // blocks until after stopGracefully() has set [stopRequested] and then does a final check on [stopRequested] before + // transitioning to publishing state. + private final Object statusLock = new Object(); + + private final RowIngestionMeters rowIngestionMeters; + private CircularBuffer savedParseExceptions; + private IngestionState ingestionState; + + private volatile boolean pauseRequested = false; + private volatile long pauseMillis = 0; + + @JsonCreator + public KinesisIndexTask( + @JsonProperty("id") String id, + @JsonProperty("resource") TaskResource taskResource, + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("tuningConfig") KinesisTuningConfig tuningConfig, + @JsonProperty("ioConfig") KinesisIOConfig ioConfig, + @JsonProperty("context") Map context, + @JacksonInject ChatHandlerProvider chatHandlerProvider, + @JacksonInject AuthorizerMapper authorizerMapper, + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + ) + { + super( + id == null ? makeTaskId(dataSchema.getDataSource(), RANDOM.nextInt()) : id, + String.format("%s_%s", TYPE, dataSchema.getDataSource()), + taskResource, + dataSchema.getDataSource(), + context + ); + + this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); + this.parser = Preconditions.checkNotNull((InputRowParser) dataSchema.getParser(), "parser"); + this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); + this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); + this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); + this.authorizerMapper = authorizerMapper; + this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters(); + this.ingestionState = IngestionState.NOT_STARTED; + if (tuningConfig.getMaxSavedParseExceptions() > 0) { + savedParseExceptions = new CircularBuffer(tuningConfig.getMaxSavedParseExceptions()); + } + this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionSequenceNumberMap()); + } + + private static String makeTaskId(String dataSource, int randomBits) + { + final StringBuilder suffix = new StringBuilder(8); + for (int i = 0; i < Ints.BYTES * 2; ++i) { + suffix.append((char) ('a' + ((randomBits >>> (i * 4)) & 0x0F))); + } + return Joiner.on("_").join(TYPE, dataSource, suffix); + } + + @Override + public int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY); + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + return true; + } + + @JsonProperty + public DataSchema getDataSchema() + { + return dataSchema; + } + + @JsonProperty + public KinesisTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @JsonProperty("ioConfig") + public KinesisIOConfig getIOConfig() + { + return ioConfig; + } + + @Override + public TaskStatus run(final TaskToolbox toolbox) throws Exception + { + log.info("Starting up!"); + startTime = DateTime.now(); + mapper = toolbox.getObjectMapper(); + status = Status.STARTING; + + if (chatHandlerProvider.isPresent()) { + log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); + chatHandlerProvider.get().register(getId(), this, false); + } else { + log.warn("No chat handler detected"); + } + + runThread = Thread.currentThread(); + + // Set up FireDepartmentMetrics + final FireDepartment fireDepartmentForMetrics = new FireDepartment( + dataSchema, + new RealtimeIOConfig(null, null, null), + null + ); + fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); + toolbox.getMonitorScheduler() + .addMonitor(TaskRealtimeMetricsMonitorBuilder.build(this, fireDepartmentForMetrics, rowIngestionMeters)); + + final LookupNodeService lookupNodeService = getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER) == null ? + toolbox.getLookupNodeService() : + new LookupNodeService((String) getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER)); + + final DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( + toolbox.getDruidNode(), + DruidNodeDiscoveryProvider.NODE_TYPE_PEON, + ImmutableMap.of( + toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), + lookupNodeService.getName(), lookupNodeService + ) + ); + + try ( + final Appenderator appenderator0 = newAppenderator(fireDepartmentMetrics, toolbox); + final StreamAppenderatorDriver driver = newDriver(appenderator0, toolbox, fireDepartmentMetrics); + final RecordSupplier recordSupplier = getRecordSupplier() + ) { + toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); + toolbox.getDataSegmentServerAnnouncer().announce(); + + appenderator = appenderator0; + + final String topic = ioConfig.getStartPartitions().getStream(); + + // Start up, set up initial offsets. + final Object restoredMetadata = driver.startJob(); + if (restoredMetadata == null) { + lastOffsets.putAll(ioConfig.getStartPartitions().getPartitionSequenceNumberMap()); + } else { + final Map restoredMetadataMap = (Map) restoredMetadata; + final KinesisPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( + restoredMetadataMap.get(METADATA_NEXT_PARTITIONS), + KinesisPartitions.class + ); + lastOffsets.putAll(restoredNextPartitions.getPartitionSequenceNumberMap()); + + // Sanity checks. + if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) { + throw new ISE( + "WTF?! Restored stream[%s] but expected stream[%s]", + restoredNextPartitions.getStream(), + ioConfig.getStartPartitions().getStream() + ); + } + + if (!lastOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet())) { + throw new ISE( + "WTF?! Restored partitions[%s] but expected partitions[%s]", + lastOffsets.keySet(), + ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet() + ); + } + } + + // Filter out partitions with END_OF_SHARD markers since these partitions have already been fully read. This + // should have been done by the supervisor already so this is defensive. + int numPreFilterPartitions = lastOffsets.size(); + if (lastOffsets.entrySet().removeIf(x -> Record.END_OF_SHARD_MARKER.equals(x.getValue()))) { + log.info( + "Removed [%d] partitions from assignment which have already been closed", + numPreFilterPartitions - lastOffsets.size() + ); + } + + // Set up sequenceNames. + final Map sequenceNames = Maps.newHashMap(); + for (String partitionNum : lastOffsets.keySet()) { + sequenceNames.put(partitionNum, String.format("%s_%s", ioConfig.getBaseSequenceName(), partitionNum)); + } + + // Set up committer. + final Supplier committerSupplier = new Supplier() + { + @Override + public Committer get() + { + final Map snapshot = ImmutableMap.copyOf(lastOffsets); + + return new Committer() + { + @Override + public Object getMetadata() + { + return ImmutableMap.of( + METADATA_NEXT_PARTITIONS, new KinesisPartitions( + ioConfig.getStartPartitions().getStream(), + snapshot + ) + ); + } + + @Override + public void run() + { + // Do nothing. + } + }; + } + }; + + Set assignment = assignPartitions(recordSupplier, topic); + seekToStartingRecords(recordSupplier, topic, assignment, toolbox); + + Map contiguousOffsetCheck = new HashMap<>(lastOffsets); + boolean verifiedAllStartingOffsets = false; + + // Main loop. + // Could eventually support leader/follower mode (for keeping replicas more in sync) + boolean stillReading = ioConfig.isPauseAfterRead() || !assignment.isEmpty(); + status = Status.READING; + try { + while (stillReading) { + if (possiblyPause(assignment)) { + assignment = assignPartitions(recordSupplier, topic); + + if (assignment.isEmpty()) { + log.info("All partitions have been fully read"); + publishOnStop = true; + stopRequested = true; + } + } + + if (stopRequested) { + break; + } + + Record record = recordSupplier.poll(POLL_TIMEOUT); + + if (record == null) { + continue; + } + + // for the first message we receive, check that we were given a message with a sequenceNumber that matches our + // expected starting sequenceNumber + if (!verifiedAllStartingOffsets && contiguousOffsetCheck.containsKey(record.getPartitionId())) { + if (!contiguousOffsetCheck.get(record.getPartitionId()).equals(record.getSequenceNumber())) { + throw new ISE( + "Starting sequenceNumber [%s] does not match expected [%s] for partition [%s]", + record.getSequenceNumber(), + contiguousOffsetCheck.get(record.getPartitionId()), + record.getPartitionId() + ); + } + + log.info( + "Verified starting sequenceNumber [%s] for partition [%s]", + record.getSequenceNumber(), record.getPartitionId() + ); + + contiguousOffsetCheck.remove(record.getPartitionId()); + if (contiguousOffsetCheck.isEmpty()) { + verifiedAllStartingOffsets = true; + log.info("Verified starting offsets for all partitions"); + } + + if (ioConfig.getExclusiveStartSequenceNumberPartitions() != null + && ioConfig.getExclusiveStartSequenceNumberPartitions().contains(record.getPartitionId())) { + log.info("Skipping starting sequenceNumber for partition [%s] marked exclusive", record.getPartitionId()); + + continue; + } + } + + if (log.isTraceEnabled()) { + log.trace( + "Got topic[%s] partition[%s] offset[%s].", + record.getStreamName(), + record.getPartitionId(), + record.getSequenceNumber() + ); + } + + if (Record.END_OF_SHARD_MARKER.equals(record.getSequenceNumber())) { + lastOffsets.put(record.getPartitionId(), record.getSequenceNumber()); + + } else if (KinesisPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffsets.get(record.getPartitionId())) + || record.getSequenceNumber().compareTo(endOffsets.get(record.getPartitionId())) <= 0) { + + try { + final List valueBytess = record.getData(); + + final List rows; + if (valueBytess == null || valueBytess.isEmpty()) { + rows = Utils.nullableListOf((InputRow) null); + } else { + rows = new ArrayList<>(); + for (byte[] valueBytes : valueBytess) { + rows.addAll(parser.parseBatch(ByteBuffer.wrap(valueBytes))); + } + } + + boolean isPersistRequired = false; + final Map> segmentsToMoveOut = new HashMap<>(); + + for (final InputRow row : rows) { + if (row != null && withinMinMaxRecordTime(row)) { + final String sequenceName = sequenceNames.get(record.getPartitionId()); + final AppenderatorDriverAddResult addResult = driver.add( + row, + sequenceName, + committerSupplier, + false, + false + ); + + if (addResult.isOk()) { + // If the number of rows in the segment exceeds the threshold after adding a row, + // move the segment out from the active segments of BaseAppenderatorDriver to make a new segment. + if (addResult.getNumRowsInSegment() > tuningConfig.getMaxRowsPerSegment()) { + segmentsToMoveOut.computeIfAbsent(sequenceName, k -> new HashSet<>()) + .add(addResult.getSegmentIdentifier()); + } + isPersistRequired |= addResult.isPersistRequired(); + } else { + // Failure to allocate segment puts determinism at risk, bail out to be safe. + // May want configurable behavior here at some point. + // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks. + throw new ISE("Could not allocate segment for row with timestamp[%s]", row.getTimestamp()); + } + + if (addResult.getParseException() != null) { + handleParseException(addResult.getParseException(), record); + } else { + rowIngestionMeters.incrementProcessed(); + } + } else { + rowIngestionMeters.incrementThrownAway(); + } + } + + if (isPersistRequired) { + driver.persist(committerSupplier.get()); + } + segmentsToMoveOut.entrySet().forEach(sequenceSegments -> driver.moveSegmentOut( + sequenceSegments.getKey(), + sequenceSegments.getValue().stream().collect(Collectors.toList()) + )); + } + catch (ParseException e) { + handleParseException(e, record); + } + + lastOffsets.put(record.getPartitionId(), record.getSequenceNumber()); + } + + if ((lastOffsets.get(record.getPartitionId()).equals(endOffsets.get(record.getPartitionId())) + || Record.END_OF_SHARD_MARKER.equals(lastOffsets.get(record.getPartitionId()))) + && assignment.remove(record.getPartitionId())) { + + log.info("Finished reading stream[%s], partition[%s].", record.getStreamName(), record.getPartitionId()); + assignPartitions(recordSupplier, topic, assignment); + stillReading = ioConfig.isPauseAfterRead() || !assignment.isEmpty(); + } + } + } + finally { + driver.persist(committerSupplier.get()); // persist pending data + } + + synchronized (statusLock) { + if (stopRequested && !publishOnStop) { + throw new InterruptedException("Stopping without publishing"); + } + + status = Status.PUBLISHING; + } + + final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { + final KinesisPartitions finalPartitions = toolbox.getObjectMapper().convertValue( + ((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS), + KinesisPartitions.class + ); + + // Sanity check, we should only be publishing things that match our desired end state. + if (!endOffsets.equals(finalPartitions.getPartitionSequenceNumberMap())) { + throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata); + } + + final SegmentTransactionalInsertAction action; + + if (ioConfig.isUseTransaction()) { + action = new SegmentTransactionalInsertAction( + segments, + new KinesisDataSourceMetadata(ioConfig.getStartPartitions()), + new KinesisDataSourceMetadata(finalPartitions) + ); + } else { + action = new SegmentTransactionalInsertAction(segments, null, null); + } + + log.info("Publishing with isTransaction[%s].", ioConfig.isUseTransaction()); + + return toolbox.getTaskActionClient().submit(action); + }; + + // Supervised Kinesis tasks are killed by KinesisSupervisor if they are stuck during publishing segments or waiting + // for hand off. See KinesisSupervisorIOConfig.completionTimeout. + final SegmentsAndMetadata published = driver.publish( + publisher, + committerSupplier.get(), + sequenceNames.values() + ).get(); + + final SegmentsAndMetadata handedOff; + if (tuningConfig.getHandoffConditionTimeout() == 0) { + handedOff = driver.registerHandoff(published) + .get(); + } else { + handedOff = driver.registerHandoff(published) + .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); + } + + if (handedOff == null) { + throw new ISE("Transaction failure publishing segments, aborting"); + } else { + log.info( + "Published segments[%s] with metadata[%s].", + Joiner.on(", ").join( + Iterables.transform( + handedOff.getSegments(), + new Function() + { + @Override + public String apply(DataSegment input) + { + return input.getIdentifier(); + } + } + ) + ), + handedOff.getCommitMetadata() + ); + } + } + catch (InterruptedException | RejectedExecutionException e) { + // handle the InterruptedException that gets wrapped in a RejectedExecutionException + if (e instanceof RejectedExecutionException + && (e.getCause() == null || !(e.getCause() instanceof InterruptedException))) { + throw e; + } + + // if we were interrupted because we were asked to stop, handle the exception and return success, else rethrow + if (!stopRequested) { + Thread.currentThread().interrupt(); + throw e; + } + + log.info("The task was asked to stop before completing"); + } + finally { + if (chatHandlerProvider.isPresent()) { + chatHandlerProvider.get().unregister(getId()); + } + } + + toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); + toolbox.getDataSegmentServerAnnouncer().unannounce(); + + return success(); + } + + @Override + public boolean canRestore() + { + return true; + } + + /** + * Authorizes action to be performed on this task's datasource + * + * @return authorization result + */ + private Access authorizationCheck(final HttpServletRequest req, Action action) + { + ResourceAction resourceAction = new ResourceAction( + new Resource(dataSchema.getDataSource(), ResourceType.DATASOURCE), + action + ); + + Access access = AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); + if (!access.isAllowed()) { + throw new ForbiddenException(access.toString()); + } + + return access; + } + + private void handleParseException(ParseException pe, Record record) + { + if (pe.isFromPartiallyValidRow()) { + rowIngestionMeters.incrementProcessedWithError(); + } else { + rowIngestionMeters.incrementUnparseable(); + } + + if (tuningConfig.isLogParseExceptions()) { + log.error( + pe, + "Encountered parse exception on row from partition[%d] sequenceNumber[%s]", + record.getPartitionId(), + record.getSequenceNumber() + ); + } + + if (savedParseExceptions != null) { + savedParseExceptions.add(pe); + } + + if (rowIngestionMeters.getUnparseable() + rowIngestionMeters.getProcessedWithError() + > tuningConfig.getMaxParseExceptions()) { + log.error("Max parse exceptions exceeded, terminating task..."); + throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); + } + } + + @Override + public void stopGracefully() + { + log.info("Stopping gracefully (status: [%s])", status); + stopRequested = true; + + synchronized (statusLock) { + if (status == Status.PUBLISHING) { + runThread.interrupt(); + return; + } + } + + try { + if (pauseLock.tryLock(LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { + try { + if (pauseRequested) { + pauseRequested = false; + shouldResume.signalAll(); + } + } + finally { + pauseLock.unlock(); + } + } else { + log.warn("While stopping: failed to acquire pauseLock before timeout, interrupting run thread"); + runThread.interrupt(); + return; + } + + if (pollRetryLock.tryLock(LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { + try { + isAwaitingRetry.signalAll(); + } + finally { + pollRetryLock.unlock(); + } + } else { + log.warn("While stopping: failed to acquire pollRetryLock before timeout, interrupting run thread"); + runThread.interrupt(); + } + } + catch (Exception e) { + Throwables.propagate(e); + } + } + + @Override + public QueryRunner getQueryRunner(Query query) + { + if (appenderator == null) { + // Not yet initialized, no data yet, just return a noop runner. + return new NoopQueryRunner<>(); + } + + return new QueryRunner() + { + @Override + public Sequence run(final QueryPlus query, final Map responseContext) + { + return query.run(appenderator, responseContext); + } + }; + } + + @POST + @Path("/stop") + public Response stop(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.WRITE); + stopGracefully(); + return Response.status(Response.Status.OK).build(); + } + + @GET + @Path("/status") + @Produces(MediaType.APPLICATION_JSON) + public Status getStatusHTTP(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return status; + } + + public Status getStatus() + { + return status; + } + + @GET + @Path("/offsets/current") + @Produces(MediaType.APPLICATION_JSON) + public Map getCurrentOffsets(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return getCurrentOffsets(); + } + + public Map getCurrentOffsets() + { + return lastOffsets; + } + + @GET + @Path("/offsets/end") + @Produces(MediaType.APPLICATION_JSON) + public Map getEndOffsetsHTTP(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return getEndOffsets(); + } + + public Map getEndOffsets() + { + return endOffsets; + } + + @POST + @Path("/offsets/end") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response setEndOffsetsHTTP( + Map offsets, + @QueryParam("resume") @DefaultValue("false") final boolean resume, + @Context final HttpServletRequest req + ) throws InterruptedException + { + authorizationCheck(req, Action.WRITE); + return setEndOffsets(offsets, resume); + } + + public Response setEndOffsets(Map offsets, final boolean resume) throws InterruptedException + { + if (offsets == null) { + return Response.status(Response.Status.BAD_REQUEST) + .entity("Request body must contain a map of { partition:endOffset }") + .build(); + } else if (!endOffsets.keySet().containsAll(offsets.keySet())) { + return Response.status(Response.Status.BAD_REQUEST) + .entity( + String.format( + "Request contains partitions not being handled by this task, my partitions: %s", + endOffsets.keySet() + ) + ) + .build(); + } + + pauseLock.lockInterruptibly(); + try { + if (!isPaused()) { + return Response.status(Response.Status.BAD_REQUEST) + .entity("Task must be paused before changing the end offsets") + .build(); + } + + for (Map.Entry entry : offsets.entrySet()) { + if (entry.getValue().compareTo(lastOffsets.get(entry.getKey())) < 0) { + return Response.status(Response.Status.BAD_REQUEST) + .entity( + String.format( + "End offset must be >= current offset for partition [%s] (current: %s)", + entry.getKey(), + lastOffsets.get(entry.getKey()) + ) + ) + .build(); + } + } + + endOffsets.putAll(offsets); + log.info("endOffsets changed to %s", endOffsets); + } + finally { + pauseLock.unlock(); + } + + if (resume) { + resume(); + } + + return Response.ok(endOffsets).build(); + } + + /** + * Signals the ingestion loop to pause. + * + * @param timeout how long to pause for before resuming in milliseconds, <= 0 means indefinitely + * + * @return one of the following Responses: 400 Bad Request if the task has started publishing; 202 Accepted if the + * method has timed out and returned before the task has paused; 200 OK with a map of the current partition offsets + * in the response body if the task successfully paused + */ + @POST + @Path("/pause") + @Produces(MediaType.APPLICATION_JSON) + public Response pauseHTTP( + @QueryParam("timeout") @DefaultValue("0") final long timeout, @Context final HttpServletRequest req + ) throws InterruptedException + { + authorizationCheck(req, Action.WRITE); + return pause(timeout); + } + + public Response pause(final long timeout) throws InterruptedException + { + if (!(status == Status.PAUSED || status == Status.READING)) { + return Response.status(Response.Status.BAD_REQUEST) + .entity(String.format("Can't pause, task is not in a pausable state (state: [%s])", status)) + .build(); + } + + pauseLock.lockInterruptibly(); + try { + pauseMillis = timeout <= 0 ? PAUSE_FOREVER : timeout; + pauseRequested = true; + + pollRetryLock.lockInterruptibly(); + try { + isAwaitingRetry.signalAll(); + } + finally { + pollRetryLock.unlock(); + } + + if (isPaused()) { + shouldResume.signalAll(); // kick the monitor so it re-awaits with the new pauseMillis + } + + long nanos = TimeUnit.SECONDS.toNanos(2); + while (!isPaused()) { + if (nanos <= 0L) { + return Response.status(Response.Status.ACCEPTED) + .entity("Request accepted but task has not yet paused") + .build(); + } + nanos = hasPaused.awaitNanos(nanos); + } + } + finally { + pauseLock.unlock(); + } + + try { + return Response.ok().entity(mapper.writeValueAsString(getCurrentOffsets())).build(); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } + } + + @POST + @Path("/resume") + public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException + { + authorizationCheck(req, Action.WRITE); + resume(); + return Response.status(Response.Status.OK).build(); + } + + public void resume() throws InterruptedException + { + pauseLock.lockInterruptibly(); + try { + pauseRequested = false; + shouldResume.signalAll(); + + long nanos = TimeUnit.SECONDS.toNanos(5); + while (isPaused()) { + if (nanos <= 0L) { + throw new RuntimeException("Resume command was not accepted within 5 seconds"); + } + nanos = shouldResume.awaitNanos(nanos); + } + } + finally { + pauseLock.unlock(); + } + } + + @GET + @Path("/time/start") + @Produces(MediaType.APPLICATION_JSON) + public DateTime getStartTime(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return startTime; + } + + @GET + @Path("/rowStats") + @Produces(MediaType.APPLICATION_JSON) + public Response getRowStats( + @Context final HttpServletRequest req + ) + { + authorizationCheck(req, Action.READ); + Map returnMap = Maps.newHashMap(); + Map totalsMap = Maps.newHashMap(); + Map averagesMap = Maps.newHashMap(); + + totalsMap.put( + RowIngestionMeters.BUILD_SEGMENTS, + rowIngestionMeters.getTotals() + ); + averagesMap.put( + RowIngestionMeters.BUILD_SEGMENTS, + rowIngestionMeters.getMovingAverages() + ); + + returnMap.put("movingAverages", averagesMap); + returnMap.put("totals", totalsMap); + return Response.ok(returnMap).build(); + } + + @GET + @Path("/unparseableEvents") + @Produces(MediaType.APPLICATION_JSON) + public Response getUnparseableEvents( + @Context final HttpServletRequest req + ) + { + authorizationCheck(req, Action.READ); + List events = IndexTaskUtils.getMessagesFromSavedParseExceptions(savedParseExceptions); + return Response.ok(events).build(); + } + + @VisibleForTesting + FireDepartmentMetrics getFireDepartmentMetrics() + { + return fireDepartmentMetrics; + } + + private boolean isPaused() + { + return status == Status.PAUSED; + } + + private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) + { + final int maxRowsInMemoryPerPartition = (tuningConfig.getMaxRowsInMemory() / + ioConfig.getStartPartitions().getPartitionSequenceNumberMap().size()); + return Appenderators.createRealtime( + dataSchema, + tuningConfig.withBasePersistDirectory(new File(toolbox.getPersistDir(), "persist")) + .withMaxRowsInMemory(maxRowsInMemoryPerPartition), + metrics, + toolbox.getSegmentPusher(), + toolbox.getObjectMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMergerV9(), + toolbox.getQueryRunnerFactoryConglomerate(), + toolbox.getSegmentAnnouncer(), + toolbox.getEmitter(), + toolbox.getQueryExecutorService(), + toolbox.getCache(), + toolbox.getCacheConfig(), + toolbox.getCachePopulatorStats() + ); + } + + private StreamAppenderatorDriver newDriver( + final Appenderator appenderator, + final TaskToolbox toolbox, + final FireDepartmentMetrics metrics + ) + { + return new StreamAppenderatorDriver( + appenderator, + new ActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + dataSchema, + (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck + ) + ), + toolbox.getSegmentHandoffNotifierFactory(), + new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), + toolbox.getDataSegmentKiller(), + toolbox.getObjectMapper(), + metrics + ); + } + + private RecordSupplier getRecordSupplier() + { + int fetchThreads = tuningConfig.getFetchThreads() != null + ? tuningConfig.getFetchThreads() + : Math.max(1, ioConfig.getStartPartitions().getPartitionSequenceNumberMap().size()); + + return new KinesisRecordSupplier( + ioConfig.getEndpoint(), + ioConfig.getAwsAccessKeyId(), + ioConfig.getAwsSecretAccessKey(), + ioConfig.getRecordsPerFetch(), + ioConfig.getFetchDelayMillis(), + fetchThreads, + ioConfig.getAwsAssumedRoleArn(), + ioConfig.getAwsExternalId(), + ioConfig.isDeaggregate(), + tuningConfig.getRecordBufferSize(), + tuningConfig.getRecordBufferOfferTimeout(), + tuningConfig.getRecordBufferFullWait(), + tuningConfig.getFetchSequenceNumberTimeout() + ); + } + + private static void assignPartitions( + final RecordSupplier recordSupplier, + final String topic, + final Set partitions + ) + { + recordSupplier.assign(partitions.stream().map(x -> StreamPartition.of(topic, x)).collect(Collectors.toSet())); + } + + private Set assignPartitions(RecordSupplier recordSupplier, String topic) + { + // Initialize consumer assignment. + final Set assignment = Sets.newHashSet(); + for (Map.Entry entry : lastOffsets.entrySet()) { + final String endOffset = endOffsets.get(entry.getKey()); + if (KinesisPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) || entry.getValue().compareTo(endOffset) < 0) { + assignment.add(entry.getKey()); + } else if (entry.getValue().equals(endOffset)) { + log.info("Finished reading partition[%s].", entry.getKey()); + } else { + throw new ISE( + "WTF?! Cannot start from offset[%s] > endOffset[%s]", + entry.getValue(), + endOffset + ); + } + } + + assignPartitions(recordSupplier, topic, assignment); + + return assignment; + } + + private void seekToStartingRecords( + RecordSupplier recordSupplier, + String topic, + Set assignment, + TaskToolbox toolbox + ) + { + // Seek to starting offsets. + for (final String partition : assignment) { + final String offset = lastOffsets.get(partition); + final StreamPartition streamPartition = StreamPartition.of(topic, partition); + + if (!tuningConfig.isSkipSequenceNumberAvailabilityCheck()) { + try { + String earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); + if (earliestSequenceNumber == null || earliestSequenceNumber.compareTo(offset) > 0) { + if (tuningConfig.isResetOffsetAutomatically()) { + log.info("Attempting to reset offsets automatically for all partitions"); + try { + sendResetRequestAndWait( + assignment.stream() + .collect(Collectors.toMap((x) -> new StreamPartition(topic, x), lastOffsets::get)), + toolbox + ); + } + catch (IOException e) { + throw new ISE(e, "Exception while attempting to automatically reset offsets"); + } + } else { + throw new ISE( + "Starting sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) and resetOffsetAutomatically is not enabled", + offset, + partition, + earliestSequenceNumber + ); + } + } + } + catch (TimeoutException e) { + throw new ISE(e, "Timeout while fetching earliest sequence number for partition [%s]", partition); + } + } + + log.info("Seeking partition[%s] to sequenceNumber[%s].", partition, offset); + + // We will seek to and start reading from the last offset that we read on the previous run so that we can confirm + // that the sequenceNumbers match, but we will discard the event instead of indexing it so we don't read it twice. + recordSupplier.seek(streamPartition, offset); + } + } + + /** + * Checks if the pauseRequested flag was set and if so blocks: + * a) if pauseMillis == PAUSE_FOREVER, until pauseRequested is cleared + * b) if pauseMillis != PAUSE_FOREVER, until pauseMillis elapses -or- pauseRequested is cleared + *

+ * If pauseMillis is changed while paused, the new pause timeout will be applied. This allows adjustment of the + * pause timeout (making a timed pause into an indefinite pause and vice versa is valid) without having to resume + * and ensures that the loop continues to stay paused without ingesting any new events. You will need to signal + * shouldResume after adjusting pauseMillis for the new value to take effect. + *

+ * Sets paused = true and signals paused so callers can be notified when the pause command has been accepted. + *

+ * Additionally, pauses if all partitions assignments have been read and pauseAfterRead flag is set. + * + * @return true if a pause request was handled, false otherwise + */ + private boolean possiblyPause(Set assignment) throws InterruptedException + { + pauseLock.lockInterruptibly(); + try { + if (ioConfig.isPauseAfterRead() && assignment.isEmpty()) { + pauseMillis = PAUSE_FOREVER; + pauseRequested = true; + } + + if (pauseRequested) { + status = Status.PAUSED; + long nanos = 0; + hasPaused.signalAll(); + + while (pauseRequested) { + if (pauseMillis == PAUSE_FOREVER) { + log.info("Pausing ingestion until resumed"); + shouldResume.await(); + } else { + if (pauseMillis > 0) { + log.info("Pausing ingestion for [%,d] ms", pauseMillis); + nanos = TimeUnit.MILLISECONDS.toNanos(pauseMillis); + pauseMillis = 0; + } + if (nanos <= 0L) { + pauseRequested = false; // timeout elapsed + } + nanos = shouldResume.awaitNanos(nanos); + } + } + + status = Status.READING; + shouldResume.signalAll(); + log.info("Ingestion loop resumed"); + return true; + } + } + finally { + pauseLock.unlock(); + } + + return false; + } + + private void sendResetRequestAndWait(Map outOfRangePartitions, TaskToolbox taskToolbox) + throws IOException + { + Map partitionOffsetMap = outOfRangePartitions + .entrySet().stream().collect(Collectors.toMap(x -> x.getKey().getPartitionId(), Map.Entry::getValue)); + + boolean result = taskToolbox + .getTaskActionClient() + .submit( + new ResetDataSourceMetadataAction( + getDataSource(), + new KinesisDataSourceMetadata( + new KinesisPartitions(ioConfig.getStartPartitions().getStream(), partitionOffsetMap) + ) + ) + ); + + if (result) { + log.makeAlert("Resetting Kinesis offsets for datasource [%s]", getDataSource()) + .addData("partitions", partitionOffsetMap.keySet()) + .emit(); + // wait for being killed by supervisor + try { + pause(-1); + } + catch (InterruptedException e) { + throw new RuntimeException("Got interrupted while pausing task"); + } + } else { + log.makeAlert("Failed to send reset request for partitions [%s]", partitionOffsetMap.keySet()).emit(); + } + } + + private boolean withinMinMaxRecordTime(final InputRow row) + { + final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent() + && ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp()); + + final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent() + && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp()); + + if (!Intervals.ETERNITY.contains(row.getTimestamp())) { + final String errorMsg = StringUtils.format( + "Encountered row with timestamp that cannot be represented as a long: [%s]", + row + ); + throw new ParseException(errorMsg); + } + + if (log.isDebugEnabled()) { + if (beforeMinimumMessageTime) { + log.debug( + "CurrentTimeStamp[%s] is before MinimumMessageTime[%s]", + row.getTimestamp(), + ioConfig.getMinimumMessageTime().get() + ); + } else if (afterMaximumMessageTime) { + log.debug( + "CurrentTimeStamp[%s] is after MaximumMessageTime[%s]", + row.getTimestamp(), + ioConfig.getMaximumMessageTime().get() + ); + } + } + + return !beforeMinimumMessageTime && !afterMaximumMessageTime; + } + + private Map getTaskCompletionRowStats() + { + Map metrics = Maps.newHashMap(); + metrics.put( + RowIngestionMeters.BUILD_SEGMENTS, + rowIngestionMeters.getTotals() + ); + return metrics; + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java new file mode 100644 index 000000000000..a7de074de1a8 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java @@ -0,0 +1,636 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Charsets; +import com.google.common.base.Optional; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.RetryPolicy; +import org.apache.druid.indexing.common.RetryPolicyConfig; +import org.apache.druid.indexing.common.RetryPolicyFactory; +import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.FullResponseHandler; +import org.apache.druid.java.util.http.client.response.FullResponseHolder; +import org.apache.druid.segment.realtime.firehose.ChatHandlerResource; +import org.jboss.netty.channel.ChannelException; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Period; + +import javax.ws.rs.core.MediaType; +import java.io.IOException; +import java.net.Socket; +import java.net.URI; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.Callable; + +public class KinesisIndexTaskClient +{ + public static class NoTaskLocationException extends RuntimeException + { + public NoTaskLocationException(String message) + { + super(message); + } + } + + public static class TaskNotRunnableException extends RuntimeException + { + public TaskNotRunnableException(String message) + { + super(message); + } + } + + public static final int MAX_RETRY_WAIT_SECONDS = 10; + + private static final int MIN_RETRY_WAIT_SECONDS = 2; + private static final EmittingLogger log = new EmittingLogger(KinesisIndexTaskClient.class); + private static final String BASE_PATH = "/druid/worker/v1/chat"; + private static final int TASK_MISMATCH_RETRY_DELAY_SECONDS = 5; + + private final HttpClient httpClient; + private final ObjectMapper jsonMapper; + private final TaskInfoProvider taskInfoProvider; + private final Duration httpTimeout; + private final RetryPolicyFactory retryPolicyFactory; + private final ListeningExecutorService executorService; + private final long numRetries; + + public KinesisIndexTaskClient( + HttpClient httpClient, + ObjectMapper jsonMapper, + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ) + { + this.httpClient = httpClient; + this.jsonMapper = jsonMapper; + this.taskInfoProvider = taskInfoProvider; + this.httpTimeout = httpTimeout; + this.numRetries = numRetries; + this.retryPolicyFactory = createRetryPolicyFactory(); + + this.executorService = MoreExecutors.listeningDecorator( + Execs.multiThreaded( + numThreads, + String.format( + "KinesisIndexTaskClient-%s-%%d", + dataSource + ) + ) + ); + } + + public void close() + { + executorService.shutdownNow(); + } + + public boolean stop(final String id, final boolean publish) + { + log.debug("Stop task[%s] publish[%s]", id, publish); + + try { + final FullResponseHolder response = submitRequest( + id, HttpMethod.POST, "stop", publish ? "publish=true" : null, true + ); + return response.getStatus().getCode() / 100 == 2; + } + catch (NoTaskLocationException e) { + return false; + } + catch (TaskNotRunnableException e) { + log.info("Task [%s] couldn't be stopped because it is no longer running", id); + return true; + } + catch (Exception e) { + log.warn(e, "Exception while stopping task [%s]", id); + return false; + } + } + + public boolean resume(final String id) + { + log.debug("Resume task[%s]", id); + + try { + final FullResponseHolder response = submitRequest(id, HttpMethod.POST, "resume", null, true); + return response.getStatus().getCode() / 100 == 2; + } + catch (NoTaskLocationException e) { + return false; + } + } + + public Map pause(final String id) + { + return pause(id, 0); + } + + public Map pause(final String id, final long timeout) + { + log.debug("Pause task[%s] timeout[%d]", id, timeout); + + try { + final FullResponseHolder response = submitRequest( + id, + HttpMethod.POST, + "pause", + timeout > 0 ? String.format("timeout=%d", timeout) : null, + true + ); + + if (response.getStatus().equals(HttpResponseStatus.OK)) { + log.info("Task [%s] paused successfully", id); + return jsonMapper.readValue(response.getContent(), new TypeReference>() {}); + } + + final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy(); + while (true) { + if (getStatus(id) == KinesisIndexTask.Status.PAUSED) { + return getCurrentOffsets(id, true); + } + + final Duration delay = retryPolicy.getAndIncrementRetryDelay(); + if (delay == null) { + log.error("Task [%s] failed to pause, aborting", id); + throw new ISE("Task [%s] failed to pause, aborting", id); + } else { + final long sleepTime = delay.getMillis(); + log.info( + "Still waiting for task [%s] to pause; will try again in [%s]", + id, + new Duration(sleepTime).toString() + ); + Thread.sleep(sleepTime); + } + } + } + catch (NoTaskLocationException e) { + log.error("Exception [%s] while pausing Task [%s]", e.getMessage(), id); + return ImmutableMap.of(); + } + catch (IOException | InterruptedException e) { + log.error("Exception [%s] while pausing Task [%s]", e.getMessage(), id); + throw Throwables.propagate(e); + } + } + + public KinesisIndexTask.Status getStatus(final String id) + { + log.debug("GetStatus task[%s]", id); + + try { + final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "status", null, true); + return jsonMapper.readValue(response.getContent(), KinesisIndexTask.Status.class); + } + catch (NoTaskLocationException e) { + return KinesisIndexTask.Status.NOT_STARTED; + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + public DateTime getStartTime(final String id) + { + log.debug("GetStartTime task[%s]", id); + + try { + final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "time/start", null, true); + return response.getContent() == null || response.getContent().isEmpty() + ? null + : jsonMapper.readValue(response.getContent(), DateTime.class); + } + catch (NoTaskLocationException e) { + return null; + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + public Map getCurrentOffsets(final String id, final boolean retry) + { + log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry); + + try { + final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "offsets/current", null, retry); + return jsonMapper.readValue(response.getContent(), new TypeReference>() {}); + } + catch (NoTaskLocationException e) { + return ImmutableMap.of(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + public Map getEndOffsets(final String id) + { + log.debug("GetEndOffsets task[%s]", id); + + try { + final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "offsets/end", null, true); + return jsonMapper.readValue(response.getContent(), new TypeReference>() {}); + } + catch (NoTaskLocationException e) { + return ImmutableMap.of(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + public boolean setEndOffsets(final String id, final Map endOffsets) + { + return setEndOffsets(id, endOffsets, false); + } + + public boolean setEndOffsets(final String id, final Map endOffsets, final boolean resume) + { + log.debug("SetEndOffsets task[%s] endOffsets[%s] resume[%s]", id, endOffsets, resume); + + try { + final FullResponseHolder response = submitRequest( + id, + HttpMethod.POST, + "offsets/end", + resume ? "resume=true" : null, + jsonMapper.writeValueAsBytes(endOffsets), + true + ); + return response.getStatus().getCode() / 100 == 2; + } + catch (NoTaskLocationException e) { + return false; + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + public ListenableFuture stopAsync(final String id, final boolean publish) + { + return executorService.submit( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return stop(id, publish); + } + } + ); + } + + public ListenableFuture resumeAsync(final String id) + { + return executorService.submit( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return resume(id); + } + } + ); + } + + public ListenableFuture> pauseAsync(final String id) + { + return pauseAsync(id, 0); + } + + public ListenableFuture> pauseAsync(final String id, final long timeout) + { + return executorService.submit( + new Callable>() + { + @Override + public Map call() throws Exception + { + return pause(id, timeout); + } + } + ); + } + + public ListenableFuture getStatusAsync(final String id) + { + return executorService.submit( + new Callable() + { + @Override + public KinesisIndexTask.Status call() throws Exception + { + return getStatus(id); + } + } + ); + } + + public ListenableFuture getStartTimeAsync(final String id) + { + return executorService.submit( + new Callable() + { + @Override + public DateTime call() throws Exception + { + return getStartTime(id); + } + } + ); + } + + public ListenableFuture> getCurrentOffsetsAsync(final String id, final boolean retry) + { + return executorService.submit( + new Callable>() + { + @Override + public Map call() throws Exception + { + return getCurrentOffsets(id, retry); + } + } + ); + } + + public ListenableFuture> getEndOffsetsAsync(final String id) + { + return executorService.submit( + new Callable>() + { + @Override + public Map call() throws Exception + { + return getEndOffsets(id); + } + } + ); + } + + public ListenableFuture setEndOffsetsAsync(final String id, final Map endOffsets) + { + return setEndOffsetsAsync(id, endOffsets, false); + } + + public ListenableFuture setEndOffsetsAsync( + final String id, final Map endOffsets, final boolean resume + ) + { + return executorService.submit( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return setEndOffsets(id, endOffsets, resume); + } + } + ); + } + + @VisibleForTesting + RetryPolicyFactory createRetryPolicyFactory() + { + // Retries [numRetries] times before giving up; this should be set long enough to handle any temporary + // unresponsiveness such as network issues, if a task is still in the process of starting up, or if the task is in + // the middle of persisting to disk and doesn't respond immediately. + return new RetryPolicyFactory( + new RetryPolicyConfig() + .setMinWait(Period.seconds(MIN_RETRY_WAIT_SECONDS)) + .setMaxWait(Period.seconds(MAX_RETRY_WAIT_SECONDS)) + .setMaxRetryCount(numRetries) + ); + } + + @VisibleForTesting + void checkConnection(String host, int port) throws IOException + { + new Socket(host, port).close(); + } + + public Map getMovingAverages(final String id) + { + log.debug("GetMovingAverages task[%s]", id); + + try { + final FullResponseHolder response = submitRequest( + id, + HttpMethod.GET, + "rowStats", + null, + true + ); + return response.getContent() == null || response.getContent().isEmpty() + ? Collections.emptyMap() + : jsonMapper.readValue(response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT); + } + catch (NoTaskLocationException e) { + return Collections.emptyMap(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + public ListenableFuture> getMovingAveragesAsync(final String id) + { + return executorService.submit( + new Callable>() + { + @Override + public Map call() + { + return getMovingAverages(id); + } + } + ); + } + + private FullResponseHolder submitRequest(String id, HttpMethod method, String pathSuffix, String query, boolean retry) + { + return submitRequest(id, method, pathSuffix, query, new byte[0], retry); + } + + private FullResponseHolder submitRequest( + String id, + HttpMethod method, + String pathSuffix, + String query, + byte[] content, + boolean retry + ) + { + final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy(); + while (true) { + FullResponseHolder response = null; + Request request = null; + TaskLocation location = TaskLocation.unknown(); + String path = String.format("%s/%s/%s", BASE_PATH, id, pathSuffix); + + Optional status = taskInfoProvider.getTaskStatus(id); + if (!status.isPresent() || !status.get().isRunnable()) { + throw new TaskNotRunnableException(String.format("Aborting request because task [%s] is not runnable", id)); + } + + String host = location.getHost(); + String scheme = ""; + int port = -1; + + try { + location = taskInfoProvider.getTaskLocation(id); + if (location.equals(TaskLocation.unknown())) { + throw new NoTaskLocationException(String.format("No TaskLocation available for task [%s]", id)); + } + + host = location.getHost(); + scheme = location.getTlsPort() >= 0 ? "https" : "http"; + port = location.getTlsPort() >= 0 ? location.getTlsPort() : location.getPort(); + + // Netty throws some annoying exceptions if a connection can't be opened, which happens relatively frequently + // for tasks that happen to still be starting up, so test the connection first to keep the logs clean. + checkConnection(host, port); + + try { + URI serviceUri = new URI(scheme, null, host, port, path, query, null); + request = new Request(method, serviceUri.toURL()); + + // used to validate that we are talking to the correct worker + request.addHeader(ChatHandlerResource.TASK_ID_HEADER, id); + + if (content.length > 0) { + request.setContent(MediaType.APPLICATION_JSON, content); + } + + log.debug("HTTP %s: %s", method.getName(), serviceUri.toString()); + response = httpClient.go(request, new FullResponseHandler(Charsets.UTF_8), httpTimeout).get(); + } + catch (Exception e) { + Throwables.propagateIfInstanceOf(e.getCause(), IOException.class); + Throwables.propagateIfInstanceOf(e.getCause(), ChannelException.class); + throw Throwables.propagate(e); + } + + int responseCode = response.getStatus().getCode(); + if (responseCode / 100 == 2) { + return response; + } else if (responseCode == 400) { // don't bother retrying if it's a bad request + throw new IAE("Received 400 Bad Request with body: %s", response.getContent()); + } else { + throw new IOException(String.format("Received status [%d]", responseCode)); + } + } + catch (IOException | ChannelException e) { + + // Since workers are free to move tasks around to different ports, there is a chance that a task may have been + // moved but our view of its location has not been updated yet from ZK. To detect this case, we send a header + // identifying our expected recipient in the request; if this doesn't correspond to the worker we messaged, the + // worker will return an HTTP 404 with its ID in the response header. If we get a mismatching task ID, then + // we will wait for a short period then retry the request indefinitely, expecting the task's location to + // eventually be updated. + + final Duration delay; + if (response != null && response.getStatus().equals(HttpResponseStatus.NOT_FOUND)) { + String headerId = response.getResponse().headers().get(ChatHandlerResource.TASK_ID_HEADER); + if (headerId != null && !headerId.equals(id)) { + log.warn( + "Expected worker to have taskId [%s] but has taskId [%s], will retry in [%d]s", + id, headerId, TASK_MISMATCH_RETRY_DELAY_SECONDS + ); + delay = Duration.standardSeconds(TASK_MISMATCH_RETRY_DELAY_SECONDS); + } else { + delay = retryPolicy.getAndIncrementRetryDelay(); + } + } else { + delay = retryPolicy.getAndIncrementRetryDelay(); + } + + String urlForLog = (request != null + ? request.getUrl().toString() + : StringUtils.format("%s://%s:%d%s", scheme, host, port, path)); + + if (!retry) { + // if retry=false, we probably aren't too concerned if the operation doesn't succeed (i.e. the request was + // for informational purposes only) so don't log a scary stack trace + log.info("submitRequest failed for [%s], with message [%s]", urlForLog, e.getMessage()); + Throwables.propagate(e); + } else if (delay == null) { + log.warn(e, "Retries exhausted for [%s], last exception:", urlForLog); + Throwables.propagate(e); + } else { + try { + final long sleepTime = delay.getMillis(); + log.debug( + "Bad response HTTP [%s] from [%s]; will try again in [%s] (body/exception: [%s])", + (response != null ? response.getStatus().getCode() : "no response"), + urlForLog, + new Duration(sleepTime).toString(), + (response != null ? response.getContent() : e.getMessage()) + ); + Thread.sleep(sleepTime); + } + catch (InterruptedException e2) { + Throwables.propagate(e2); + } + } + } + catch (NoTaskLocationException e) { + log.info( + "No TaskLocation available for task [%s], this task may not have been assigned to a worker yet or " + + "may have already completed", id + ); + throw e; + } + catch (Exception e) { + log.warn(e, "Exception while sending request"); + throw e; + } + } + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java new file mode 100644 index 000000000000..b16f22161680 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java @@ -0,0 +1,60 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.java.util.http.client.HttpClient; +import org.joda.time.Duration; + +public class KinesisIndexTaskClientFactory +{ + private HttpClient httpClient; + private ObjectMapper mapper; + + @Inject + public KinesisIndexTaskClientFactory(@EscalatedGlobal HttpClient httpClient, @Json ObjectMapper mapper) + { + this.httpClient = httpClient; + this.mapper = mapper; + } + + public KinesisIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ) + { + return new KinesisIndexTaskClient( + httpClient, + mapper, + taskInfoProvider, + dataSource, + numThreads, + httpTimeout, + numRetries + ); + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java new file mode 100644 index 000000000000..c96736af45b0 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java @@ -0,0 +1,54 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec; +import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig; +import org.apache.druid.initialization.DruidModule; + +import java.util.List; + +public class KinesisIndexingServiceModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + return ImmutableList.of( + new SimpleModule(getClass().getSimpleName()) + .registerSubtypes( + new NamedType(KinesisIndexTask.class, "index_kinesis"), + new NamedType(KinesisDataSourceMetadata.class, "kinesis"), + new NamedType(KinesisIOConfig.class, "kinesis"), + new NamedType(KinesisSupervisorTuningConfig.class, "kinesis"), + new NamedType(KinesisSupervisorSpec.class, "kinesis") + ) + ); + } + + @Override + public void configure(Binder binder) + { + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java new file mode 100644 index 000000000000..9586f42e82f3 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java @@ -0,0 +1,99 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; + +import java.util.Map; +import java.util.Objects; + +public class KinesisPartitions +{ + public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER"; + + private final String stream; + private final Map partitionSequenceNumberMap; + + @JsonCreator + public KinesisPartitions( + @JsonProperty("stream") final String stream, + @JsonProperty("partitionSequenceNumberMap") final Map partitionSequenceNumberMap + ) + { + this.stream = stream; + this.partitionSequenceNumberMap = ImmutableMap.copyOf(partitionSequenceNumberMap); + + // Validate partitionSequenceNumberMap + for (Map.Entry entry : partitionSequenceNumberMap.entrySet()) { + Preconditions.checkArgument( + entry.getValue() != null, + String.format( + "partition[%s] sequenceNumber[%s] invalid", + entry.getKey(), + entry.getValue() + ) + ); + } + } + + @JsonProperty + public String getStream() + { + return stream; + } + + @JsonProperty + public Map getPartitionSequenceNumberMap() + { + return partitionSequenceNumberMap; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KinesisPartitions that = (KinesisPartitions) o; + return Objects.equals(stream, that.stream) && + Objects.equals(partitionSequenceNumberMap, that.partitionSequenceNumberMap); + } + + @Override + public int hashCode() + { + return Objects.hash(stream, partitionSequenceNumberMap); + } + + @Override + public String toString() + { + return "KinesisPartitions{" + + "stream='" + stream + '\'' + + ", partitionSequenceNumberMap=" + partitionSequenceNumberMap + + '}'; + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java new file mode 100644 index 000000000000..48a185f7bf7f --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java @@ -0,0 +1,451 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.indexing.RealtimeTuningConfig; +import org.apache.druid.segment.indexing.TuningConfig; +import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.Objects; + +public class KinesisTuningConfig implements TuningConfig, AppenderatorConfig +{ + private static final int DEFAULT_MAX_ROWS_PER_SEGMENT = 5_000_000; + private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false; + private static final boolean DEFAULT_SKIP_SEQUENCE_NUMBER_AVAILABILITY_CHECK = false; + private static final int DEFAULT_RECORD_BUFFER_SIZE = 10000; + private static final int DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT = 5000; + private static final int DEFAULT_RECORD_BUFFER_FULL_WAIT = 5000; + private static final int DEFAULT_FETCH_SEQUENCE_NUMBER_TIMEOUT = 60000; + + private final int maxRowsInMemory; + private final long maxBytesInMemory; + private final int maxRowsPerSegment; + private final Period intermediatePersistPeriod; + private final File basePersistDirectory; + private final int maxPendingPersists; + private final IndexSpec indexSpec; + private final boolean buildV9Directly; + private final boolean reportParseExceptions; + private final long handoffConditionTimeout; + private final boolean resetOffsetAutomatically; + private final boolean skipSequenceNumberAvailabilityCheck; + private final int recordBufferSize; + private final int recordBufferOfferTimeout; + private final int recordBufferFullWait; + private final int fetchSequenceNumberTimeout; + private final Integer fetchThreads; + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + + private final boolean logParseExceptions; + private final int maxParseExceptions; + private final int maxSavedParseExceptions; + + @JsonCreator + public KinesisTuningConfig( + @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, + @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, + @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, + @JsonProperty("basePersistDirectory") File basePersistDirectory, + @JsonProperty("maxPendingPersists") Integer maxPendingPersists, + @JsonProperty("indexSpec") IndexSpec indexSpec, + @JsonProperty("buildV9Directly") Boolean buildV9Directly, + @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, + @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, + @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, + @JsonProperty("skipSequenceNumberAvailabilityCheck") Boolean skipSequenceNumberAvailabilityCheck, + @JsonProperty("recordBufferSize") Integer recordBufferSize, + @JsonProperty("recordBufferOfferTimeout") Integer recordBufferOfferTimeout, + @JsonProperty("recordBufferFullWait") Integer recordBufferFullWait, + @JsonProperty("fetchSequenceNumberTimeout") Integer fetchSequenceNumberTimeout, + @JsonProperty("fetchThreads") Integer fetchThreads, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions + ) + { + // Cannot be a static because default basePersistDirectory is unique per-instance + final RealtimeTuningConfig defaults = RealtimeTuningConfig.makeDefaultTuningConfig(basePersistDirectory); + + this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; + this.maxBytesInMemory = maxBytesInMemory == null ? defaults.getMaxBytesInMemory() : maxBytesInMemory; + this.maxRowsPerSegment = maxRowsPerSegment == null ? DEFAULT_MAX_ROWS_PER_SEGMENT : maxRowsPerSegment; + this.intermediatePersistPeriod = intermediatePersistPeriod == null + ? defaults.getIntermediatePersistPeriod() + : intermediatePersistPeriod; + this.basePersistDirectory = defaults.getBasePersistDirectory(); + this.maxPendingPersists = maxPendingPersists == null ? defaults.getMaxPendingPersists() : maxPendingPersists; + this.indexSpec = indexSpec == null ? defaults.getIndexSpec() : indexSpec; + this.buildV9Directly = buildV9Directly == null ? defaults.getBuildV9Directly() : buildV9Directly; + this.reportParseExceptions = reportParseExceptions == null + ? defaults.isReportParseExceptions() + : reportParseExceptions; + this.handoffConditionTimeout = handoffConditionTimeout == null + ? defaults.getHandoffConditionTimeout() + : handoffConditionTimeout; + this.resetOffsetAutomatically = resetOffsetAutomatically == null + ? DEFAULT_RESET_OFFSET_AUTOMATICALLY + : resetOffsetAutomatically; + this.skipSequenceNumberAvailabilityCheck = skipSequenceNumberAvailabilityCheck == null + ? DEFAULT_SKIP_SEQUENCE_NUMBER_AVAILABILITY_CHECK + : skipSequenceNumberAvailabilityCheck; + this.recordBufferSize = recordBufferSize == null ? DEFAULT_RECORD_BUFFER_SIZE : recordBufferSize; + this.recordBufferOfferTimeout = recordBufferOfferTimeout == null + ? DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT + : recordBufferOfferTimeout; + this.recordBufferFullWait = recordBufferFullWait == null ? DEFAULT_RECORD_BUFFER_FULL_WAIT : recordBufferFullWait; + this.fetchSequenceNumberTimeout = fetchSequenceNumberTimeout + == null ? DEFAULT_FETCH_SEQUENCE_NUMBER_TIMEOUT : fetchSequenceNumberTimeout; + this.fetchThreads = fetchThreads; // we handle this being null later + + Preconditions.checkArgument( + !this.resetOffsetAutomatically || !this.skipSequenceNumberAvailabilityCheck, + "resetOffsetAutomatically cannot be used if skipSequenceNumberAvailabilityCheck=true" + ); + + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + + if (this.reportParseExceptions) { + this.maxParseExceptions = 0; + this.maxSavedParseExceptions = maxSavedParseExceptions == null ? 0 : Math.min(1, maxSavedParseExceptions); + } else { + this.maxParseExceptions = maxParseExceptions == null ? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS : maxParseExceptions; + this.maxSavedParseExceptions = maxSavedParseExceptions == null + ? TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS + : maxSavedParseExceptions; + } + this.logParseExceptions = logParseExceptions == null ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS : logParseExceptions; + } + + public static KinesisTuningConfig copyOf(KinesisTuningConfig config) + { + return new KinesisTuningConfig( + config.maxRowsInMemory, + config.maxBytesInMemory, + config.maxRowsPerSegment, + config.intermediatePersistPeriod, + config.basePersistDirectory, + config.maxPendingPersists, + config.indexSpec, + config.buildV9Directly, + config.reportParseExceptions, + config.handoffConditionTimeout, + config.resetOffsetAutomatically, + config.skipSequenceNumberAvailabilityCheck, + config.recordBufferSize, + config.recordBufferOfferTimeout, + config.recordBufferFullWait, + config.fetchSequenceNumberTimeout, + config.fetchThreads, + config.segmentWriteOutMediumFactory, + config.logParseExceptions, + config.maxParseExceptions, + config.maxSavedParseExceptions + ); + } + + @Override + @JsonProperty + public int getMaxRowsInMemory() + { + return maxRowsInMemory; + } + + @Override + @JsonProperty + public long getMaxBytesInMemory() + { + return maxBytesInMemory; + } + + @JsonProperty + public int getMaxRowsPerSegment() + { + return maxRowsPerSegment; + } + + @Override + @JsonProperty + public Period getIntermediatePersistPeriod() + { + return intermediatePersistPeriod; + } + + @Override + @JsonProperty + public File getBasePersistDirectory() + { + return basePersistDirectory; + } + + @Nullable + @Override + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + return segmentWriteOutMediumFactory; + } + + @Override + @JsonProperty + public int getMaxPendingPersists() + { + return maxPendingPersists; + } + + @Override + @JsonProperty + public IndexSpec getIndexSpec() + { + return indexSpec; + } + + @JsonProperty + public boolean getBuildV9Directly() + { + return buildV9Directly; + } + + @Override + @JsonProperty + public boolean isReportParseExceptions() + { + return reportParseExceptions; + } + + @JsonProperty + public long getHandoffConditionTimeout() + { + return handoffConditionTimeout; + } + + @JsonProperty + public boolean isResetOffsetAutomatically() + { + return resetOffsetAutomatically; + } + + @JsonProperty + public boolean isSkipSequenceNumberAvailabilityCheck() + { + return skipSequenceNumberAvailabilityCheck; + } + + @JsonProperty + public int getRecordBufferSize() + { + return recordBufferSize; + } + + @JsonProperty + public int getRecordBufferOfferTimeout() + { + return recordBufferOfferTimeout; + } + + @JsonProperty + public int getRecordBufferFullWait() + { + return recordBufferFullWait; + } + + @JsonProperty + public int getFetchSequenceNumberTimeout() + { + return fetchSequenceNumberTimeout; + } + + @JsonProperty + public Integer getFetchThreads() + { + return fetchThreads; + } + + @JsonProperty + public boolean isLogParseExceptions() + { + return logParseExceptions; + } + + @JsonProperty + public int getMaxParseExceptions() + { + return maxParseExceptions; + } + + @JsonProperty + public int getMaxSavedParseExceptions() + { + return maxSavedParseExceptions; + } + + public KinesisTuningConfig withBasePersistDirectory(File dir) + { + return new KinesisTuningConfig( + maxRowsInMemory, + maxBytesInMemory, + maxRowsPerSegment, + intermediatePersistPeriod, + dir, + maxPendingPersists, + indexSpec, + buildV9Directly, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + skipSequenceNumberAvailabilityCheck, + recordBufferSize, + recordBufferOfferTimeout, + recordBufferFullWait, + fetchSequenceNumberTimeout, + fetchThreads, + segmentWriteOutMediumFactory, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions + ); + } + + public KinesisTuningConfig withMaxRowsInMemory(int rows) + { + return new KinesisTuningConfig( + rows, + maxBytesInMemory, + maxRowsPerSegment, + intermediatePersistPeriod, + basePersistDirectory, + maxPendingPersists, + indexSpec, + buildV9Directly, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + skipSequenceNumberAvailabilityCheck, + recordBufferSize, + recordBufferOfferTimeout, + recordBufferFullWait, + fetchSequenceNumberTimeout, + fetchThreads, + segmentWriteOutMediumFactory, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions + ); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KinesisTuningConfig that = (KinesisTuningConfig) o; + return getMaxRowsInMemory() == that.getMaxRowsInMemory() && + getMaxBytesInMemory() == that.getMaxBytesInMemory() && + getMaxRowsPerSegment() == that.getMaxRowsPerSegment() && + getMaxPendingPersists() == that.getMaxPendingPersists() && + getBuildV9Directly() == that.getBuildV9Directly() && + isReportParseExceptions() == that.isReportParseExceptions() && + getHandoffConditionTimeout() == that.getHandoffConditionTimeout() && + isResetOffsetAutomatically() == that.isResetOffsetAutomatically() && + isSkipSequenceNumberAvailabilityCheck() == that.isSkipSequenceNumberAvailabilityCheck() && + getRecordBufferSize() == that.getRecordBufferSize() && + getRecordBufferOfferTimeout() == that.getRecordBufferOfferTimeout() && + getRecordBufferFullWait() == that.getRecordBufferFullWait() && + getFetchSequenceNumberTimeout() == that.getFetchSequenceNumberTimeout() && + isLogParseExceptions() == that.isLogParseExceptions() && + getMaxParseExceptions() == that.getMaxParseExceptions() && + getMaxSavedParseExceptions() == that.getMaxSavedParseExceptions() && + Objects.equals(getIntermediatePersistPeriod(), that.getIntermediatePersistPeriod()) && + Objects.equals(getBasePersistDirectory(), that.getBasePersistDirectory()) && + Objects.equals(getIndexSpec(), that.getIndexSpec()) && + Objects.equals(getFetchThreads(), that.getFetchThreads()) && + Objects.equals(getSegmentWriteOutMediumFactory(), that.getSegmentWriteOutMediumFactory()); + } + + @Override + public int hashCode() + { + return Objects.hash( + getMaxRowsInMemory(), + getMaxBytesInMemory(), + getMaxRowsPerSegment(), + getIntermediatePersistPeriod(), + getBasePersistDirectory(), + getMaxPendingPersists(), + getIndexSpec(), + getBuildV9Directly(), + isReportParseExceptions(), + getHandoffConditionTimeout(), + isResetOffsetAutomatically(), + isSkipSequenceNumberAvailabilityCheck(), + getRecordBufferSize(), + getRecordBufferOfferTimeout(), + getRecordBufferFullWait(), + getFetchSequenceNumberTimeout(), + getFetchThreads(), + getSegmentWriteOutMediumFactory(), + isLogParseExceptions(), + getMaxParseExceptions(), + getMaxSavedParseExceptions() + ); + } + + @Override + public String toString() + { + return "KinesisTuningConfig{" + + "maxRowsInMemory=" + maxRowsInMemory + + ", maxBytesInMemory=" + maxBytesInMemory + + ", maxRowsPerSegment=" + maxRowsPerSegment + + ", intermediatePersistPeriod=" + intermediatePersistPeriod + + ", basePersistDirectory=" + basePersistDirectory + + ", maxPendingPersists=" + maxPendingPersists + + ", indexSpec=" + indexSpec + + ", buildV9Directly=" + buildV9Directly + + ", reportParseExceptions=" + reportParseExceptions + + ", handoffConditionTimeout=" + handoffConditionTimeout + + ", resetOffsetAutomatically=" + resetOffsetAutomatically + + ", skipSequenceNumberAvailabilityCheck=" + skipSequenceNumberAvailabilityCheck + + ", recordBufferSize=" + recordBufferSize + + ", recordBufferOfferTimeout=" + recordBufferOfferTimeout + + ", recordBufferFullWait=" + recordBufferFullWait + + ", fetchSequenceNumberTimeout=" + fetchSequenceNumberTimeout + + ", fetchThreads=" + fetchThreads + + ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + + ", logParseExceptions=" + logParseExceptions + + ", maxParseExceptions=" + maxParseExceptions + + ", maxSavedParseExceptions=" + maxSavedParseExceptions + + '}'; + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/common/SequenceNumberPlus.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/common/SequenceNumberPlus.java new file mode 100644 index 000000000000..3f173498ba7a --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/common/SequenceNumberPlus.java @@ -0,0 +1,28 @@ +package org.apache.druid.indexing.kinesis.common; + +public class SequenceNumberPlus +{ + private final String sequenceNumber; + private final boolean exclusive; + + private SequenceNumberPlus(String sequenceNumber, boolean exclusive) + { + this.sequenceNumber = sequenceNumber; + this.exclusive = exclusive; + } + + public String get() + { + return sequenceNumber; + } + + public boolean isExclusive() + { + return exclusive; + } + + public static SequenceNumberPlus of(String sequenceNumber, boolean exclusive) + { + return new SequenceNumberPlus(sequenceNumber, exclusive); + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java new file mode 100644 index 000000000000..98b0d2432854 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -0,0 +1,2107 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis.supervisor; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.primitives.Ints; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.apache.commons.codec.digest.DigestUtils; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata; +import org.apache.druid.indexing.kinesis.KinesisIOConfig; +import org.apache.druid.indexing.kinesis.KinesisIndexTask; +import org.apache.druid.indexing.kinesis.KinesisIndexTaskClient; +import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; +import org.apache.druid.indexing.kinesis.KinesisPartitions; +import org.apache.druid.indexing.kinesis.KinesisRecordSupplier; +import org.apache.druid.indexing.kinesis.KinesisTuningConfig; +import org.apache.druid.indexing.kinesis.common.Record; +import org.apache.druid.indexing.kinesis.common.RecordSupplier; +import org.apache.druid.indexing.kinesis.common.SequenceNumberPlus; +import org.apache.druid.indexing.kinesis.common.StreamPartition; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskQueue; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.Supervisor; +import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.metadata.EntryExistsException; +import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +/** + * Supervisor responsible for managing the KafkaIndexTasks for a single dataSource. At a high level, the class accepts a + * {@link KinesisSupervisorSpec} which includes the Kafka topic and configuration as well as an ingestion spec which will + * be used to generate the indexing tasks. The run loop periodically refreshes its view of the Kafka topic's partitions + * and the list of running indexing tasks and ensures that all partitions are being read from and that there are enough + * tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of + * Kafka offsets. + */ +public class KinesisSupervisor implements Supervisor +{ + private static final EmittingLogger log = new EmittingLogger(KinesisSupervisor.class); + private static final Random RANDOM = new Random(); + private static final long MAX_RUN_FREQUENCY_MILLIS = 1000; // prevent us from running too often in response to events + private static final String NOT_SET = ""; + private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120; + + // Internal data structures + // -------------------------------------------------------- + + /** + * A TaskGroup is the main data structure used by KinesisSupervisor to organize and monitor Kafka partitions and + * indexing tasks. All the tasks in a TaskGroup should always be doing the same thing (reading the same partitions and + * starting from the same offset) and if [replicas] is configured to be 1, a TaskGroup will contain a single task (the + * exception being if the supervisor started up and discovered and adopted some already running tasks). At any given + * 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]). + */ + private static class TaskGroup + { + // 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 + // this task group has completed successfully, at which point this will be destroyed and a new task group will be + // created with new starting offsets. This allows us to create replacement tasks for failed tasks that process the + // same offsets, even if the values in [partitionGroups] has been changed. + final Map partitionOffsets; + + final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); + final Optional minimumMessageTime; + final Optional maximumMessageTime; + final Set exclusiveStartSequenceNumberPartitions; + + DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action + + public TaskGroup( + Map partitionOffsets, + Optional minimumMessageTime, + Optional maximumMessageTime, + Set exclusiveStartSequenceNumberPartitions + ) + { + this.partitionOffsets = partitionOffsets; + this.minimumMessageTime = minimumMessageTime; + this.maximumMessageTime = maximumMessageTime; + this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions != null + ? exclusiveStartSequenceNumberPartitions + : new HashSet<>(); + } + + Set taskIds() + { + return tasks.keySet(); + } + } + + private static class TaskData + { + TaskStatus status; + DateTime startTime; + } + + // Map<{group ID}, {actively reading task group}>; see documentation for TaskGroup class + private final ConcurrentHashMap taskGroups = new ConcurrentHashMap<>(); + + // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [taskGroups] to here so + // we can monitor its status while we queue new tasks to read the next range of offsets. This is a list since we could + // have multiple sets of tasks publishing at once if time-to-publish > taskDuration. + // Map<{group ID}, List<{pending completion task groups}>> + private final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); + + // 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 + // offset value from the metadata store, and if it can't find it there, from Kafka. Once a task begins + // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet- + // completed task, which will cause the next set of tasks to begin reading from where the previous task left + // off. If that previous task now fails, we will set the offset in [partitionGroups] back to NOT_SET which will + // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to + // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task + // failures during publishing. + // Map<{group ID}, Map<{partition ID}, {startingOffset}>> + private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); + // -------------------------------------------------------- + + private final TaskStorage taskStorage; + private final TaskMaster taskMaster; + private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; + private final KinesisIndexTaskClient taskClient; + private final ObjectMapper sortingMapper; + private final KinesisSupervisorSpec spec; + private final ServiceEmitter emitter; + private final DruidMonitorSchedulerConfig monitorSchedulerConfig; + private final String dataSource; + private final KinesisSupervisorIOConfig ioConfig; + private final KinesisSupervisorTuningConfig tuningConfig; + private final KinesisTuningConfig taskTuningConfig; + private final String supervisorId; + private final TaskInfoProvider taskInfoProvider; + private final long futureTimeoutInSeconds; // how long to wait for async operations to complete + private final RowIngestionMetersFactory rowIngestionMetersFactory; + + private final ExecutorService exec; + private final ScheduledExecutorService scheduledExec; + private final ListeningExecutorService workerExec; + private final BlockingQueue notices = new LinkedBlockingDeque<>(); + private final Object stopLock = new Object(); + private final Object stateChangeLock = new Object(); + + private boolean listenerRegistered = false; + private long lastRunTime; + + private volatile DateTime firstRunTime; + private volatile DateTime earlyPublishTime = null; + private volatile RecordSupplier recordSupplier; + + private volatile boolean started = false; + private volatile boolean stopped = false; + + private final ScheduledExecutorService metricEmittingExec; + // used while reporting lag + private final Map lastCurrentOffsets = new HashMap<>(); + + private final List partitionIds = new CopyOnWriteArrayList<>(); + private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); + + public KinesisSupervisor( + final TaskStorage taskStorage, + final TaskMaster taskMaster, + final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + final KinesisIndexTaskClientFactory taskClientFactory, + final ObjectMapper mapper, + final KinesisSupervisorSpec spec, + final RowIngestionMetersFactory rowIngestionMetersFactory + ) + { + this.taskStorage = taskStorage; + this.taskMaster = taskMaster; + this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; + this.sortingMapper = mapper.copy().configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); + this.spec = spec; + this.emitter = spec.getEmitter(); + this.monitorSchedulerConfig = spec.getMonitorSchedulerConfig(); + this.rowIngestionMetersFactory = rowIngestionMetersFactory; + + this.dataSource = spec.getDataSchema().getDataSource(); + this.ioConfig = spec.getIoConfig(); + this.tuningConfig = spec.getTuningConfig(); + this.taskTuningConfig = KinesisTuningConfig.copyOf(this.tuningConfig); + this.supervisorId = String.format("KinesisSupervisor-%s", dataSource); + this.exec = Execs.singleThreaded(supervisorId); + this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d"); + this.metricEmittingExec = Execs.scheduledSingleThreaded(supervisorId + "-Emitter-%d"); + + int workerThreads = (this.tuningConfig.getWorkerThreads() != null + ? this.tuningConfig.getWorkerThreads() + : Math.min(10, this.ioConfig.getTaskCount())); + this.workerExec = MoreExecutors.listeningDecorator(Execs.multiThreaded(workerThreads, supervisorId + "-Worker-%d")); + log.info("Created worker pool with [%d] threads for dataSource [%s]", workerThreads, this.dataSource); + + this.taskInfoProvider = new TaskInfoProvider() + { + @Override + public TaskLocation getTaskLocation(final String id) + { + Preconditions.checkNotNull(id, "id"); + Optional taskRunner = taskMaster.getTaskRunner(); + if (taskRunner.isPresent()) { + Optional item = Iterables.tryFind( + taskRunner.get().getRunningTasks(), new Predicate() + { + @Override + public boolean apply(TaskRunnerWorkItem taskRunnerWorkItem) + { + return id.equals(taskRunnerWorkItem.getTaskId()); + } + } + ); + + if (item.isPresent()) { + return item.get().getLocation(); + } + } else { + log.error("Failed to get task runner because I'm not the leader!"); + } + + return TaskLocation.unknown(); + } + + @Override + public Optional getTaskStatus(String id) + { + return taskStorage.getStatus(id); + } + }; + + this.futureTimeoutInSeconds = Math.max( + MINIMUM_FUTURE_TIMEOUT_IN_SECONDS, + tuningConfig.getChatRetries() * (tuningConfig.getHttpTimeout().getStandardSeconds() + + KinesisIndexTaskClient.MAX_RETRY_WAIT_SECONDS) + ); + + int chatThreads = (this.tuningConfig.getChatThreads() != null + ? this.tuningConfig.getChatThreads() + : Math.min(10, this.ioConfig.getTaskCount() * this.ioConfig.getReplicas())); + this.taskClient = taskClientFactory.build( + taskInfoProvider, + dataSource, + chatThreads, + this.tuningConfig.getHttpTimeout(), + this.tuningConfig.getChatRetries() + ); + log.info( + "Created taskClient with dataSource[%s] chatThreads[%d] httpTimeout[%s] chatRetries[%d]", + dataSource, + chatThreads, + this.tuningConfig.getHttpTimeout(), + this.tuningConfig.getChatRetries() + ); + } + + @Override + public void start() + { + synchronized (stateChangeLock) { + Preconditions.checkState(!started, "already started"); + Preconditions.checkState(!exec.isShutdown(), "already stopped"); + + try { + setupRecordSupplier(); + + exec.submit( + new Runnable() + { + @Override + public void run() + { + try { + while (!Thread.currentThread().isInterrupted()) { + final Notice notice = notices.take(); + + try { + notice.handle(); + } + catch (Throwable e) { + log.makeAlert(e, "KinesisSupervisor[%s] failed to handle notice", dataSource) + .addData("noticeClass", notice.getClass().getSimpleName()) + .emit(); + } + } + } + catch (InterruptedException e) { + log.info("KinesisSupervisor[%s] interrupted, exiting", dataSource); + } + } + } + ); + firstRunTime = DateTime.now().plus(ioConfig.getStartDelay()); + scheduledExec.scheduleAtFixedRate( + buildRunTask(), + ioConfig.getStartDelay().getMillis(), + Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS), + TimeUnit.MILLISECONDS + ); + +// TODO: Implement this for Kinesis which uses approximate time from latest instead of offset lag +/* + metricEmittingExec.scheduleAtFixedRate( + computeAndEmitLag(taskClient), + ioConfig.getStartDelay().getMillis() + 10000, // wait for tasks to start up + Math.max(monitorSchedulerConfig.getEmitterPeriod().getMillis(), 60 * 1000), + TimeUnit.MILLISECONDS + ); +*/ + started = true; + log.info( + "Started KinesisSupervisor[%s], first run in [%s], with spec: [%s]", + dataSource, + ioConfig.getStartDelay(), + spec.toString() + ); + } + catch (Exception e) { + if (recordSupplier != null) { + recordSupplier.close(); + } + log.makeAlert(e, "Exception starting KinesisSupervisor[%s]", dataSource) + .emit(); + throw Throwables.propagate(e); + } + } + } + + @Override + public void stop(boolean stopGracefully) + { + synchronized (stateChangeLock) { + Preconditions.checkState(started, "not started"); + + log.info("Beginning shutdown of KinesisSupervisor[%s]", dataSource); + + try { + scheduledExec.shutdownNow(); // stop recurring executions + metricEmittingExec.shutdownNow(); + recordSupplier.close(); // aborts any in-flight sequenceNumber fetches + + Optional taskRunner = taskMaster.getTaskRunner(); + if (taskRunner.isPresent()) { + taskRunner.get().unregisterListener(supervisorId); + } + + // Stopping gracefully will synchronize the end offsets of the tasks and signal them to publish, and will block + // until the tasks have acknowledged or timed out. We want this behavior when we're explicitly shut down through + // the API, but if we shut down for other reasons (e.g. we lose leadership) we want to just stop and leave the + // tasks as they are. + synchronized (stopLock) { + if (stopGracefully) { + log.info("Posting GracefulShutdownNotice, signalling managed tasks to complete and publish"); + notices.add(new GracefulShutdownNotice()); + } else { + log.info("Posting ShutdownNotice"); + notices.add(new ShutdownNotice()); + } + + long shutdownTimeoutMillis = tuningConfig.getShutdownTimeout().getMillis(); + long endTime = System.currentTimeMillis() + shutdownTimeoutMillis; + while (!stopped) { + long sleepTime = endTime - System.currentTimeMillis(); + if (sleepTime <= 0) { + log.info("Timed out while waiting for shutdown (timeout [%,dms])", shutdownTimeoutMillis); + stopped = true; + break; + } + stopLock.wait(sleepTime); + } + } + log.info("Shutdown notice handled"); + + taskClient.close(); + workerExec.shutdownNow(); + exec.shutdownNow(); + started = false; + + log.info("KinesisSupervisor[%s] has stopped", dataSource); + } + catch (Exception e) { + log.makeAlert(e, "Exception stopping KinesisSupervisor[%s]", dataSource) + .emit(); + } + } + } + + @Override + public SupervisorReport getStatus() + { + return generateReport(true); + } + + @Override + public Map> getStats() + { + try { + return getCurrentTotalStats(); + } + catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + log.error(ie, "getStats() interrupted."); + throw new RuntimeException(ie); + } + catch (ExecutionException | TimeoutException eete) { + throw new RuntimeException(eete); + } + } + + @Override + public void reset(DataSourceMetadata dataSourceMetadata) + { + log.info("Posting ResetNotice"); + notices.add(new ResetNotice(dataSourceMetadata)); + } + + @Override + public void checkpoint( + @Nullable Integer taskGroupId, + @Deprecated String baseSequenceName, + DataSourceMetadata previousCheckPoint, + DataSourceMetadata currentCheckPoint + ) + { + // not supported right now + } + + public void possiblyRegisterListener() + { + // getTaskRunner() sometimes fails if the task queue is still being initialized so retry later until we succeed + + if (listenerRegistered) { + return; + } + + Optional taskRunner = taskMaster.getTaskRunner(); + if (taskRunner.isPresent()) { + taskRunner.get().registerListener( + new TaskRunnerListener() + { + @Override + public String getListenerId() + { + return supervisorId; + } + + @Override + public void locationChanged(final String taskId, final TaskLocation newLocation) + { + // do nothing + } + + @Override + public void statusChanged(String taskId, TaskStatus status) + { + notices.add(new RunNotice()); + } + }, MoreExecutors.sameThreadExecutor() + ); + + listenerRegistered = true; + } + } + + private interface Notice + { + void handle() throws ExecutionException, InterruptedException, TimeoutException; + } + + private class RunNotice implements Notice + { + @Override + public void handle() throws ExecutionException, InterruptedException, TimeoutException + { + long nowTime = System.currentTimeMillis(); + if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { + return; + } + lastRunTime = nowTime; + + runInternal(); + } + } + + private class GracefulShutdownNotice extends ShutdownNotice + { + @Override + public void handle() throws InterruptedException, ExecutionException, TimeoutException + { + gracefulShutdownInternal(); + super.handle(); + } + } + + private class ShutdownNotice implements Notice + { + @Override + public void handle() throws InterruptedException, ExecutionException, TimeoutException + { + recordSupplier.close(); + + synchronized (stopLock) { + stopped = true; + stopLock.notifyAll(); + } + } + } + + private class ResetNotice implements Notice + { + final DataSourceMetadata dataSourceMetadata; + + ResetNotice(DataSourceMetadata dataSourceMetadata) + { + this.dataSourceMetadata = dataSourceMetadata; + } + + @Override + public void handle() + { + log.makeAlert("Resetting dataSource [%s]", dataSource).emit(); + resetInternal(dataSourceMetadata); + } + } + + @VisibleForTesting + void resetInternal(DataSourceMetadata dataSourceMetadata) + { + if (dataSourceMetadata == null) { + // Reset everything + boolean result = indexerMetadataStorageCoordinator.deleteDataSourceMetadata(dataSource); + log.info("Reset dataSource[%s] - dataSource metadata entry deleted? [%s]", dataSource, result); + killTaskGroupForPartitions(partitionIds); + } else if (!(dataSourceMetadata instanceof KinesisDataSourceMetadata)) { + throw new IAE("Expected KinesisDataSourceMetadata but found instance of [%s]", dataSourceMetadata.getClass()); + } else { + // Reset only the partitions in dataSourceMetadata if it has not been reset yet + final KinesisDataSourceMetadata resetKafkaMetadata = (KinesisDataSourceMetadata) dataSourceMetadata; + + if (resetKafkaMetadata.getKinesisPartitions().getStream().equals(ioConfig.getStream())) { + // metadata can be null + final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); + if (metadata != null && !(metadata instanceof KinesisDataSourceMetadata)) { + throw new IAE( + "Expected KinesisDataSourceMetadata from metadata store but found instance of [%s]", + metadata.getClass() + ); + } + final KinesisDataSourceMetadata currentMetadata = (KinesisDataSourceMetadata) metadata; + + // defend against consecutive reset requests from replicas + // as well as the case where the metadata store do not have an entry for the reset partitions + boolean doReset = false; + for (Map.Entry resetPartitionOffset : resetKafkaMetadata.getKinesisPartitions() + .getPartitionSequenceNumberMap() + .entrySet()) { + final String partitionOffsetInMetadataStore = currentMetadata == null + ? null + : currentMetadata.getKinesisPartitions() + .getPartitionSequenceNumberMap() + .get(resetPartitionOffset.getKey()); + final TaskGroup partitionTaskGroup = taskGroups.get(getTaskGroupIdForPartition(resetPartitionOffset.getKey())); + if (partitionOffsetInMetadataStore != null || + (partitionTaskGroup != null && partitionTaskGroup.partitionOffsets.get(resetPartitionOffset.getKey()) + .equals(resetPartitionOffset.getValue()))) { + doReset = true; + break; + } + } + + if (!doReset) { + return; + } + + boolean metadataUpdateSuccess = false; + if (currentMetadata == null) { + metadataUpdateSuccess = true; + } else { + final DataSourceMetadata newMetadata = currentMetadata.minus(resetKafkaMetadata); + try { + metadataUpdateSuccess = indexerMetadataStorageCoordinator.resetDataSourceMetadata(dataSource, newMetadata); + } + catch (IOException e) { + log.error("Resetting DataSourceMetadata failed [%s]", e.getMessage()); + Throwables.propagate(e); + } + } + if (metadataUpdateSuccess) { + killTaskGroupForPartitions( + resetKafkaMetadata.getKinesisPartitions() + .getPartitionSequenceNumberMap() + .keySet() + ); + } else { + throw new ISE("Unable to reset metadata"); + } + } else { + log.warn( + "Reset metadata topic [%s] and supervisor's topic [%s] do not match", + resetKafkaMetadata.getKinesisPartitions().getStream(), + ioConfig.getStream() + ); + } + } + } + + private void killTaskGroupForPartitions(Collection partitions) + { + for (String partition : partitions) { + TaskGroup taskGroup = taskGroups.get(getTaskGroupIdForPartition(partition)); + if (taskGroup != null) { + // kill all tasks in this task group + for (String taskId : taskGroup.tasks.keySet()) { + log.info("Reset dataSource[%s] - killing task [%s]", dataSource, taskId); + killTask(taskId); + } + } + partitionGroups.remove(getTaskGroupIdForPartition(partition)); + taskGroups.remove(getTaskGroupIdForPartition(partition)); + } + } + + @VisibleForTesting + void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException + { + // Prepare for shutdown by 1) killing all tasks that haven't been assigned to a worker yet, and 2) causing all + // running tasks to begin publishing by setting their startTime to a very long time ago so that the logic in + // checkTaskDuration() will be triggered. This is better than just telling these tasks to publish whatever they + // have, as replicas that are supposed to publish the same segment may not have read the same set of offsets. + for (TaskGroup taskGroup : taskGroups.values()) { + for (Map.Entry entry : taskGroup.tasks.entrySet()) { + if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) { + killTask(entry.getKey()); + } else { + entry.getValue().startTime = new DateTime(0); + } + } + } + + checkTaskDuration(); + } + + @VisibleForTesting + void runInternal() throws ExecutionException, InterruptedException, TimeoutException + { + possiblyRegisterListener(); + updatePartitionDataFromKinesis(); + discoverTasks(); + updateTaskStatus(); + checkTaskDuration(); + checkPendingCompletionTasks(); + checkCurrentTaskState(); + + if (!spec.isSuspended()) { + log.info("[%s] supervisor is running.", dataSource); + createNewTasks(); + } else { + log.info("[%s] supervisor is suspended.", dataSource); + gracefulShutdownInternal(); + } + + if (log.isDebugEnabled()) { + log.debug(generateReport(true).toString()); + } else { + log.info(generateReport(false).toString()); + } + } + + @VisibleForTesting + String generateSequenceName(int groupId) + { + StringBuilder sb = new StringBuilder(); + Map startPartitions = taskGroups.get(groupId).partitionOffsets; + + for (Map.Entry entry : startPartitions.entrySet()) { + sb.append(String.format("+%s(%s)", entry.getKey(), entry.getValue())); + } + String partitionOffsetStr = sb.toString().substring(1); + + Optional minimumMessageTime = taskGroups.get(groupId).minimumMessageTime; + Optional maximumMessageTime = taskGroups.get(groupId).maximumMessageTime; + String minMsgTimeStr = (minimumMessageTime.isPresent() ? String.valueOf(minimumMessageTime.get().getMillis()) : ""); + String maxMsgTimeStr = (maximumMessageTime.isPresent() ? String.valueOf(maximumMessageTime.get().getMillis()) : ""); + + String dataSchema, tuningConfig; + try { + dataSchema = sortingMapper.writeValueAsString(spec.getDataSchema()); + tuningConfig = sortingMapper.writeValueAsString(taskTuningConfig); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } + + String hashCode = DigestUtils.sha1Hex(dataSchema + tuningConfig + partitionOffsetStr + minMsgTimeStr + maxMsgTimeStr) + .substring(0, 15); + + return Joiner.on("_").join("index_kinesis", dataSource, hashCode); + } + + private static String getRandomId() + { + final StringBuilder suffix = new StringBuilder(8); + for (int i = 0; i < Ints.BYTES * 2; ++i) { + suffix.append((char) ('a' + ((RANDOM.nextInt() >>> (i * 4)) & 0x0F))); + } + return suffix.toString(); + } + + private RecordSupplier setupRecordSupplier() + { + if (recordSupplier == null) { + recordSupplier = new KinesisRecordSupplier( + ioConfig.getEndpoint(), + ioConfig.getAwsAccessKeyId(), + ioConfig.getAwsSecretAccessKey(), + ioConfig.getRecordsPerFetch(), + ioConfig.getFetchDelayMillis(), + 1, + ioConfig.getAwsAssumedRoleArn(), + ioConfig.getAwsExternalId(), + ioConfig.isDeaggregate(), + taskTuningConfig.getRecordBufferSize(), + taskTuningConfig.getRecordBufferOfferTimeout(), + taskTuningConfig.getRecordBufferFullWait(), + taskTuningConfig.getFetchSequenceNumberTimeout() + ); + } + + return recordSupplier; + } + + private void updatePartitionDataFromKinesis() + { + Set partitionIds = recordSupplier.getPartitionIds(ioConfig.getStream()); + + if (partitionIds == null) { + log.warn("Could not fetch partition IDs for stream[%s]", ioConfig.getStream()); + return; + } + + log.debug("Found [%d] Kinesis partitions for stream [%s]", partitionIds.size(), ioConfig.getStream()); + + Set closedPartitions = getOffsetsFromMetadataStorage() + .entrySet() + .stream() + .filter(x -> Record.END_OF_SHARD_MARKER.equals(x.getValue())) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + + boolean initialPartitionDiscovery = this.partitionIds.isEmpty(); + for (String partition : partitionIds) { + if (closedPartitions.contains(partition)) { + continue; + } + + if (!initialPartitionDiscovery && !this.partitionIds.contains(partition)) { + subsequentlyDiscoveredPartitions.add(partition); + + if (earlyPublishTime == null) { + for (TaskGroup taskGroup : taskGroups.values()) { + if (!taskGroup.taskIds().isEmpty()) { + // a new partition was added and we are managing active tasks - set an early publish time 2 minutes in the + // future to give things time to settle + + earlyPublishTime = DateTime.now().plusMinutes(2); + log.info("New partition discovered - requesting early publish in 2 minutes [%s]", earlyPublishTime); + break; + } + } + } + } + + int taskGroupId = getTaskGroupIdForPartition(partition); + partitionGroups.putIfAbsent(taskGroupId, new ConcurrentHashMap<>()); + + ConcurrentHashMap partitionMap = partitionGroups.get(taskGroupId); + + // 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 + // offset value from the metadata store, and if it can't find it there, from Kafka. Once a task begins + // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet- + // completed task, which will cause the next set of tasks to begin reading from where the previous task left + // off. If that previous task now fails, we will set the offset in [partitionGroups] back to NOT_SET which will + // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to + // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task + // failures during publishing. + if (partitionMap.putIfAbsent(partition, NOT_SET) == null) { + log.info( + "New partition [%s] discovered for topic [%s], added to task group [%d]", + partition, + ioConfig.getStream(), + taskGroupId + ); + } + } + } + + private void discoverTasks() throws ExecutionException, InterruptedException, TimeoutException + { + int taskCount = 0; + List futureTaskIds = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + List tasks = taskStorage.getActiveTasks(); + + for (Task task : tasks) { + if (!(task instanceof KinesisIndexTask) || !dataSource.equals(task.getDataSource())) { + continue; + } + + taskCount++; + final KinesisIndexTask kinesisTask = (KinesisIndexTask) task; + final String taskId = task.getId(); + + // Determine which task group this task belongs to based on one of the partitions handled by this task. If we + // later determine that this task is actively reading, we will make sure that it matches our current partition + // allocation (getTaskGroupIdForPartition(partition) should return the same value for every partition being read + // by this task) and kill it if it is not compatible. If the task is instead found to be in the publishing + // state, we will permit it to complete even if it doesn't match our current partition allocation to support + // seamless schema migration. + + Iterator it = kinesisTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceNumberMap() + .keySet() + .iterator(); + final Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null); + + if (taskGroupId != null) { + // check to see if we already know about this task, either in [taskGroups] or in [pendingCompletionTaskGroups] + // and if not add it to taskGroups or pendingCompletionTaskGroups (if status = PUBLISHING) + TaskGroup taskGroup = taskGroups.get(taskGroupId); + if (!isTaskInPendingCompletionGroups(taskId) && (taskGroup == null || !taskGroup.tasks.containsKey(taskId))) { + + futureTaskIds.add(taskId); + futures.add( + Futures.transform( + taskClient.getStatusAsync(taskId), new Function() + { + @Override + public Boolean apply(KinesisIndexTask.Status status) + { + if (status == KinesisIndexTask.Status.PUBLISHING) { + addDiscoveredTaskToPendingCompletionTaskGroups( + taskGroupId, + taskId, + kinesisTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceNumberMap() + ); + + // update partitionGroups with the publishing task's offsets (if they are greater than what is + // existing) so that the next tasks will start reading from where this task left off + Map publishingTaskCurrentOffsets = taskClient.getCurrentOffsets(taskId, true); + + for (Map.Entry entry : publishingTaskCurrentOffsets.entrySet()) { + String partition = entry.getKey(); + String offset = entry.getValue(); + ConcurrentHashMap partitionOffsets = partitionGroups.get( + getTaskGroupIdForPartition(partition) + ); + + boolean succeeded; + do { + succeeded = true; + String previousOffset = partitionOffsets.putIfAbsent(partition, offset); + if (previousOffset != null && previousOffset.compareTo(offset) < 0) { + succeeded = partitionOffsets.replace(partition, previousOffset, offset); + } + } while (!succeeded); + } + + } else { + for (String partition : kinesisTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceNumberMap() + .keySet()) { + if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { + log.warn( + "Stopping task [%s] which does not match the expected partition allocation", + taskId + ); + try { + stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + catch (InterruptedException | ExecutionException | TimeoutException e) { + log.warn(e, "Exception while stopping task"); + } + return false; + } + } + + if (taskGroups.putIfAbsent( + taskGroupId, + new TaskGroup( + ImmutableMap.copyOf( + kinesisTask.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap() + ), + kinesisTask.getIOConfig().getMinimumMessageTime(), + kinesisTask.getIOConfig().getMaximumMessageTime(), + kinesisTask.getIOConfig().getExclusiveStartSequenceNumberPartitions() + ) + ) == null) { + log.debug("Created new task group [%d]", taskGroupId); + } + + if (!isTaskCurrent(taskGroupId, taskId)) { + log.info( + "Stopping task [%s] which does not match the expected parameters and ingestion spec", + taskId + ); + try { + stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + catch (InterruptedException | ExecutionException | TimeoutException e) { + log.warn(e, "Exception while stopping task"); + } + return false; + } else { + taskGroups.get(taskGroupId).tasks.putIfAbsent(taskId, new TaskData()); + } + } + return true; + } + }, workerExec + ) + ); + } + } + } + + List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + for (int i = 0; i < results.size(); i++) { + if (results.get(i) == null) { + String taskId = futureTaskIds.get(i); + log.warn("Task [%s] failed to return status, killing task", taskId); + killTask(taskId); + } + } + log.debug("Found [%d] Kafka indexing tasks for dataSource [%s]", taskCount, dataSource); + } + + private void addDiscoveredTaskToPendingCompletionTaskGroups( + int groupId, + String taskId, + Map startingPartitions + ) + { + pendingCompletionTaskGroups.putIfAbsent(groupId, Lists.newCopyOnWriteArrayList()); + + CopyOnWriteArrayList taskGroupList = pendingCompletionTaskGroups.get(groupId); + for (TaskGroup taskGroup : taskGroupList) { + if (taskGroup.partitionOffsets.equals(startingPartitions)) { + if (taskGroup.tasks.putIfAbsent(taskId, new TaskData()) == null) { + log.info("Added discovered task [%s] to existing pending task group", taskId); + } + return; + } + } + + log.info("Creating new pending completion task group for discovered task [%s]", taskId); + + // reading the minimumMessageTime & maximumMessageTime from the publishing task and setting it here is not necessary as this task cannot + // change to a state where it will read any more events + TaskGroup newTaskGroup = new TaskGroup(ImmutableMap.copyOf(startingPartitions), Optional.absent(), Optional.absent(), null); + + newTaskGroup.tasks.put(taskId, new TaskData()); + newTaskGroup.completionTimeout = DateTime.now().plus(ioConfig.getCompletionTimeout()); + + taskGroupList.add(newTaskGroup); + } + + private void updateTaskStatus() throws ExecutionException, InterruptedException, TimeoutException + { + final List> futures = Lists.newArrayList(); + final List futureTaskIds = Lists.newArrayList(); + + // update status (and startTime if unknown) of current tasks in taskGroups + for (TaskGroup group : taskGroups.values()) { + for (Map.Entry entry : group.tasks.entrySet()) { + final String taskId = entry.getKey(); + final TaskData taskData = entry.getValue(); + + if (taskData.startTime == null) { + futureTaskIds.add(taskId); + futures.add( + Futures.transform( + taskClient.getStartTimeAsync(taskId), new Function() + { + @Nullable + @Override + public Boolean apply(@Nullable DateTime startTime) + { + if (startTime == null) { + return false; + } + + taskData.startTime = startTime; + long millisRemaining = ioConfig.getTaskDuration().getMillis() - (System.currentTimeMillis() + - taskData.startTime.getMillis()); + if (millisRemaining > 0) { + scheduledExec.schedule( + buildRunTask(), + millisRemaining + MAX_RUN_FREQUENCY_MILLIS, + TimeUnit.MILLISECONDS + ); + } + + return true; + } + }, workerExec + ) + ); + } + + taskData.status = taskStorage.getStatus(taskId).get(); + } + } + + // update status of pending completion tasks in pendingCompletionTaskGroups + for (List taskGroups : pendingCompletionTaskGroups.values()) { + for (TaskGroup group : taskGroups) { + for (Map.Entry entry : group.tasks.entrySet()) { + entry.getValue().status = taskStorage.getStatus(entry.getKey()).get(); + } + } + } + + List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + for (int i = 0; i < results.size(); i++) { + // false means the task hasn't started running yet and that's okay; null means it should be running but the HTTP + // request threw an exception so kill the task + if (results.get(i) == null) { + String taskId = futureTaskIds.get(i); + log.warn("Task [%s] failed to return start time, killing task", taskId); + killTask(taskId); + } + } + } + + private void checkTaskDuration() + throws InterruptedException, ExecutionException, TimeoutException + { + final List>> futures = Lists.newArrayList(); + final List futureGroupIds = Lists.newArrayList(); + + for (Map.Entry entry : taskGroups.entrySet()) { + Integer groupId = entry.getKey(); + TaskGroup group = entry.getValue(); + + // find the longest running task from this group + DateTime earliestTaskStart = DateTime.now(); + for (TaskData taskData : group.tasks.values()) { + if (earliestTaskStart.isAfter(taskData.startTime)) { + earliestTaskStart = taskData.startTime; + } + } + + boolean doEarlyPublish = false; + if (earlyPublishTime != null && (earlyPublishTime.isBeforeNow() || earlyPublishTime.isEqualNow())) { + log.info("Early publish requested - signalling tasks to publish"); + + earlyPublishTime = null; + doEarlyPublish = true; + } + + // if this task has run longer than the configured duration, signal all tasks in the group to persist + if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow() || doEarlyPublish) { + log.info("Task group [%d] has run for [%s]", groupId, ioConfig.getTaskDuration()); + futureGroupIds.add(groupId); + futures.add(signalTasksToFinish(groupId)); + } + } + + List> results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + for (int j = 0; j < results.size(); j++) { + Integer groupId = futureGroupIds.get(j); + TaskGroup group = taskGroups.get(groupId); + Map endOffsets = results.get(j); + + if (endOffsets != null) { + // set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion + group.completionTimeout = DateTime.now().plus(ioConfig.getCompletionTimeout()); + pendingCompletionTaskGroups.putIfAbsent(groupId, Lists.newCopyOnWriteArrayList()); + pendingCompletionTaskGroups.get(groupId).add(group); + + // set endOffsets as the next startOffsets + for (Map.Entry entry : endOffsets.entrySet()) { + partitionGroups.get(groupId).put(entry.getKey(), entry.getValue()); + } + } else { + log.warn( + "All tasks in group [%s] failed to transition to publishing state, killing tasks [%s]", + groupId, + group.taskIds() + ); + for (String id : group.taskIds()) { + killTask(id); + } + } + + // remove this task group from the list of current task groups now that it has been handled + taskGroups.remove(groupId); + } + } + + private ListenableFuture> signalTasksToFinish(final int groupId) + { + final TaskGroup taskGroup = taskGroups.get(groupId); + + // 1) Check if any task completed (in which case we're done) and kill unassigned tasks + Iterator> i = taskGroup.tasks.entrySet().iterator(); + while (i.hasNext()) { + Map.Entry taskEntry = i.next(); + String taskId = taskEntry.getKey(); + TaskData task = taskEntry.getValue(); + + if (task.status.isSuccess()) { + // If any task in this group has already completed, stop the rest of the tasks in the group and return. + // This will cause us to create a new set of tasks next cycle that will start from the offsets in + // 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) + return Futures.transform( + stopTasksInGroup(taskGroup), new Function>() + { + @Nullable + @Override + public Map apply(@Nullable Object input) + { + return null; + } + } + ); + } + + if (task.status.isRunnable()) { + if (taskInfoProvider.getTaskLocation(taskId).equals(TaskLocation.unknown())) { + log.info("Killing task [%s] which hasn't been assigned to a worker", taskId); + killTask(taskId); + i.remove(); + } + } + } + + // 2) Pause running tasks + final List>> pauseFutures = Lists.newArrayList(); + final List pauseTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); + for (final String taskId : pauseTaskIds) { + pauseFutures.add(taskClient.pauseAsync(taskId)); + } + + return Futures.transform( + Futures.successfulAsList(pauseFutures), new Function>, Map>() + { + @Nullable + @Override + public Map apply(List> input) + { + // 3) Build a map of the highest offset read by any task in the group for each partition + final Map endOffsets = new HashMap<>(); + for (int i = 0; i < input.size(); i++) { + Map result = input.get(i); + + if (result == null || result.isEmpty()) { // kill tasks that didn't return a value + String taskId = pauseTaskIds.get(i); + log.warn("Task [%s] failed to respond to [pause] in a timely manner, killing task", taskId); + killTask(taskId); + taskGroup.tasks.remove(taskId); + + } else { // otherwise build a map of the highest offsets seen + for (Map.Entry offset : result.entrySet()) { + if (!endOffsets.containsKey(offset.getKey()) + || endOffsets.get(offset.getKey()).compareTo(offset.getValue()) < 0) { + endOffsets.put(offset.getKey(), offset.getValue()); + } + } + } + } + + // 4) Set the end offsets for each task to the values from step 3 and resume the tasks. All the tasks should + // finish reading and start publishing within a short period, depending on how in sync the tasks were. + final List> setEndOffsetFutures = Lists.newArrayList(); + final List setEndOffsetTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); + + if (setEndOffsetTaskIds.isEmpty()) { + log.info("All tasks in taskGroup [%d] have failed, tasks will be re-created", groupId); + return null; + } + + log.info("Setting endOffsets for tasks in taskGroup [%d] to %s and resuming", groupId, endOffsets); + for (final String taskId : setEndOffsetTaskIds) { + setEndOffsetFutures.add(taskClient.setEndOffsetsAsync(taskId, endOffsets, true)); + } + + try { + List results = Futures.successfulAsList(setEndOffsetFutures) + .get(futureTimeoutInSeconds, TimeUnit.SECONDS); + for (int i = 0; i < results.size(); i++) { + if (results.get(i) == null || !results.get(i)) { + String taskId = setEndOffsetTaskIds.get(i); + log.warn("Task [%s] failed to respond to [set end offsets] in a timely manner, killing task", taskId); + killTask(taskId); + taskGroup.tasks.remove(taskId); + } + } + } + catch (Exception e) { + Throwables.propagate(e); + } + + if (taskGroup.tasks.isEmpty()) { + log.info("All tasks in taskGroup [%d] have failed, tasks will be re-created", groupId); + return null; + } + + return endOffsets; + } + }, workerExec + ); + } + + /** + * Monitors [pendingCompletionTaskGroups] for tasks that have completed. If any task in a task group has completed, we + * can safely stop the rest of the tasks in that group. If a task group has exceeded its publishing timeout, then + * we need to stop all tasks in not only that task group but also 1) any subsequent task group that is also pending + * completion and 2) the current task group that is running, because the assumption that we have handled up to the + * starting offset for subsequent task groups is no longer valid, and subsequent tasks would fail as soon as they + * attempted to publish because of the contiguous range consistency check. + */ + private void checkPendingCompletionTasks() throws ExecutionException, InterruptedException, TimeoutException + { + List> futures = Lists.newArrayList(); + + for (Map.Entry> pendingGroupList : pendingCompletionTaskGroups.entrySet()) { + + boolean stopTasksInTaskGroup = false; + Integer groupId = pendingGroupList.getKey(); + CopyOnWriteArrayList taskGroupList = pendingGroupList.getValue(); + List toRemove = Lists.newArrayList(); + + for (TaskGroup group : taskGroupList) { + boolean foundSuccess = false, entireTaskGroupFailed = false; + + if (stopTasksInTaskGroup) { + // One of the earlier groups that was handling the same partition set timed out before the segments were + // published so stop any additional groups handling the same partition set that are pending completion. + futures.add(stopTasksInGroup(group)); + toRemove.add(group); + continue; + } + + Iterator> iTask = group.tasks.entrySet().iterator(); + while (iTask.hasNext()) { + Map.Entry task = iTask.next(); + + if (task.getValue().status.isFailure()) { + iTask.remove(); // remove failed task + if (group.tasks.isEmpty()) { + // if all tasks in the group have failed, just nuke all task groups with this partition set and restart + entireTaskGroupFailed = true; + break; + } + } + + if (task.getValue().status.isSuccess()) { + // If one of the pending completion tasks was successful, stop the rest of the tasks in the group as + // we no longer need them to publish their segment. + log.info("Task [%s] completed successfully, stopping tasks %s", task.getKey(), group.taskIds()); + futures.add(stopTasksInGroup(group)); + foundSuccess = true; + toRemove.add(group); // remove the TaskGroup from the list of pending completion task groups + break; // skip iterating the rest of the tasks in this group as they've all been stopped now + } + } + + if ((!foundSuccess && group.completionTimeout.isBeforeNow()) || entireTaskGroupFailed) { + if (entireTaskGroupFailed) { + log.warn("All tasks in group [%d] failed to publish, killing all tasks for these partitions", groupId); + } else { + log.makeAlert( + "No task in [%s] succeeded before the completion timeout elapsed [%s]!", + group.taskIds(), + ioConfig.getCompletionTimeout() + ).emit(); + } + + // reset partitions offsets for this task group so that they will be re-read from metadata storage + partitionGroups.remove(groupId); + + // stop all the tasks in this pending completion group + futures.add(stopTasksInGroup(group)); + + // set a flag so the other pending completion groups for this set of partitions will also stop + stopTasksInTaskGroup = true; + + // stop all the tasks in the currently reading task group and remove the bad task group + futures.add(stopTasksInGroup(taskGroups.remove(groupId))); + + toRemove.add(group); + } + } + + taskGroupList.removeAll(toRemove); + } + + // wait for all task shutdowns to complete before returning + Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + + private void checkCurrentTaskState() throws ExecutionException, InterruptedException, TimeoutException + { + List> futures = Lists.newArrayList(); + Iterator> iTaskGroups = taskGroups.entrySet().iterator(); + while (iTaskGroups.hasNext()) { + Map.Entry taskGroupEntry = iTaskGroups.next(); + Integer groupId = taskGroupEntry.getKey(); + TaskGroup taskGroup = taskGroupEntry.getValue(); + + // Iterate the list of known tasks in this group and: + // 1) Kill any tasks which are not "current" (have the partitions, starting offsets, and minimumMessageTime & maximumMessageTime + // (if applicable) in [taskGroups]) + // 2) Remove any tasks that have failed from the list + // 3) If any task completed successfully, stop all the tasks in this group and move to the next group + + log.debug("Task group [%d] pre-pruning: %s", groupId, taskGroup.taskIds()); + + Iterator> iTasks = taskGroup.tasks.entrySet().iterator(); + while (iTasks.hasNext()) { + Map.Entry task = iTasks.next(); + String taskId = task.getKey(); + TaskData taskData = task.getValue(); + + // stop and remove bad tasks from the task group + if (!isTaskCurrent(groupId, taskId)) { + log.info("Stopping task [%s] which does not match the expected offset range and ingestion spec", taskId); + futures.add(stopTask(taskId, false)); + iTasks.remove(); + continue; + } + + // remove failed tasks + if (taskData.status.isFailure()) { + iTasks.remove(); + continue; + } + + // check for successful tasks, and if we find one, stop all tasks in the group and remove the group so it can + // be recreated with the next set of offsets + if (taskData.status.isSuccess()) { + futures.add(stopTasksInGroup(taskGroup)); + iTaskGroups.remove(); + break; + } + } + log.debug("Task group [%d] post-pruning: %s", groupId, taskGroup.taskIds()); + } + + // wait for all task shutdowns to complete before returning + Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + } + + void createNewTasks() + { + // check that there is a current task group for each group of partitions in [partitionGroups] + for (Integer groupId : partitionGroups.keySet()) { + if (!taskGroups.containsKey(groupId)) { + log.info( + "Creating new task group [%d] for partitions %s", + groupId, + partitionGroups.get(groupId).keySet() + ); + + Optional minimumMessageTime = (ioConfig.getLateMessageRejectionPeriod().isPresent() ? Optional.of( + DateTime.now().minus(ioConfig.getLateMessageRejectionPeriod().get()) + ) : Optional.absent()); + + Optional maximumMessageTime = (ioConfig.getEarlyMessageRejectionPeriod().isPresent() ? Optional.of( + DateTime.now().plus(ioConfig.getEarlyMessageRejectionPeriod().get()) + ) : Optional.absent()); + + try { + Map startingOffsets = generateStartingOffsetsForPartitionGroup(groupId); + + Map simpleStartingOffsets = startingOffsets + .entrySet().stream() + .filter(x -> x.getValue().get() != null) + .collect(Collectors.toMap(Map.Entry::getKey, x -> x.getValue().get())); + + Set exclusiveStartSequenceNumberPartitions = startingOffsets + .entrySet().stream() + .filter(x -> x.getValue().get() != null && x.getValue().isExclusive()) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + + taskGroups.put( + groupId, + new TaskGroup( + simpleStartingOffsets, + minimumMessageTime, + maximumMessageTime, + exclusiveStartSequenceNumberPartitions + ) + ); + } + catch (TimeoutException e) { + log.warn( + e, + "Timeout while fetching sequence numbers - if you are reading from the latest sequence number, you need to write events to the stream before the sequence number can be determined" + ); + } + } + } + + // iterate through all the current task groups and make sure each one has the desired number of replica tasks + boolean createdTask = false; + for (Map.Entry entry : taskGroups.entrySet()) { + TaskGroup taskGroup = entry.getValue(); + Integer groupId = entry.getKey(); + + if (taskGroup.partitionOffsets == null || taskGroup.partitionOffsets + .values().stream().allMatch(x -> x == null || Record.END_OF_SHARD_MARKER.equals(x))) { + log.debug("Nothing to read in any partition for taskGroup [%d], skipping task creation", groupId); + continue; + } + + if (ioConfig.getReplicas() > taskGroup.tasks.size()) { + log.info( + "Number of tasks [%d] does not match configured numReplicas [%d] in task group [%d], creating more tasks", + taskGroup.tasks.size(), ioConfig.getReplicas(), groupId + ); + createKinesisTasksForGroup(groupId, ioConfig.getReplicas() - taskGroup.tasks.size()); + createdTask = true; + } + } + + if (createdTask && firstRunTime.isBeforeNow()) { + // Schedule a run event after a short delay to update our internal data structures with the new tasks that were + // just created. This is mainly for the benefit of the status API in situations where the run period is lengthy. + scheduledExec.schedule(buildRunTask(), 5000, TimeUnit.MILLISECONDS); + } + } + + private void createKinesisTasksForGroup(int groupId, int replicas) + { + Map startPartitions = taskGroups.get(groupId).partitionOffsets; + Map endPartitions = new HashMap<>(); + Set exclusiveStartSequenceNumberPartitions = taskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; + for (String partition : startPartitions.keySet()) { + endPartitions.put(partition, KinesisPartitions.NO_END_SEQUENCE_NUMBER); + } + + String sequenceName = generateSequenceName(groupId); + + DateTime minimumMessageTime = taskGroups.get(groupId).minimumMessageTime.orNull(); + DateTime maximumMessageTime = taskGroups.get(groupId).maximumMessageTime.orNull(); + + KinesisIOConfig kinesisIOConfig = new KinesisIOConfig( + sequenceName, + new KinesisPartitions(ioConfig.getStream(), startPartitions), + new KinesisPartitions(ioConfig.getStream(), endPartitions), + true, + true, // should pause after reading otherwise the task may complete early which will confuse the supervisor + minimumMessageTime, + maximumMessageTime, + ioConfig.getEndpoint(), + ioConfig.getRecordsPerFetch(), + ioConfig.getFetchDelayMillis(), + ioConfig.getAwsAccessKeyId(), + ioConfig.getAwsSecretAccessKey(), + exclusiveStartSequenceNumberPartitions, + ioConfig.getAwsAssumedRoleArn(), + ioConfig.getAwsExternalId(), + ioConfig.isDeaggregate() + ); + + for (int i = 0; i < replicas; i++) { + String taskId = Joiner.on("_").join(sequenceName, getRandomId()); + KinesisIndexTask indexTask = new KinesisIndexTask( + taskId, + new TaskResource(sequenceName, 1), + spec.getDataSchema(), + taskTuningConfig, + kinesisIOConfig, + spec.getContext(), + null, + null, + rowIngestionMetersFactory + ); + + Optional taskQueue = taskMaster.getTaskQueue(); + if (taskQueue.isPresent()) { + try { + taskQueue.get().add(indexTask); + } + catch (EntryExistsException e) { + log.error("Tried to add task [%s] but it already exists", indexTask.getId()); + } + } else { + log.error("Failed to get task queue because I'm not the leader!"); + } + } + } + + private ImmutableMap generateStartingOffsetsForPartitionGroup(int groupId) + throws TimeoutException + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (Map.Entry entry : partitionGroups.get(groupId).entrySet()) { + String partition = entry.getKey(); + String offset = entry.getValue(); + + if (offset != null && !NOT_SET.equals(offset)) { + // if we are given a startingOffset (set by a previous task group which is pending completion) then use it + if (!Record.END_OF_SHARD_MARKER.equals(offset)) { + builder.put(partition, SequenceNumberPlus.of(offset, true)); + } + } else { + // if we don't have a startingOffset (first run or we had some previous failures and reset the offsets) then + // get the offset from metadata storage (if available) or Kafka (otherwise) + + SequenceNumberPlus offsetFromStorage = getOffsetFromStorageForPartition(partition); + + if (offsetFromStorage != null && !Record.END_OF_SHARD_MARKER.equals(offsetFromStorage.get())) { + builder.put(partition, offsetFromStorage); + } + } + } + return builder.build(); + } + + /** + * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't + * find any data, it will retrieve the latest or earliest Kinesis sequence number depending on the + * useEarliestSequenceNumber config. + */ + private SequenceNumberPlus getOffsetFromStorageForPartition(String partition) throws TimeoutException + { + Map metadataOffsets = getOffsetsFromMetadataStorage(); + String offset = metadataOffsets.get(partition); + + if (offset != null) { + log.debug("Getting sequence number [%s] from metadata storage for partition [%s]", offset, partition); + + if (!tuningConfig.isSkipSequenceNumberAvailabilityCheck()) { + final StreamPartition streamPartition = StreamPartition.of(ioConfig.getStream(), partition); + try { + String earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); + if (earliestSequenceNumber == null || earliestSequenceNumber.compareTo(offset) > 0) { + if (tuningConfig.isResetOffsetAutomatically()) { + resetInternal( + new KinesisDataSourceMetadata( + new KinesisPartitions(ioConfig.getStream(), ImmutableMap.of(partition, offset)) + ) + ); + throw new ISE( + "Previous sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) - automatically resetting offset", + offset, + partition, + earliestSequenceNumber + ); + + } else { + throw new ISE( + "Previous sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]). You can clear the previous sequenceNumber and start reading from a valid message by using the supervisor's reset API.", + offset, + partition, + earliestSequenceNumber + ); + } + } + } + catch (TimeoutException e) { + throw new ISE(e, "Timeout while fetching earliest sequence number for partition [%s]", partition); + } + } + + // Removed check comparing these offsets to the latest in Kinesis because it's potentially quite expensive - if no + // data has been written recently the call will block until timeout. + + return SequenceNumberPlus.of(offset, true); + + } else { + boolean useEarliestSequenceNumber = ioConfig.isUseEarliestSequenceNumber(); + if (subsequentlyDiscoveredPartitions.contains(partition)) { + log.info( + "Overriding useEarliestSequenceNumber and starting from beginning of newly discovered partition [%s] (which is probably from a split or merge)", + partition + ); + useEarliestSequenceNumber = true; + } + + offset = getOffsetFromKinesisForPartition(partition, useEarliestSequenceNumber); + log.info("Getting sequence number [%s] from Kinesis for partition [%s]", offset, partition); + return SequenceNumberPlus.of(offset, false); + } + } + + private Map getOffsetsFromMetadataStorage() + { + DataSourceMetadata dataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); + if (dataSourceMetadata != null && dataSourceMetadata instanceof KinesisDataSourceMetadata) { + KinesisPartitions partitions = ((KinesisDataSourceMetadata) dataSourceMetadata).getKinesisPartitions(); + if (partitions != null) { + if (!ioConfig.getStream().equals(partitions.getStream())) { + log.warn( + "Stream name in metadata storage [%s] doesn't match spec stream name [%s], ignoring stored sequence numbers", + partitions.getStream(), + ioConfig.getStream() + ); + return ImmutableMap.of(); + } else if (partitions.getPartitionSequenceNumberMap() != null) { + return partitions.getPartitionSequenceNumberMap(); + } + } + } + + return ImmutableMap.of(); + } + + private String getOffsetFromKinesisForPartition(String partition, boolean useEarliestSequenceNumber) + throws TimeoutException + { + log.info( + "Fetching starting sequence number from Kinesis with useEarliestSequenceNumber=%s", + useEarliestSequenceNumber + ? "true. If there is no data in the stream, Kinesis will not return a sequence number and this call will fail with a timeout." + : "false. If there is no new data coming into the stream, Kinesis will not return a sequence number and this call will fail with a timeout." + ); + + return useEarliestSequenceNumber + ? recordSupplier.getEarliestSequenceNumber(StreamPartition.of(ioConfig.getStream(), partition)) + : recordSupplier.getLatestSequenceNumber(StreamPartition.of(ioConfig.getStream(), partition)); + } + + /** + * Compares the sequence name from the task with one generated for the task's group ID and returns false if they do + * not match. The sequence name is generated from a hash of the dataSchema, tuningConfig, starting offsets, and the + * minimumMessageTime or maximumMessageTime if set. + */ + private boolean isTaskCurrent(int taskGroupId, String taskId) + { + Optional taskOptional = taskStorage.getTask(taskId); + if (!taskOptional.isPresent() || !(taskOptional.get() instanceof KinesisIndexTask)) { + return false; + } + + String taskSequenceName = ((KinesisIndexTask) taskOptional.get()).getIOConfig().getBaseSequenceName(); + + return generateSequenceName(taskGroupId).equals(taskSequenceName); + } + + private ListenableFuture stopTasksInGroup(TaskGroup taskGroup) + { + if (taskGroup == null) { + return Futures.immediateFuture(null); + } + + final List> futures = Lists.newArrayList(); + for (Map.Entry entry : taskGroup.tasks.entrySet()) { + if (!entry.getValue().status.isComplete()) { + futures.add(stopTask(entry.getKey(), false)); + } + } + + return Futures.successfulAsList(futures); + } + + private ListenableFuture stopTask(final String id, final boolean publish) + { + return Futures.transform( + taskClient.stopAsync(id, publish), new Function() + { + @Nullable + @Override + public Void apply(@Nullable Boolean result) + { + if (result == null || !result) { + log.info("Task [%s] failed to stop in a timely manner, killing task", id); + killTask(id); + } + return null; + } + } + ); + } + + private void killTask(final String id) + { + Optional taskQueue = taskMaster.getTaskQueue(); + if (taskQueue.isPresent()) { + taskQueue.get().shutdown(id); + } else { + log.error("Failed to get task queue because I'm not the leader!"); + } + } + + private int getTaskGroupIdForPartition(String partitionId) + { + if (!partitionIds.contains(partitionId)) { + partitionIds.add(partitionId); + } + + return partitionIds.indexOf(partitionId) % ioConfig.getTaskCount(); + } + + private boolean isTaskInPendingCompletionGroups(String taskId) + { + for (List taskGroups : pendingCompletionTaskGroups.values()) { + for (TaskGroup taskGroup : taskGroups) { + if (taskGroup.tasks.containsKey(taskId)) { + return true; + } + } + } + return false; + } + + private SupervisorReport generateReport(boolean includeOffsets) + { + int numPartitions = 0; + for (Map partitionGroup : partitionGroups.values()) { + numPartitions += partitionGroup.size(); + } + + KinesisSupervisorReportPayload payload = new KinesisSupervisorReportPayload( + dataSource, + ioConfig.getStream(), + numPartitions, + ioConfig.getReplicas(), + ioConfig.getTaskDuration().getMillis() / 1000, + spec.isSuspended() + ); + + SupervisorReport report = new SupervisorReport<>( + dataSource, + DateTimes.nowUtc(), + payload + ); + + List taskReports = Lists.newArrayList(); + List>> futures = Lists.newArrayList(); + + try { + for (TaskGroup taskGroup : taskGroups.values()) { + for (Map.Entry entry : taskGroup.tasks.entrySet()) { + String taskId = entry.getKey(); + DateTime startTime = entry.getValue().startTime; + Long remainingSeconds = null; + if (startTime != null) { + remainingSeconds = Math.max( + 0, ioConfig.getTaskDuration().getMillis() - (DateTime.now().getMillis() - startTime.getMillis()) + ) / 1000; + } + + taskReports.add( + new TaskReportData( + taskId, + (includeOffsets ? taskGroup.partitionOffsets : null), + null, + startTime, + remainingSeconds, + TaskReportData.TaskType.ACTIVE + ) + ); + + if (includeOffsets) { + futures.add(taskClient.getCurrentOffsetsAsync(taskId, false)); + } + } + } + + for (List taskGroups : pendingCompletionTaskGroups.values()) { + for (TaskGroup taskGroup : taskGroups) { + for (Map.Entry entry : taskGroup.tasks.entrySet()) { + String taskId = entry.getKey(); + DateTime startTime = entry.getValue().startTime; + Long remainingSeconds = null; + if (taskGroup.completionTimeout != null) { + remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - DateTime.now().getMillis()) + / 1000; + } + + taskReports.add( + new TaskReportData( + taskId, + (includeOffsets ? taskGroup.partitionOffsets : null), + null, + startTime, + remainingSeconds, + TaskReportData.TaskType.PUBLISHING + ) + ); + + if (includeOffsets) { + futures.add(taskClient.getCurrentOffsetsAsync(taskId, false)); + } + } + } + } + + List> results = Futures.successfulAsList(futures) + .get(futureTimeoutInSeconds, TimeUnit.SECONDS); + for (int i = 0; i < taskReports.size(); i++) { + TaskReportData reportData = taskReports.get(i); + if (includeOffsets) { + reportData.setCurrentSequenceNumbers(results.get(i)); + } + payload.addTask(reportData); + } + } + catch (Exception e) { + log.warn(e, "Failed to generate status report"); + } + + return report; + } + + private Runnable buildRunTask() + { + return new Runnable() + { + @Override + public void run() + { + notices.add(new RunNotice()); + } + }; + } + + /** + * Collect row ingestion stats from all tasks managed by this supervisor. + * + * @return A map of groupId->taskId->task row stats + * + * @throws InterruptedException + * @throws ExecutionException + * @throws TimeoutException + */ + private Map> getCurrentTotalStats() throws InterruptedException, ExecutionException, TimeoutException + { + Map> allStats = Maps.newHashMap(); + final List> futures = new ArrayList<>(); + final List> groupAndTaskIds = new ArrayList<>(); + + for (int groupId : taskGroups.keySet()) { + TaskGroup group = taskGroups.get(groupId); + for (String taskId : group.taskIds()) { + futures.add( + Futures.transform( + taskClient.getMovingAveragesAsync(taskId), + (Function, StatsFromTaskResult>) (currentStats) -> { + return new StatsFromTaskResult( + groupId, + taskId, + currentStats + ); + } + ) + ); + groupAndTaskIds.add(new Pair<>(groupId, taskId)); + } + } + + for (int groupId : pendingCompletionTaskGroups.keySet()) { + TaskGroup group = taskGroups.get(groupId); + for (String taskId : group.taskIds()) { + futures.add( + Futures.transform( + taskClient.getMovingAveragesAsync(taskId), + (Function, StatsFromTaskResult>) (currentStats) -> { + return new StatsFromTaskResult( + groupId, + taskId, + currentStats + ); + } + ) + ); + groupAndTaskIds.add(new Pair<>(groupId, taskId)); + } + } + + List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + for (int i = 0; i < results.size(); i++) { + StatsFromTaskResult result = results.get(i); + if (result != null) { + Map groupMap = allStats.computeIfAbsent(result.getGroupId(), k -> Maps.newHashMap()); + groupMap.put(result.getTaskId(), result.getStats()); + } else { + Pair groupAndTaskId = groupAndTaskIds.get(i); + log.error("Failed to get stats for group[%d]-task[%s]", groupAndTaskId.lhs, groupAndTaskId.rhs); + } + } + + return allStats; + } + + private static class StatsFromTaskResult + { + private final String groupId; + private final String taskId; + private final Map stats; + + public StatsFromTaskResult( + int groupId, + String taskId, + Map stats + ) + { + this.groupId = String.valueOf(groupId); + this.taskId = taskId; + this.stats = stats; + } + + public String getGroupId() + { + return groupId; + } + + public String getTaskId() + { + return taskId; + } + + public Map getStats() + { + return stats; + } + } + + +// TODO: Implement this for Kinesis which uses approximate time from latest instead of offset lag +/* + private Runnable computeAndEmitLag(final KinesisIndexTaskClient taskClient) + { + return new Runnable() + { + @Override + public void run() + { + try { + final Map> topics = lagComputingConsumer.listTopics(); + final List partitionInfoList = topics.get(ioConfig.getStream()); + lagComputingConsumer.assign( + Lists.transform(partitionInfoList, new Function() + { + @Override + public TopicPartition apply(PartitionInfo input) + { + return new TopicPartition(ioConfig.getStream(), input.partition()); + } + }) + ); + final Map offsetsResponse = new ConcurrentHashMap<>(); + final List> futures = Lists.newArrayList(); + for (TaskGroup taskGroup : taskGroups.values()) { + for (String taskId : taskGroup.taskIds()) { + futures.add(Futures.transform( + taskClient.getCurrentOffsetsAsync(taskId, false), + new Function, Void>() + { + @Override + public Void apply(Map taskResponse) + { + if (taskResponse != null) { + for (final Map.Entry partitionOffsets : taskResponse.entrySet()) { + offsetsResponse.compute(partitionOffsets.getKey(), new BiFunction() + { + @Override + public Long apply(Integer key, Long existingOffsetInMap) + { + // If existing value is null use the offset returned by task + // otherwise use the max (makes sure max offset is taken from replicas) + return existingOffsetInMap == null + ? partitionOffsets.getValue() + : Math.max(partitionOffsets.getValue(), existingOffsetInMap); + } + }); + } + } + return null; + } + } + ) + ); + } + } + // not using futureTimeoutInSeconds as its min value is 120 seconds + // and minimum emission period for this metric is 60 seconds + Futures.successfulAsList(futures).get(30, TimeUnit.SECONDS); + + // for each partition, seek to end to get the highest offset + // check the offsetsResponse map for the latest consumed offset + // if partition info not present in offsetsResponse then use lastCurrentOffsets map + // if not present there as well, fail the compute + + long lag = 0; + for (PartitionInfo partitionInfo : partitionInfoList) { + long diff; + final TopicPartition topicPartition = new TopicPartition(ioConfig.getStream(), partitionInfo.partition()); + lagComputingConsumer.seekToEnd(ImmutableList.of(topicPartition)); + if (offsetsResponse.get(topicPartition.partition()) != null) { + diff = lagComputingConsumer.position(topicPartition) - offsetsResponse.get(topicPartition.partition()); + lastCurrentOffsets.put(topicPartition.partition(), offsetsResponse.get(topicPartition.partition())); + } else if (lastCurrentOffsets.get(topicPartition.partition()) != null) { + diff = lagComputingConsumer.position(topicPartition) - lastCurrentOffsets.get(topicPartition.partition()); + } else { + throw new ISE("Could not find latest consumed offset for partition [%d]", topicPartition.partition()); + } + lag += diff; + log.debug( + "Topic - [%s] Partition - [%d] : Partition lag [%,d], Total lag so far [%,d]", + topicPartition.topic(), + topicPartition.partition(), + diff, + lag + ); + } + emitter.emit( + ServiceMetricEvent.builder().setDimension("dataSource", dataSource).build("ingest/kinesis/lag", lag) + ); + } + catch (InterruptedException e) { + // do nothing, probably we are shutting down + } + catch (Exception e) { + log.warn(e, "Unable to compute Kinesis lag"); + } + } + }; + } + */ +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java new file mode 100644 index 000000000000..3937d00aca14 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java @@ -0,0 +1,249 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis.supervisor; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import org.apache.druid.indexing.kinesis.KinesisRegion; +import org.joda.time.Duration; +import org.joda.time.Period; + +public class KinesisSupervisorIOConfig +{ + private final String stream; + private final String endpoint; + private final Integer replicas; + private final Integer taskCount; + private final Duration taskDuration; + private final Duration startDelay; + private final Duration period; + private final boolean useEarliestSequenceNumber; + private final Duration completionTimeout; + private final Optional lateMessageRejectionPeriod; + private final Optional earlyMessageRejectionPeriod; + + // In determining a suitable value for recordsPerFetch: + // - Each data record can be up to 1 MB in size + // - Each shard can read up to 2 MB per second + // - The maximum size of data that GetRecords can return is 10 MB. If a call returns this amount of data, + // subsequent calls made within the next 5 seconds throw ProvisionedThroughputExceededException. + // + // If there is insufficient provisioned throughput on the shard, subsequent calls made within the next 1 second + // throw ProvisionedThroughputExceededException. Note that GetRecords won't return any data when it throws an + // exception. For this reason, we recommend that you wait one second between calls to GetRecords; however, it's + // possible that the application will get exceptions for longer than 1 second. + private final Integer recordsPerFetch; + private final Integer fetchDelayMillis; + + private final String awsAccessKeyId; + private final String awsSecretAccessKey; + private final String awsAssumedRoleArn; + private final String awsExternalId; + private final boolean deaggregate; + + @JsonCreator + public KinesisSupervisorIOConfig( + @JsonProperty("stream") String stream, + @JsonProperty("endpoint") String endpoint, + @JsonProperty("region") KinesisRegion region, + @JsonProperty("replicas") Integer replicas, + @JsonProperty("taskCount") Integer taskCount, + @JsonProperty("taskDuration") Period taskDuration, + @JsonProperty("startDelay") Period startDelay, + @JsonProperty("period") Period period, + @JsonProperty("useEarliestSequenceNumber") Boolean useEarliestSequenceNumber, + @JsonProperty("completionTimeout") Period completionTimeout, + @JsonProperty("lateMessageRejectionPeriod") Period lateMessageRejectionPeriod, + @JsonProperty("earlyMessageRejectionPeriod") Period earlyMessageRejectionPeriod, + @JsonProperty("recordsPerFetch") Integer recordsPerFetch, + @JsonProperty("fetchDelayMillis") Integer fetchDelayMillis, + @JsonProperty("awsAccessKeyId") String awsAccessKeyId, + @JsonProperty("awsSecretAccessKey") String awsSecretAccessKey, + @JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn, + @JsonProperty("awsExternalId") String awsExternalId, + @JsonProperty("deaggregate") boolean deaggregate + ) + { + this.stream = Preconditions.checkNotNull(stream, "stream cannot be null"); + this.endpoint = endpoint != null + ? endpoint + : (region != null ? region.getEndpoint() : KinesisRegion.US_EAST_1.getEndpoint()); + this.replicas = replicas != null ? replicas : 1; + this.taskCount = taskCount != null ? taskCount : 1; + this.taskDuration = defaultDuration(taskDuration, "PT1H"); + this.startDelay = defaultDuration(startDelay, "PT5S"); + this.period = defaultDuration(period, "PT30S"); + this.useEarliestSequenceNumber = useEarliestSequenceNumber != null ? useEarliestSequenceNumber : false; + this.completionTimeout = defaultDuration(completionTimeout, "PT6H"); + this.lateMessageRejectionPeriod = lateMessageRejectionPeriod == null + ? Optional.absent() + : Optional.of(lateMessageRejectionPeriod.toStandardDuration()); + this.earlyMessageRejectionPeriod = earlyMessageRejectionPeriod == null + ? Optional.absent() + : Optional.of(earlyMessageRejectionPeriod.toStandardDuration()); + this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : 4000; + this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : 0; + this.awsAccessKeyId = awsAccessKeyId; + this.awsSecretAccessKey = awsSecretAccessKey; + this.awsAssumedRoleArn = awsAssumedRoleArn; + this.awsExternalId = awsExternalId; + this.deaggregate = deaggregate; + } + + @JsonProperty + public String getStream() + { + return stream; + } + + @JsonProperty + public String getEndpoint() + { + return endpoint; + } + + @JsonProperty + public Integer getReplicas() + { + return replicas; + } + + @JsonProperty + public Integer getTaskCount() + { + return taskCount; + } + + @JsonProperty + public Duration getTaskDuration() + { + return taskDuration; + } + + @JsonProperty + public Duration getStartDelay() + { + return startDelay; + } + + @JsonProperty + public Duration getPeriod() + { + return period; + } + + @JsonProperty + public boolean isUseEarliestSequenceNumber() + { + return useEarliestSequenceNumber; + } + + @JsonProperty + public Duration getCompletionTimeout() + { + return completionTimeout; + } + + @JsonProperty + public Optional getLateMessageRejectionPeriod() + { + return lateMessageRejectionPeriod; + } + + @JsonProperty + public Optional getEarlyMessageRejectionPeriod() + { + return earlyMessageRejectionPeriod; + } + + @JsonProperty + public Integer getRecordsPerFetch() + { + return recordsPerFetch; + } + + @JsonProperty + public Integer getFetchDelayMillis() + { + return fetchDelayMillis; + } + + @JsonProperty + public String getAwsAccessKeyId() + { + return awsAccessKeyId; + } + + @JsonProperty + public String getAwsSecretAccessKey() + { + return awsSecretAccessKey; + } + + @JsonProperty + public String getAwsAssumedRoleArn() + { + return awsAssumedRoleArn; + } + + @JsonProperty + public String getAwsExternalId() + { + return awsExternalId; + } + + @JsonProperty + public boolean isDeaggregate() + { + return deaggregate; + } + + @Override + public String toString() + { + return "KinesisSupervisorIOConfig{" + + "stream='" + stream + '\'' + + ", endpoint='" + endpoint + '\'' + + ", replicas=" + replicas + + ", taskCount=" + taskCount + + ", taskDuration=" + taskDuration + + ", startDelay=" + startDelay + + ", period=" + period + + ", useEarliestSequenceNumber=" + useEarliestSequenceNumber + + ", completionTimeout=" + completionTimeout + + ", lateMessageRejectionPeriod=" + lateMessageRejectionPeriod + + ", earlyMessageRejectionPeriod=" + earlyMessageRejectionPeriod + + ", recordsPerFetch=" + recordsPerFetch + + ", fetchDelayMillis=" + fetchDelayMillis + + ", awsAccessKeyId='" + awsAccessKeyId + '\'' + + ", awsSecretAccessKey=" + "************************" + + ", awsAssumedRoleArn='" + awsAssumedRoleArn + '\'' + + ", awsExternalId='" + awsExternalId + '\'' + + ", deaggregate=" + deaggregate + + '}'; + } + + private static Duration defaultDuration(final Period period, final String theDefault) + { + return (period == null ? new Period(theDefault) : period).toStandardDuration(); + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java new file mode 100644 index 000000000000..d539b460f96f --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java @@ -0,0 +1,112 @@ +package org.apache.druid.indexing.kinesis.supervisor; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Lists; +import org.apache.druid.java.util.common.IAE; + +import java.util.List; + +public class KinesisSupervisorReportPayload +{ + private final String dataSource; + private final String stream; + private final Integer partitions; + private final Integer replicas; + private final Long durationSeconds; + private final List activeTasks; + private final List publishingTasks; + private final boolean suspended; + + public KinesisSupervisorReportPayload( + String dataSource, + String stream, + Integer partitions, + Integer replicas, + Long durationSeconds, + boolean suspended + ) + { + this.dataSource = dataSource; + this.stream = stream; + this.partitions = partitions; + this.replicas = replicas; + this.durationSeconds = durationSeconds; + this.activeTasks = Lists.newArrayList(); + this.publishingTasks = Lists.newArrayList(); + this.suspended = suspended; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public String getStream() + { + return stream; + } + + @JsonProperty + public Integer getPartitions() + { + return partitions; + } + + @JsonProperty + public Integer getReplicas() + { + return replicas; + } + + @JsonProperty + public Long getDurationSeconds() + { + return durationSeconds; + } + + @JsonProperty + public List getActiveTasks() + { + return activeTasks; + } + + @JsonProperty + public List getPublishingTasks() + { + return publishingTasks; + } + + @JsonProperty + public boolean getSuspended() + { + return suspended; + } + + @Override + public String toString() + { + return "{" + + "dataSource='" + dataSource + '\'' + + ", stream='" + stream + '\'' + + ", partitions=" + partitions + + ", replicas=" + replicas + + ", durationSeconds=" + durationSeconds + + ", active=" + activeTasks + + ", publishing=" + publishingTasks + + ", suspended=" + suspended + + '}'; + } + + public void addTask(TaskReportData data) + { + if (data.getType().equals(TaskReportData.TaskType.ACTIVE)) { + activeTasks.add(data); + } else if (data.getType().equals(TaskReportData.TaskType.PUBLISHING)) { + publishingTasks.add(data); + } else { + throw new IAE("Unknown task type [%s]", data.getType().name()); + } + } +} \ No newline at end of file diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java new file mode 100644 index 000000000000..b89748ff1f0b --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -0,0 +1,230 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis.supervisor; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.Supervisor; +import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; + +import java.util.List; +import java.util.Map; + +public class KinesisSupervisorSpec implements SupervisorSpec +{ + private final DataSchema dataSchema; + private final KinesisSupervisorTuningConfig tuningConfig; + private final KinesisSupervisorIOConfig ioConfig; + private final Map context; + private final boolean suspended; + + private final TaskStorage taskStorage; + private final TaskMaster taskMaster; + private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; + private final KinesisIndexTaskClientFactory kinesisIndexTaskClientFactory; + private final ObjectMapper mapper; + private final ServiceEmitter emitter; + private final DruidMonitorSchedulerConfig monitorSchedulerConfig; + private final RowIngestionMetersFactory rowIngestionMetersFactory; + + @JsonCreator + public KinesisSupervisorSpec( + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("tuningConfig") KinesisSupervisorTuningConfig tuningConfig, + @JsonProperty("ioConfig") KinesisSupervisorIOConfig ioConfig, + @JsonProperty("context") Map context, + @JsonProperty("suspended") Boolean suspended, + @JacksonInject TaskStorage taskStorage, + @JacksonInject TaskMaster taskMaster, + @JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + @JacksonInject KinesisIndexTaskClientFactory kinesisIndexTaskClientFactory, + @JacksonInject @Json ObjectMapper mapper, + @JacksonInject ServiceEmitter emitter, + @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + ) + { + this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); + this.tuningConfig = tuningConfig != null + ? tuningConfig + : new KinesisSupervisorTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); + this.context = context; + this.suspended = suspended != null ? suspended : false; + + this.taskStorage = taskStorage; + this.taskMaster = taskMaster; + this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; + this.kinesisIndexTaskClientFactory = kinesisIndexTaskClientFactory; + this.mapper = mapper; + this.emitter = emitter; + this.monitorSchedulerConfig = monitorSchedulerConfig; + this.rowIngestionMetersFactory = rowIngestionMetersFactory; + } + + @JsonProperty + public DataSchema getDataSchema() + { + return dataSchema; + } + + @JsonProperty + public KinesisSupervisorTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @JsonProperty + public KinesisSupervisorIOConfig getIoConfig() + { + return ioConfig; + } + + @JsonProperty + public Map getContext() + { + return context; + } + + @Override + @JsonProperty("suspended") + public boolean isSuspended() + { + return suspended; + } + + public ServiceEmitter getEmitter() + { + return emitter; + } + + @Override + public String getId() + { + return dataSchema.getDataSource(); + } + + public DruidMonitorSchedulerConfig getMonitorSchedulerConfig() + { + return monitorSchedulerConfig; + } + + @Override + public Supervisor createSupervisor() + { + return new KinesisSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + kinesisIndexTaskClientFactory, + mapper, + this, + rowIngestionMetersFactory + ); + } + + @Override + public List getDataSources() + { + return ImmutableList.of(getDataSchema().getDataSource()); + } + + @Override + public String toString() + { + return "KinesisSupervisorSpec{" + + "dataSchema=" + dataSchema + + ", tuningConfig=" + tuningConfig + + ", ioConfig=" + ioConfig + + ", suspended=" + suspended + + '}'; + } + + @Override + public KinesisSupervisorSpec createSuspendedSpec() + { + return toggleSuspend(true); + } + + @Override + public KinesisSupervisorSpec createRunningSpec() + { + return toggleSuspend(false); + } + + private KinesisSupervisorSpec toggleSuspend(boolean suspend) + { + return new KinesisSupervisorSpec( + dataSchema, + tuningConfig, + ioConfig, + context, + suspend, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + kinesisIndexTaskClientFactory, + mapper, + emitter, + monitorSchedulerConfig, + rowIngestionMetersFactory + ); + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java new file mode 100644 index 000000000000..54e1fd67f945 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -0,0 +1,163 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis.supervisor; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.kinesis.KinesisTuningConfig; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.joda.time.Duration; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import java.io.File; + +public class KinesisSupervisorTuningConfig extends KinesisTuningConfig +{ + private final Integer workerThreads; + private final Integer chatThreads; + private final Long chatRetries; + private final Duration httpTimeout; + private final Duration shutdownTimeout; + + public KinesisSupervisorTuningConfig( + @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, + @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, + @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, + @JsonProperty("basePersistDirectory") File basePersistDirectory, + @JsonProperty("maxPendingPersists") Integer maxPendingPersists, + @JsonProperty("indexSpec") IndexSpec indexSpec, + @JsonProperty("buildV9Directly") Boolean buildV9Directly, + @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, + @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, + @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, + @JsonProperty("skipSequenceNumberAvailabilityCheck") Boolean skipSequenceNumberAvailabilityCheck, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("workerThreads") Integer workerThreads, + @JsonProperty("chatThreads") Integer chatThreads, + @JsonProperty("chatRetries") Long chatRetries, + @JsonProperty("httpTimeout") Period httpTimeout, + @JsonProperty("shutdownTimeout") Period shutdownTimeout, + @JsonProperty("recordBufferSize") Integer recordBufferSize, + @JsonProperty("recordBufferOfferTimeout") Integer recordBufferOfferTimeout, + @JsonProperty("recordBufferFullWait") Integer recordBufferFullWait, + @JsonProperty("fetchSequenceNumberTimeout") Integer fetchSequenceNumberTimeout, + @JsonProperty("fetchThreads") Integer fetchThreads, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions + ) + { + super( + maxRowsInMemory, + maxBytesInMemory, + maxRowsPerSegment, + intermediatePersistPeriod, + basePersistDirectory, + maxPendingPersists, + indexSpec, + buildV9Directly, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + skipSequenceNumberAvailabilityCheck, + recordBufferSize, + recordBufferOfferTimeout, + recordBufferFullWait, + fetchSequenceNumberTimeout, + fetchThreads, + segmentWriteOutMediumFactory, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions + ); + + this.workerThreads = workerThreads; + this.chatThreads = chatThreads; + this.chatRetries = (chatRetries != null ? chatRetries : 8); + this.httpTimeout = defaultDuration(httpTimeout, "PT10S"); + this.shutdownTimeout = defaultDuration(shutdownTimeout, "PT80S"); + } + + @JsonProperty + public Integer getWorkerThreads() + { + return workerThreads; + } + + @JsonProperty + public Integer getChatThreads() + { + return chatThreads; + } + + @JsonProperty + public Long getChatRetries() + { + return chatRetries; + } + + @JsonProperty + public Duration getHttpTimeout() + { + return httpTimeout; + } + + @JsonProperty + public Duration getShutdownTimeout() + { + return shutdownTimeout; + } + + @Override + public String toString() + { + return "KinesisSupervisorTuningConfig{" + + "maxRowsInMemory=" + getMaxRowsInMemory() + + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + + ", basePersistDirectory=" + getBasePersistDirectory() + + ", maxPendingPersists=" + getMaxPendingPersists() + + ", indexSpec=" + getIndexSpec() + + ", buildV9Directly=" + getBuildV9Directly() + + ", reportParseExceptions=" + isReportParseExceptions() + + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + + ", skipSequenceNumberAvailabilityCheck=" + isSkipSequenceNumberAvailabilityCheck() + + ", workerThreads=" + workerThreads + + ", chatThreads=" + chatThreads + + ", chatRetries=" + chatRetries + + ", httpTimeout=" + httpTimeout + + ", shutdownTimeout=" + shutdownTimeout + + ", recordBufferSize=" + getRecordBufferSize() + + ", recordBufferOfferTimeout=" + getRecordBufferOfferTimeout() + + ", recordBufferFullWait=" + getRecordBufferFullWait() + + ", fetchSequenceNumberTimeout=" + getFetchSequenceNumberTimeout() + + ", fetchThreads=" + getFetchThreads() + + ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() + + '}'; + } + + private static Duration defaultDuration(final Period period, final String theDefault) + { + return (period == null ? new Period(theDefault) : period).toStandardDuration(); + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/TaskReportData.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/TaskReportData.java new file mode 100644 index 000000000000..d1cae2834d86 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/TaskReportData.java @@ -0,0 +1,110 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis.supervisor; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.DateTime; + +import java.util.Map; + +public class TaskReportData +{ + public enum TaskType + { + ACTIVE, PUBLISHING, UNKNOWN + } + + private final String id; + private final Map startingSequenceNumbers; + private final DateTime startTime; + private final Long remainingSeconds; + private final TaskType type; + private Map currentSequenceNumbers; + + public TaskReportData( + String id, + Map startingSequenceNumbers, + Map currentSequenceNumbers, + DateTime startTime, + Long remainingSeconds, + TaskType type + ) + { + this.id = id; + this.startingSequenceNumbers = startingSequenceNumbers; + this.currentSequenceNumbers = currentSequenceNumbers; + this.startTime = startTime; + this.remainingSeconds = remainingSeconds; + this.type = type; + } + + @JsonProperty + public String getId() + { + return id; + } + + @JsonProperty + public Map getStartingSequenceNumbers() + { + return startingSequenceNumbers; + } + + @JsonProperty + public Map getCurrentSequenceNumbers() + { + return currentSequenceNumbers; + } + + public void setCurrentSequenceNumbers(Map currentSequenceNumbers) + { + this.currentSequenceNumbers = currentSequenceNumbers; + } + + @JsonProperty + public DateTime getStartTime() + { + return startTime; + } + + @JsonProperty + public Long getRemainingSeconds() + { + return remainingSeconds; + } + + @JsonProperty + public TaskType getType() + { + return type; + } + + @Override + public String toString() + { + return "{" + + "id='" + id + '\'' + + (startingSequenceNumbers != null ? ", startingSequenceNumbers=" + startingSequenceNumbers : "") + + (currentSequenceNumbers != null ? ", currentSequenceNumbers=" + currentSequenceNumbers : "") + + ", startTime=" + startTime + + ", remainingSeconds=" + remainingSeconds + + '}'; + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule new file mode 100644 index 000000000000..5665f7ce1d9c --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -0,0 +1 @@ +org.apache.druid.indexing.kinesis.KinesisIndexingServiceModule diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java new file mode 100644 index 000000000000..52d40185a118 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java @@ -0,0 +1,130 @@ +///* +// * Licensed to Metamarkets Group Inc. (Metamarkets) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. Metamarkets licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, +// * software distributed under the License is distributed on an +// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// * KIND, either express or implied. See the License for the +// * specific language governing permissions and limitations +// * under the License. +// */ +// +//package org.apache.druid.indexing.kinesis; +// +//import com.google.common.collect.ImmutableMap; +//import org.junit.Assert; +//import org.junit.Test; +// +//import java.util.Map; +// +//public class KinesisDataSourceMetadataTest +//{ +// private static final KinesisDataSourceMetadata KM0 = KM("foo", ImmutableMap.of()); +// private static final KinesisDataSourceMetadata KM1 = KM("foo", ImmutableMap.of(0, 2L, 1, 3L)); +// private static final KinesisDataSourceMetadata KM2 = KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)); +// private static final KinesisDataSourceMetadata KM3 = KM("foo", ImmutableMap.of(0, 2L, 2, 5L)); +// +// @Test +// public void testMatches() +// { +// Assert.assertTrue(KM0.matches(KM0)); +// Assert.assertTrue(KM0.matches(KM1)); +// Assert.assertTrue(KM0.matches(KM2)); +// Assert.assertTrue(KM0.matches(KM3)); +// +// Assert.assertTrue(KM1.matches(KM0)); +// Assert.assertTrue(KM1.matches(KM1)); +// Assert.assertFalse(KM1.matches(KM2)); +// Assert.assertTrue(KM1.matches(KM3)); +// +// Assert.assertTrue(KM2.matches(KM0)); +// Assert.assertFalse(KM2.matches(KM1)); +// Assert.assertTrue(KM2.matches(KM2)); +// Assert.assertTrue(KM2.matches(KM3)); +// +// Assert.assertTrue(KM3.matches(KM0)); +// Assert.assertTrue(KM3.matches(KM1)); +// Assert.assertTrue(KM3.matches(KM2)); +// Assert.assertTrue(KM3.matches(KM3)); +// } +// +// @Test +// public void testIsValidStart() +// { +// Assert.assertTrue(KM0.isValidStart()); +// Assert.assertTrue(KM1.isValidStart()); +// Assert.assertTrue(KM2.isValidStart()); +// Assert.assertTrue(KM3.isValidStart()); +// } +// +// @Test +// public void testPlus() +// { +// Assert.assertEquals( +// KM("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)), +// KM1.plus(KM3) +// ); +// +// Assert.assertEquals( +// KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)), +// KM0.plus(KM2) +// ); +// +// Assert.assertEquals( +// KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)), +// KM1.plus(KM2) +// ); +// +// Assert.assertEquals( +// KM("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)), +// KM2.plus(KM1) +// ); +// +// Assert.assertEquals( +// KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)), +// KM2.plus(KM2) +// ); +// } +// +// @Test +// public void testMinus() +// { +// Assert.assertEquals( +// KM("foo", ImmutableMap.of(1, 3L)), +// KM1.minus(KM3) +// ); +// +// Assert.assertEquals( +// KM("foo", ImmutableMap.of()), +// KM0.minus(KM2) +// ); +// +// Assert.assertEquals( +// KM("foo", ImmutableMap.of()), +// KM1.minus(KM2) +// ); +// +// Assert.assertEquals( +// KM("foo", ImmutableMap.of(2, 5L)), +// KM2.minus(KM1) +// ); +// +// Assert.assertEquals( +// KM("foo", ImmutableMap.of()), +// KM2.minus(KM2) +// ); +// } +// +// private static KinesisDataSourceMetadata KM(String topic, Map offsets) +// { +// return new KinesisDataSourceMetadata(new KinesisPartitions(topic, offsets)); +// } +//} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java new file mode 100644 index 000000000000..25435f590095 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -0,0 +1,251 @@ +///* +// * Licensed to Metamarkets Group Inc. (Metamarkets) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. Metamarkets licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, +// * software distributed under the License is distributed on an +// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// * KIND, either express or implied. See the License for the +// * specific language governing permissions and limitations +// * under the License. +// */ +// +//package org.apache.druid.indexing.kinesis; +// +//import com.fasterxml.jackson.databind.JsonMappingException; +//import com.fasterxml.jackson.databind.Module; +//import com.fasterxml.jackson.databind.ObjectMapper; +//import com.google.common.collect.ImmutableMap; +//import org.apache.druid.jackson.DefaultObjectMapper; +//import org.apache.druid.segment.indexing.IOConfig; +//import org.hamcrest.CoreMatchers; +//import org.joda.time.DateTime; +//import org.junit.Assert; +//import org.junit.Rule; +//import org.junit.Test; +//import org.junit.rules.ExpectedException; +// +//public class KinesisIOConfigTest +//{ +// private final ObjectMapper mapper; +// +// public KinesisIOConfigTest() +// { +// mapper = new DefaultObjectMapper(); +// mapper.registerModules((Iterable) new KinesisIndexingServiceModule().getJacksonModules()); +// } +// +// @Rule +// public final ExpectedException exception = ExpectedException.none(); +// +// @Test +// public void testSerdeWithDefaults() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"baseSequenceName\": \"my-sequence-name\",\n" +// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" +// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n" +// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"}\n" +// + "}"; +// +// KinesisIOConfig config = (KinesisIOConfig) mapper.readValue( +// mapper.writeValueAsString( +// mapper.readValue( +// jsonStr, +// IOConfig.class +// ) +// ), IOConfig.class +// ); +// +// Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); +// Assert.assertEquals("mytopic", config.getStartPartitions().getStream()); +// Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartPartitions().getPartitionSequenceNumberMap()); +// Assert.assertEquals("mytopic", config.getEndPartitions().getStream()); +// Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndPartitions().getPartitionSequenceNumberMap()); +// Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties()); +// Assert.assertEquals(true, config.isUseTransaction()); +// Assert.assertEquals(false, config.isPauseAfterRead()); +// Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent()); +// Assert.assertFalse("skipOffsetGaps", config.isSkipOffsetGaps()); +// } +// +// @Test +// public void testSerdeWithNonDefaults() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"baseSequenceName\": \"my-sequence-name\",\n" +// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" +// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n" +// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" +// + " \"useTransaction\": false,\n" +// + " \"pauseAfterRead\": true,\n" +// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\",\n" +// + " \"skipOffsetGaps\": true\n" +// + "}"; +// +// KinesisIOConfig config = (KinesisIOConfig) mapper.readValue( +// mapper.writeValueAsString( +// mapper.readValue( +// jsonStr, +// IOConfig.class +// ) +// ), IOConfig.class +// ); +// +// Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); +// Assert.assertEquals("mytopic", config.getStartPartitions().getStream()); +// Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartPartitions().getPartitionSequenceNumberMap()); +// Assert.assertEquals("mytopic", config.getEndPartitions().getStream()); +// Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndPartitions().getPartitionSequenceNumberMap()); +// Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties()); +// Assert.assertEquals(false, config.isUseTransaction()); +// Assert.assertEquals(true, config.isPauseAfterRead()); +// Assert.assertEquals(new DateTime("2016-05-31T12:00Z"), config.getMinimumMessageTime().get()); +// Assert.assertTrue("skipOffsetGaps", config.isSkipOffsetGaps()); +// } +// +// @Test +// public void testBaseSequenceNameRequired() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" +// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n" +// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" +// + " \"useTransaction\": false,\n" +// + " \"pauseAfterRead\": true,\n" +// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" +// + "}"; +// +// exception.expect(JsonMappingException.class); +// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); +// exception.expectMessage(CoreMatchers.containsString("baseSequenceName")); +// mapper.readValue(jsonStr, IOConfig.class); +// } +// +// @Test +// public void testStartPartitionsRequired() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"baseSequenceName\": \"my-sequence-name\",\n" +// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n" +// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" +// + " \"useTransaction\": false,\n" +// + " \"pauseAfterRead\": true,\n" +// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" +// + "}"; +// +// exception.expect(JsonMappingException.class); +// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); +// exception.expectMessage(CoreMatchers.containsString("startPartitions")); +// mapper.readValue(jsonStr, IOConfig.class); +// } +// +// @Test +// public void testEndPartitionsRequired() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"baseSequenceName\": \"my-sequence-name\",\n" +// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" +// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" +// + " \"useTransaction\": false,\n" +// + " \"pauseAfterRead\": true,\n" +// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" +// + "}"; +// +// exception.expect(JsonMappingException.class); +// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); +// exception.expectMessage(CoreMatchers.containsString("endPartitions")); +// mapper.readValue(jsonStr, IOConfig.class); +// } +// +// @Test +// public void testConsumerPropertiesRequired() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"baseSequenceName\": \"my-sequence-name\",\n" +// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" +// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n" +// + " \"useTransaction\": false,\n" +// + " \"pauseAfterRead\": true,\n" +// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" +// + "}"; +// +// exception.expect(JsonMappingException.class); +// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); +// exception.expectMessage(CoreMatchers.containsString("consumerProperties")); +// mapper.readValue(jsonStr, IOConfig.class); +// } +// +// @Test +// public void testStartAndEndTopicMatch() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"baseSequenceName\": \"my-sequence-name\",\n" +// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" +// + " \"endPartitions\": {\"topic\":\"other\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n" +// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" +// + " \"useTransaction\": false,\n" +// + " \"pauseAfterRead\": true,\n" +// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" +// + "}"; +// +// exception.expect(JsonMappingException.class); +// exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class)); +// exception.expectMessage(CoreMatchers.containsString("start topic and end topic must match")); +// mapper.readValue(jsonStr, IOConfig.class); +// } +// +// @Test +// public void testStartAndEndPartitionSetMatch() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"baseSequenceName\": \"my-sequence-name\",\n" +// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" +// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15}},\n" +// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" +// + " \"useTransaction\": false,\n" +// + " \"pauseAfterRead\": true,\n" +// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" +// + "}"; +// +// exception.expect(JsonMappingException.class); +// exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class)); +// exception.expectMessage(CoreMatchers.containsString("start partition set and end partition set must match")); +// mapper.readValue(jsonStr, IOConfig.class); +// } +// +// @Test +// public void testEndOffsetGreaterThanStart() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"baseSequenceName\": \"my-sequence-name\",\n" +// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" +// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":2}},\n" +// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" +// + " \"useTransaction\": false,\n" +// + " \"pauseAfterRead\": true,\n" +// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" +// + "}"; +// +// exception.expect(JsonMappingException.class); +// exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class)); +// exception.expectMessage(CoreMatchers.containsString("end offset must be >= start offset")); +// mapper.readValue(jsonStr, IOConfig.class); +// } +//} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java new file mode 100644 index 000000000000..b7d267d9371c --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java @@ -0,0 +1,985 @@ +///* +// * Licensed to Metamarkets Group Inc. (Metamarkets) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. Metamarkets licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, +// * software distributed under the License is distributed on an +// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// * KIND, either express or implied. See the License for the +// * specific language governing permissions and limitations +// * under the License. +// */ +// +//package org.apache.druid.indexing.kinesis; +// +//import com.fasterxml.jackson.databind.ObjectMapper; +//import com.google.common.base.Optional; +//import com.google.common.collect.ImmutableList; +//import com.google.common.collect.ImmutableMap; +//import com.google.common.collect.Lists; +//import com.google.common.collect.Maps; +//import com.google.common.util.concurrent.Futures; +//import com.google.common.util.concurrent.ListenableFuture; +//import com.metamx.http.client.HttpClient; +//import com.metamx.http.client.Request; +//import com.metamx.http.client.response.FullResponseHandler; +//import com.metamx.http.client.response.FullResponseHolder; +//import org.apache.druid.indexing.common.TaskInfoProvider; +//import org.apache.druid.indexing.common.TaskLocation; +//import org.apache.druid.indexing.common.TaskStatus; +//import org.apache.druid.jackson.DefaultObjectMapper; +//import org.apache.druid.java.util.common.IAE; +//import org.easymock.Capture; +//import org.easymock.CaptureType; +//import org.easymock.EasyMockSupport; +//import org.jboss.netty.handler.codec.http.HttpHeaders; +//import org.jboss.netty.handler.codec.http.HttpMethod; +//import org.jboss.netty.handler.codec.http.HttpResponse; +//import org.jboss.netty.handler.codec.http.HttpResponseStatus; +//import org.joda.time.DateTime; +//import org.joda.time.Duration; +//import org.junit.After; +//import org.junit.Assert; +//import org.junit.Before; +//import org.junit.Test; +//import org.junit.runner.RunWith; +//import org.junit.runners.Parameterized; +// +//import java.io.IOException; +//import java.net.URL; +//import java.util.List; +//import java.util.Map; +// +//import static org.easymock.EasyMock.anyObject; +//import static org.easymock.EasyMock.capture; +//import static org.easymock.EasyMock.eq; +//import static org.easymock.EasyMock.expect; +//import static org.easymock.EasyMock.reset; +// +//@RunWith(Parameterized.class) +//public class KinesisIndexTaskClientTest extends EasyMockSupport +//{ +// private static final ObjectMapper objectMapper = new DefaultObjectMapper(); +// private static final String TEST_ID = "test-id"; +// private static final List TEST_IDS = Lists.newArrayList("test-id1", "test-id2", "test-id3", "test-id4"); +// private static final String TEST_HOST = "test-host"; +// private static final int TEST_PORT = 1234; +// private static final String TEST_DATASOURCE = "test-datasource"; +// private static final Duration TEST_HTTP_TIMEOUT = new Duration(5000); +// private static final long TEST_NUM_RETRIES = 0; +// private static final String URL_FORMATTER = "http://%s:%d/druid/worker/v1/chat/%s/%s"; +// +// private int numThreads; +// private HttpClient httpClient; +// private TaskInfoProvider taskInfoProvider; +// private FullResponseHolder responseHolder; +// private HttpResponse response; +// private HttpHeaders headers; +// private KinesisIndexTaskClient client; +// +// @Parameterized.Parameters(name = "numThreads = {0}") +// public static Iterable constructorFeeder() +// { +// return ImmutableList.of(new Object[]{1}, new Object[]{8}); +// } +// +// public KinesisIndexTaskClientTest(int numThreads) +// { +// this.numThreads = numThreads; +// } +// +// @Before +// public void setUp() throws Exception +// { +// httpClient = createMock(HttpClient.class); +// taskInfoProvider = createMock(TaskInfoProvider.class); +// responseHolder = createMock(FullResponseHolder.class); +// response = createMock(HttpResponse.class); +// headers = createMock(HttpHeaders.class); +// +// client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider); +// expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT)).anyTimes(); +// expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); +// +// for (int i = 0; i < TEST_IDS.size(); i++) { +// expect(taskInfoProvider.getTaskLocation(TEST_IDS.get(i))).andReturn(new TaskLocation(TEST_HOST, TEST_PORT)) +// .anyTimes(); +// expect(taskInfoProvider.getTaskStatus(TEST_IDS.get(i))).andReturn(Optional.of(TaskStatus.running(TEST_IDS.get(i)))) +// .anyTimes(); +// } +// } +// +// @After +// public void tearDown() throws Exception +// { +// client.close(); +// } +// +// @Test +// public void testNoTaskLocation() throws Exception +// { +// reset(taskInfoProvider); +// expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); +// expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); +// replayAll(); +// +// Assert.assertEquals(false, client.stop(TEST_ID, true)); +// Assert.assertEquals(false, client.resume(TEST_ID)); +// Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID)); +// Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID, 10)); +// Assert.assertEquals(KinesisIndexTask.Status.NOT_STARTED, client.getStatus(TEST_ID)); +// Assert.assertEquals(null, client.getStartTime(TEST_ID)); +// Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsets(TEST_ID, true)); +// Assert.assertEquals(ImmutableMap.of(), client.getEndOffsets(TEST_ID)); +// Assert.assertEquals(false, client.setEndOffsets(TEST_ID, ImmutableMap.of())); +// Assert.assertEquals(false, client.setEndOffsets(TEST_ID, ImmutableMap.of(), true)); +// +// verifyAll(); +// } +// +// @Test(expected = KinesisIndexTaskClient.TaskNotRunnableException.class) +// public void testTaskNotRunnableException() throws Exception +// { +// reset(taskInfoProvider); +// expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT)).anyTimes(); +// expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.failure(TEST_ID))).anyTimes(); +// replayAll(); +// +// client.getCurrentOffsets(TEST_ID, true); +// verifyAll(); +// } +// +// @Test(expected = RuntimeException.class) +// public void testInternalServerError() throws Exception +// { +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2); +// expect( +// httpClient.go( +// anyObject(Request.class), +// anyObject(FullResponseHandler.class), +// eq(TEST_HTTP_TIMEOUT) +// ) +// ).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// replayAll(); +// +// client.getCurrentOffsets(TEST_ID, true); +// verifyAll(); +// } +// +// @Test(expected = IAE.class) +// public void testBadRequest() throws Exception +// { +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2); +// expect(responseHolder.getContent()).andReturn(""); +// expect( +// httpClient.go( +// anyObject(Request.class), +// anyObject(FullResponseHandler.class), +// eq(TEST_HTTP_TIMEOUT) +// ) +// ).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// replayAll(); +// +// client.getCurrentOffsets(TEST_ID, true); +// verifyAll(); +// } +// +// @Test +// public void testTaskLocationMismatch() throws Exception +// { +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) +// .andReturn(HttpResponseStatus.OK); +// expect(responseHolder.getResponse()).andReturn(response); +// expect(responseHolder.getContent()).andReturn("") +// .andReturn("{}"); +// expect(response.headers()).andReturn(headers); +// expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id"); +// expect( +// httpClient.go( +// anyObject(Request.class), +// anyObject(FullResponseHandler.class), +// eq(TEST_HTTP_TIMEOUT) +// ) +// ).andReturn( +// Futures.immediateFuture(responseHolder) +// ).times(2); +// replayAll(); +// +// Map results = client.getCurrentOffsets(TEST_ID, true); +// verifyAll(); +// +// Assert.assertEquals(0, results.size()); +// } +// +// @Test +// public void testGetCurrentOffsets() throws Exception +// { +// Capture captured = Capture.newInstance(); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); +// expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// replayAll(); +// +// Map results = client.getCurrentOffsets(TEST_ID, true); +// verifyAll(); +// +// Request request = captured.getValue(); +// Assert.assertEquals(HttpMethod.GET, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"), +// request.getUrl() +// ); +// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); +// +// Assert.assertEquals(2, results.size()); +// Assert.assertEquals(1, (long) results.get(0)); +// Assert.assertEquals(10, (long) results.get(1)); +// } +// +// @Test +// public void testGetCurrentOffsetsWithRetry() throws Exception +// { +// client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 3); +// +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) +// .andReturn(HttpResponseStatus.OK).times(1); +// expect(responseHolder.getContent()).andReturn("").times(2) +// .andReturn("{\"0\":1, \"1\":10}"); +// expect(responseHolder.getResponse()).andReturn(response).times(2); +// expect(response.headers()).andReturn(headers).times(2); +// expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2); +// +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ).times(3); +// +// replayAll(); +// +// Map results = client.getCurrentOffsets(TEST_ID, true); +// verifyAll(); +// +// Assert.assertEquals(3, captured.getValues().size()); +// for (Request request : captured.getValues()) { +// Assert.assertEquals(HttpMethod.GET, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"), +// request.getUrl() +// ); +// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); +// } +// +// Assert.assertEquals(2, results.size()); +// Assert.assertEquals(1, (long) results.get(0)); +// Assert.assertEquals(10, (long) results.get(1)); +// } +// +// @Test(expected = RuntimeException.class) +// public void testGetCurrentOffsetsWithExhaustedRetries() throws Exception +// { +// client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); +// +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); +// expect(responseHolder.getContent()).andReturn("").anyTimes(); +// expect(responseHolder.getResponse()).andReturn(response).anyTimes(); +// expect(response.headers()).andReturn(headers).anyTimes(); +// expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).anyTimes(); +// +// expect( +// httpClient.go( +// anyObject(Request.class), +// anyObject(FullResponseHandler.class), +// eq(TEST_HTTP_TIMEOUT) +// ) +// ).andReturn(Futures.immediateFuture(responseHolder)).anyTimes(); +// replayAll(); +// +// client.getCurrentOffsets(TEST_ID, true); +// verifyAll(); +// } +// +// @Test +// public void testGetEndOffsets() throws Exception +// { +// Capture captured = Capture.newInstance(); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); +// expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// replayAll(); +// +// Map results = client.getEndOffsets(TEST_ID); +// verifyAll(); +// +// Request request = captured.getValue(); +// Assert.assertEquals(HttpMethod.GET, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end"), +// request.getUrl() +// ); +// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); +// +// Assert.assertEquals(2, results.size()); +// Assert.assertEquals(1, (long) results.get(0)); +// Assert.assertEquals(10, (long) results.get(1)); +// } +// +// @Test +// public void testGetStartTime() throws Exception +// { +// client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); +// DateTime now = DateTime.now(); +// +// Capture captured = Capture.newInstance(); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) +// .andReturn(HttpResponseStatus.OK); +// expect(responseHolder.getResponse()).andReturn(response); +// expect(response.headers()).andReturn(headers); +// expect(headers.get("X-Druid-Task-Id")).andReturn(null); +// expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ).times(2); +// replayAll(); +// +// DateTime results = client.getStartTime(TEST_ID); +// verifyAll(); +// +// Request request = captured.getValue(); +// Assert.assertEquals(HttpMethod.GET, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/time/start"), +// request.getUrl() +// ); +// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); +// +// Assert.assertEquals(now, results); +// } +// +// @Test +// public void testGetStatus() throws Exception +// { +// KinesisIndexTask.Status status = KinesisIndexTask.Status.READING; +// +// Capture captured = Capture.newInstance(); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); +// expect(responseHolder.getContent()).andReturn(String.format("\"%s\"", status.toString())).anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// replayAll(); +// +// KinesisIndexTask.Status results = client.getStatus(TEST_ID); +// verifyAll(); +// +// Request request = captured.getValue(); +// Assert.assertEquals(HttpMethod.GET, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/status"), +// request.getUrl() +// ); +// Assert.assertTrue(null, request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); +// +// Assert.assertEquals(status, results); +// } +// +// @Test +// public void testPause() throws Exception +// { +// Capture captured = Capture.newInstance(); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); +// expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// replayAll(); +// +// Map results = client.pause(TEST_ID); +// verifyAll(); +// +// Request request = captured.getValue(); +// Assert.assertEquals(HttpMethod.POST, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/pause"), +// request.getUrl() +// ); +// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); +// +// Assert.assertEquals(2, results.size()); +// Assert.assertEquals(1, (long) results.get(0)); +// Assert.assertEquals(10, (long) results.get(1)); +// } +// +// @Test +// public void testPauseWithTimeout() throws Exception +// { +// Capture captured = Capture.newInstance(); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); +// expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// replayAll(); +// +// Map results = client.pause(TEST_ID, 101); +// verifyAll(); +// +// Request request = captured.getValue(); +// Assert.assertEquals(HttpMethod.POST, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/pause?timeout=101"), +// request.getUrl() +// ); +// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); +// +// Assert.assertEquals(2, results.size()); +// Assert.assertEquals(1, (long) results.get(0)); +// Assert.assertEquals(10, (long) results.get(1)); +// } +// +// @Test +// public void testPauseWithSubsequentGetOffsets() throws Exception +// { +// Capture captured = Capture.newInstance(); +// Capture captured2 = Capture.newInstance(); +// Capture captured3 = Capture.newInstance(); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.ACCEPTED).times(2) +// .andReturn(HttpResponseStatus.OK).times(2); +// expect(responseHolder.getContent()).andReturn("\"PAUSED\"") +// .andReturn("{\"0\":1, \"1\":10}").anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// expect(httpClient.go(capture(captured2), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// expect(httpClient.go(capture(captured3), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// +// replayAll(); +// +// Map results = client.pause(TEST_ID); +// verifyAll(); +// +// Request request = captured.getValue(); +// Assert.assertEquals(HttpMethod.POST, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/pause"), +// request.getUrl() +// ); +// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); +// +// request = captured2.getValue(); +// Assert.assertEquals(HttpMethod.GET, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/status"), +// request.getUrl() +// ); +// +// request = captured3.getValue(); +// Assert.assertEquals(HttpMethod.GET, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"), +// request.getUrl() +// ); +// +// Assert.assertEquals(2, results.size()); +// Assert.assertEquals(1, (long) results.get(0)); +// Assert.assertEquals(10, (long) results.get(1)); +// } +// +// @Test +// public void testResume() throws Exception +// { +// Capture captured = Capture.newInstance(); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// replayAll(); +// +// client.resume(TEST_ID); +// verifyAll(); +// +// Request request = captured.getValue(); +// Assert.assertEquals(HttpMethod.POST, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/resume"), +// request.getUrl() +// ); +// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); +// } +// +// @Test +// public void testSetEndOffsets() throws Exception +// { +// Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); +// +// Capture captured = Capture.newInstance(); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// replayAll(); +// +// client.setEndOffsets(TEST_ID, endOffsets); +// verifyAll(); +// +// Request request = captured.getValue(); +// Assert.assertEquals(HttpMethod.POST, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end"), +// request.getUrl() +// ); +// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); +// Assert.assertEquals("{\"0\":15,\"1\":120}", new String(request.getContent().array())); +// } +// +// @Test +// public void testSetEndOffsetsAndResume() throws Exception +// { +// Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); +// +// Capture captured = Capture.newInstance(); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// replayAll(); +// +// client.setEndOffsets(TEST_ID, endOffsets, true); +// verifyAll(); +// +// Request request = captured.getValue(); +// Assert.assertEquals(HttpMethod.POST, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end?resume=true"), +// request.getUrl() +// ); +// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); +// Assert.assertEquals("{\"0\":15,\"1\":120}", new String(request.getContent().array())); +// } +// +// @Test +// public void testStop() throws Exception +// { +// Capture captured = Capture.newInstance(); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// replayAll(); +// +// client.stop(TEST_ID, false); +// verifyAll(); +// +// Request request = captured.getValue(); +// Assert.assertEquals(HttpMethod.POST, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/stop"), +// request.getUrl() +// ); +// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); +// } +// +// @Test +// public void testStopAndPublish() throws Exception +// { +// Capture captured = Capture.newInstance(); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ); +// replayAll(); +// +// client.stop(TEST_ID, true); +// verifyAll(); +// +// Request request = captured.getValue(); +// Assert.assertEquals(HttpMethod.POST, request.getMethod()); +// Assert.assertEquals( +// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/stop?publish=true"), +// request.getUrl() +// ); +// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); +// } +// +// @Test +// public void testStopAsync() throws Exception +// { +// final int numRequests = TEST_IDS.size(); +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ).times(numRequests); +// replayAll(); +// +// List expectedUrls = Lists.newArrayList(); +// List> futures = Lists.newArrayList(); +// for (int i = 0; i < numRequests; i++) { +// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "stop"))); +// futures.add(client.stopAsync(TEST_IDS.get(i), false)); +// } +// +// List responses = Futures.allAsList(futures).get(); +// +// verifyAll(); +// List requests = captured.getValues(); +// +// Assert.assertEquals(numRequests, requests.size()); +// Assert.assertEquals(numRequests, responses.size()); +// for (int i = 0; i < numRequests; i++) { +// Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); +// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); +// Assert.assertTrue(responses.get(i)); +// } +// } +// +// @Test +// public void testResumeAsync() throws Exception +// { +// final int numRequests = TEST_IDS.size(); +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ).times(numRequests); +// replayAll(); +// +// List expectedUrls = Lists.newArrayList(); +// List> futures = Lists.newArrayList(); +// for (int i = 0; i < numRequests; i++) { +// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "resume"))); +// futures.add(client.resumeAsync(TEST_IDS.get(i))); +// } +// +// List responses = Futures.allAsList(futures).get(); +// +// verifyAll(); +// List requests = captured.getValues(); +// +// Assert.assertEquals(numRequests, requests.size()); +// Assert.assertEquals(numRequests, responses.size()); +// for (int i = 0; i < numRequests; i++) { +// Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); +// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); +// Assert.assertTrue(responses.get(i)); +// } +// } +// +// @Test +// public void testPauseAsync() throws Exception +// { +// final int numRequests = TEST_IDS.size(); +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ).times(numRequests); +// replayAll(); +// +// List expectedUrls = Lists.newArrayList(); +// List>> futures = Lists.newArrayList(); +// for (int i = 0; i < numRequests; i++) { +// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "pause"))); +// futures.add(client.pauseAsync(TEST_IDS.get(i))); +// } +// +// List> responses = Futures.allAsList(futures).get(); +// +// verifyAll(); +// List requests = captured.getValues(); +// +// Assert.assertEquals(numRequests, requests.size()); +// Assert.assertEquals(numRequests, responses.size()); +// for (int i = 0; i < numRequests; i++) { +// Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); +// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); +// Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of(0, 1L)), responses.get(i)); +// } +// } +// +// @Test +// public void testPauseAsyncWithTimeout() throws Exception +// { +// final int numRequests = TEST_IDS.size(); +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ).times(numRequests); +// replayAll(); +// +// List expectedUrls = Lists.newArrayList(); +// List>> futures = Lists.newArrayList(); +// for (int i = 0; i < numRequests; i++) { +// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "pause?timeout=9"))); +// futures.add(client.pauseAsync(TEST_IDS.get(i), 9)); +// } +// +// List> responses = Futures.allAsList(futures).get(); +// +// verifyAll(); +// List requests = captured.getValues(); +// +// Assert.assertEquals(numRequests, requests.size()); +// Assert.assertEquals(numRequests, responses.size()); +// for (int i = 0; i < numRequests; i++) { +// Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); +// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); +// Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of(0, 1L)), responses.get(i)); +// } +// } +// +// @Test +// public void testGetStatusAsync() throws Exception +// { +// final int numRequests = TEST_IDS.size(); +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ).times(numRequests); +// replayAll(); +// +// List expectedUrls = Lists.newArrayList(); +// List> futures = Lists.newArrayList(); +// for (int i = 0; i < numRequests; i++) { +// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "status"))); +// futures.add(client.getStatusAsync(TEST_IDS.get(i))); +// } +// +// List responses = Futures.allAsList(futures).get(); +// +// verifyAll(); +// List requests = captured.getValues(); +// +// Assert.assertEquals(numRequests, requests.size()); +// Assert.assertEquals(numRequests, responses.size()); +// for (int i = 0; i < numRequests; i++) { +// Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); +// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); +// Assert.assertEquals(KinesisIndexTask.Status.READING, responses.get(i)); +// } +// } +// +// @Test +// public void testGetStartTimeAsync() throws Exception +// { +// final DateTime now = DateTime.now(); +// final int numRequests = TEST_IDS.size(); +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ).times(numRequests); +// replayAll(); +// +// List expectedUrls = Lists.newArrayList(); +// List> futures = Lists.newArrayList(); +// for (int i = 0; i < numRequests; i++) { +// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "time/start"))); +// futures.add(client.getStartTimeAsync(TEST_IDS.get(i))); +// } +// +// List responses = Futures.allAsList(futures).get(); +// +// verifyAll(); +// List requests = captured.getValues(); +// +// Assert.assertEquals(numRequests, requests.size()); +// Assert.assertEquals(numRequests, responses.size()); +// for (int i = 0; i < numRequests; i++) { +// Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); +// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); +// Assert.assertEquals(now, responses.get(i)); +// } +// } +// +// @Test +// public void testGetCurrentOffsetsAsync() throws Exception +// { +// final int numRequests = TEST_IDS.size(); +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ).times(numRequests); +// replayAll(); +// +// List expectedUrls = Lists.newArrayList(); +// List>> futures = Lists.newArrayList(); +// for (int i = 0; i < numRequests; i++) { +// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/current"))); +// futures.add(client.getCurrentOffsetsAsync(TEST_IDS.get(i), false)); +// } +// +// List> responses = Futures.allAsList(futures).get(); +// +// verifyAll(); +// List requests = captured.getValues(); +// +// Assert.assertEquals(numRequests, requests.size()); +// Assert.assertEquals(numRequests, responses.size()); +// for (int i = 0; i < numRequests; i++) { +// Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); +// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); +// Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of(0, 1L)), responses.get(i)); +// } +// } +// +// @Test +// public void testGetEndOffsetsAsync() throws Exception +// { +// final int numRequests = TEST_IDS.size(); +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ).times(numRequests); +// replayAll(); +// +// List expectedUrls = Lists.newArrayList(); +// List>> futures = Lists.newArrayList(); +// for (int i = 0; i < numRequests; i++) { +// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/end"))); +// futures.add(client.getEndOffsetsAsync(TEST_IDS.get(i))); +// } +// +// List> responses = Futures.allAsList(futures).get(); +// +// verifyAll(); +// List requests = captured.getValues(); +// +// Assert.assertEquals(numRequests, requests.size()); +// Assert.assertEquals(numRequests, responses.size()); +// for (int i = 0; i < numRequests; i++) { +// Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); +// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); +// Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of(0, 1L)), responses.get(i)); +// } +// } +// +// @Test +// public void testSetEndOffsetsAsync() throws Exception +// { +// final Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); +// final int numRequests = TEST_IDS.size(); +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ).times(numRequests); +// replayAll(); +// +// List expectedUrls = Lists.newArrayList(); +// List> futures = Lists.newArrayList(); +// for (int i = 0; i < numRequests; i++) { +// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/end"))); +// futures.add(client.setEndOffsetsAsync(TEST_IDS.get(i), endOffsets)); +// } +// +// List responses = Futures.allAsList(futures).get(); +// +// verifyAll(); +// List requests = captured.getValues(); +// +// Assert.assertEquals(numRequests, requests.size()); +// Assert.assertEquals(numRequests, responses.size()); +// for (int i = 0; i < numRequests; i++) { +// Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); +// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); +// Assert.assertTrue(responses.get(i)); +// } +// } +// +// @Test +// public void testSetEndOffsetsAsyncWithResume() throws Exception +// { +// final Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); +// final int numRequests = TEST_IDS.size(); +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); +// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( +// Futures.immediateFuture(responseHolder) +// ).times(numRequests); +// replayAll(); +// +// List expectedUrls = Lists.newArrayList(); +// List> futures = Lists.newArrayList(); +// for (int i = 0; i < numRequests; i++) { +// expectedUrls.add( +// new URL( +// String.format( +// URL_FORMATTER, +// TEST_HOST, +// TEST_PORT, +// TEST_IDS.get(i), +// "offsets/end?resume=true" +// ) +// ) +// ); +// futures.add(client.setEndOffsetsAsync(TEST_IDS.get(i), endOffsets, true)); +// } +// +// List responses = Futures.allAsList(futures).get(); +// +// verifyAll(); +// List requests = captured.getValues(); +// +// Assert.assertEquals(numRequests, requests.size()); +// Assert.assertEquals(numRequests, responses.size()); +// for (int i = 0; i < numRequests; i++) { +// Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); +// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); +// Assert.assertTrue(responses.get(i)); +// } +// } +// +// private class TestableKinesisIndexTaskClient extends KinesisIndexTaskClient +// { +// public TestableKinesisIndexTaskClient( +// HttpClient httpClient, +// ObjectMapper jsonMapper, +// TaskInfoProvider taskInfoProvider +// ) +// { +// this(httpClient, jsonMapper, taskInfoProvider, TEST_NUM_RETRIES); +// } +// +// public TestableKinesisIndexTaskClient( +// HttpClient httpClient, +// ObjectMapper jsonMapper, +// TaskInfoProvider taskInfoProvider, +// long numRetries +// ) +// { +// super(httpClient, jsonMapper, taskInfoProvider, TEST_DATASOURCE, numThreads, TEST_HTTP_TIMEOUT, numRetries); +// } +// +// @Override +// void checkConnection(String host, int port) throws IOException { } +// } +//} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java new file mode 100644 index 000000000000..065a64998cf1 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -0,0 +1,1665 @@ +///* +// * Licensed to Metamarkets Group Inc. (Metamarkets) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. Metamarkets licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, +// * software distributed under the License is distributed on an +// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// * KIND, either express or implied. See the License for the +// * specific language governing permissions and limitations +// * under the License. +// */ +// +//package org.apache.druid.indexing.kinesis; +// +//import com.fasterxml.jackson.core.type.TypeReference; +//import com.fasterxml.jackson.databind.Module; +//import com.fasterxml.jackson.databind.ObjectMapper; +//import com.google.common.base.Charsets; +//import com.google.common.base.Function; +//import com.google.common.base.Predicate; +//import com.google.common.base.Predicates; +//import com.google.common.base.Throwables; +//import com.google.common.collect.FluentIterable; +//import com.google.common.collect.ImmutableList; +//import com.google.common.collect.ImmutableMap; +//import com.google.common.collect.ImmutableSet; +//import com.google.common.collect.Iterables; +//import com.google.common.collect.Lists; +//import com.google.common.io.Files; +//import com.google.common.util.concurrent.ListenableFuture; +//import com.google.common.util.concurrent.ListeningExecutorService; +//import com.google.common.util.concurrent.MoreExecutors; +//import com.metamx.common.logger.Logger; +//import com.metamx.emitter.EmittingLogger; +//import com.metamx.emitter.core.LoggingEmitter; +//import com.metamx.emitter.service.ServiceEmitter; +//import com.metamx.metrics.MonitorScheduler; +//import org.apache.druid.client.cache.CacheConfig; +//import org.apache.druid.client.cache.MapCache; +//import org.apache.druid.concurrent.Execs; +//import org.apache.druid.data.input.impl.DimensionsSpec; +//import org.apache.druid.data.input.impl.JSONParseSpec; +//import org.apache.druid.data.input.impl.JSONPathFieldSpec; +//import org.apache.druid.data.input.impl.JSONPathSpec; +//import org.apache.druid.data.input.impl.StringInputRowParser; +//import org.apache.druid.data.input.impl.TimestampSpec; +//import org.apache.druid.indexing.common.SegmentLoaderFactory; +//import org.apache.druid.indexing.common.TaskLock; +//import org.apache.druid.indexing.common.TaskStatus; +//import org.apache.druid.indexing.common.TaskToolbox; +//import org.apache.druid.indexing.common.TaskToolboxFactory; +//import org.apache.druid.indexing.common.TestUtils; +//import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; +//import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +//import org.apache.druid.indexing.common.actions.TaskActionToolbox; +//import org.apache.druid.indexing.common.config.TaskConfig; +//import org.apache.druid.indexing.common.config.TaskStorageConfig; +//import org.apache.druid.indexing.common.task.Task; +//import org.apache.druid.indexing.kinesis.test.TestBroker; +//import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +//import org.apache.druid.indexing.overlord.MetadataTaskStorage; +//import org.apache.druid.indexing.overlord.TaskLockbox; +//import org.apache.druid.indexing.overlord.TaskStorage; +//import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +//import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; +//import org.apache.druid.indexing.test.TestDataSegmentKiller; +//import org.apache.druid.jackson.DefaultObjectMapper; +//import org.apache.druid.java.util.common.CompressionUtils; +//import org.apache.druid.java.util.common.ISE; +//import org.apache.druid.java.util.common.granularity.Granularities; +//import org.apache.druid.java.util.common.guava.Sequences; +//import org.apache.druid.metadata.EntryExistsException; +//import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; +//import org.apache.druid.metadata.SQLMetadataStorageActionHandlerFactory; +//import org.apache.druid.metadata.TestDerbyConnector; +//import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; +//import org.apache.druid.query.Druids; +//import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; +//import org.apache.druid.query.Query; +//import org.apache.druid.query.QueryRunner; +//import org.apache.druid.query.QueryRunnerFactory; +//import org.apache.druid.query.QueryRunnerFactoryConglomerate; +//import org.apache.druid.query.QueryToolChest; +//import org.apache.druid.query.QueryWatcher; +//import org.apache.druid.query.Result; +//import org.apache.druid.query.SegmentDescriptor; +//import org.apache.druid.query.aggregation.AggregatorFactory; +//import org.apache.druid.query.aggregation.CountAggregatorFactory; +//import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +//import org.apache.druid.query.timeseries.TimeseriesQuery; +//import org.apache.druid.query.timeseries.TimeseriesQueryEngine; +//import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +//import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; +//import org.apache.druid.query.timeseries.TimeseriesResultValue; +//import org.apache.druid.segment.IndexIO; +//import org.apache.druid.segment.QueryableIndex; +//import org.apache.druid.segment.column.DictionaryEncodedColumn; +//import org.apache.druid.segment.indexing.DataSchema; +//import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +//import org.apache.druid.segment.loading.DataSegmentPusher; +//import org.apache.druid.segment.loading.LocalDataSegmentPusher; +//import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; +//import org.apache.druid.segment.loading.SegmentLoaderConfig; +//import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager; +//import org.apache.druid.segment.loading.StorageLocationConfig; +//import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; +//import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +//import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; +//import org.apache.druid.timeline.DataSegment; +//import org.apache.curator.test.TestingCluster; +//import org.apache.kafka.clients.producer.KafkaProducer; +//import org.apache.kafka.clients.producer.ProducerRecord; +//import org.easymock.EasyMock; +//import org.joda.time.DateTime; +//import org.joda.time.Interval; +//import org.joda.time.Period; +//import org.junit.After; +//import org.junit.Assert; +//import org.junit.Before; +//import org.junit.Rule; +//import org.junit.Test; +//import org.junit.rules.TemporaryFolder; +//import org.junit.runner.RunWith; +//import org.junit.runners.Parameterized; +// +//import java.io.File; +//import java.io.IOException; +//import java.util.ArrayList; +//import java.util.List; +//import java.util.Map; +//import java.util.Set; +//import java.util.concurrent.Callable; +//import java.util.concurrent.Executor; +//import java.util.concurrent.Executors; +//import java.util.concurrent.TimeUnit; +//import java.util.concurrent.TimeoutException; +// +//@RunWith(Parameterized.class) +//public class KinesisIndexTaskTest +//{ +// private final boolean buildV9Directly; +// private long handoffConditionTimeout = 0; +// private boolean reportParseExceptions = false; +// private boolean doHandoff = true; +// +// private TestingCluster zkServer; +// private TestBroker kafkaServer; +// private ServiceEmitter emitter; +// private ListeningExecutorService taskExec; +// private TaskToolboxFactory toolboxFactory; +// private IndexerMetadataStorageCoordinator metadataStorageCoordinator; +// private TaskStorage taskStorage; +// private TaskLockbox taskLockbox; +// private File directory; +// +// private final List runningTasks = Lists.newArrayList(); +// +// private static final Logger log = new Logger(KinesisIndexTaskTest.class); +// private static final ObjectMapper objectMapper = new DefaultObjectMapper(); +// +// private static final DataSchema DATA_SCHEMA; +// +// private static final List> RECORDS = ImmutableList.of( +// new ProducerRecord("topic0", 0, null, JB("2008", "a", "y", 1.0f)), +// new ProducerRecord("topic0", 0, null, JB("2009", "b", "y", 1.0f)), +// new ProducerRecord("topic0", 0, null, JB("2010", "c", "y", 1.0f)), +// new ProducerRecord("topic0", 0, null, JB("2011", "d", "y", 1.0f)), +// new ProducerRecord("topic0", 0, null, JB("2011", "e", "y", 1.0f)), +// new ProducerRecord("topic0", 0, null, "unparseable".getBytes()), +// new ProducerRecord("topic0", 0, null, null), +// new ProducerRecord("topic0", 0, null, JB("2013", "f", "y", 1.0f)), +// new ProducerRecord("topic0", 1, null, JB("2012", "g", "y", 1.0f)), +// new ProducerRecord("topic0", 1, null, JB("2011", "h", "y", 1.0f)) +// ); +// +// static { +// DATA_SCHEMA = new DataSchema( +// "test_ds", +// objectMapper.convertValue( +// new StringInputRowParser( +// new JSONParseSpec( +// new TimestampSpec("timestamp", "iso", null), +// new DimensionsSpec( +// DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2")), +// null, +// null +// ), +// new JSONPathSpec(true, ImmutableList.of()), +// ImmutableMap.of() +// ), +// Charsets.UTF_8.name() +// ), +// Map.class +// ), +// new AggregatorFactory[]{new CountAggregatorFactory("rows")}, +// new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), +// objectMapper +// ); +// } +// +// @Parameterized.Parameters(name = "buildV9Directly = {0}") +// public static Iterable constructorFeeder() +// { +// return ImmutableList.of(new Object[]{true}, new Object[]{false}); +// } +// +// public KinesisIndexTaskTest(boolean buildV9Directly) +// { +// this.buildV9Directly = buildV9Directly; +// } +// +// @Rule +// public final TemporaryFolder tempFolder = new TemporaryFolder(); +// +// @Rule +// public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); +// +// @Before +// public void setUp() throws Exception +// { +// emitter = new ServiceEmitter( +// "service", +// "host", +// new LoggingEmitter( +// log, +// LoggingEmitter.Level.ERROR, +// new DefaultObjectMapper() +// ) +// ); +// emitter.start(); +// EmittingLogger.registerEmitter(emitter); +// +// makeToolboxFactory(); +// +// zkServer = new TestingCluster(1); +// zkServer.start(); +// +// kafkaServer = new TestBroker( +// zkServer.getConnectString(), +// tempFolder.newFolder(), +// 1, +// ImmutableMap.of("num.partitions", "2") +// ); +// kafkaServer.start(); +// +// taskExec = MoreExecutors.listeningDecorator( +// Executors.newCachedThreadPool( +// Execs.makeThreadFactory("kinesis-task-test-%d") +// ) +// ); +// +// handoffConditionTimeout = 0; +// reportParseExceptions = false; +// doHandoff = true; +// } +// +// @After +// public void tearDown() throws Exception +// { +// emitter.close(); +// +// synchronized (runningTasks) { +// for (Task task : runningTasks) { +// task.stopGracefully(); +// } +// +// runningTasks.clear(); +// } +// +// taskExec.shutdown(); +// taskExec.awaitTermination(9999, TimeUnit.DAYS); +// +// kafkaServer.close(); +// kafkaServer = null; +// +// zkServer.stop(); +// zkServer = null; +// +// destroyToolboxFactory(); +// } +// +// @Test(timeout = 60_000L) +// public void testRunAfterDataInserted() throws Exception +// { +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// final KinesisIndexTask task = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future = runTask(task); +// +// // Wait for task to exit +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); +// Assert.assertEquals( +// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// } +// +// @Test(timeout = 60_000L) +// public void testRunBeforeDataInserted() throws Exception +// { +// final KinesisIndexTask task = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future = runTask(task); +// +// // Wait for the task to start reading +// while (task.getStatus() != KinesisIndexTask.Status.READING) { +// Thread.sleep(10); +// } +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// // Wait for task to exit +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); +// Assert.assertEquals( +// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// } +// +// @Test(timeout = 60_000L) +// public void testRunWithMinimumMessageTime() throws Exception +// { +// final KinesisIndexTask task = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 0L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// new DateTime("2010"), +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future = runTask(task); +// +// // Wait for the task to start reading +// while (task.getStatus() != KinesisIndexTask.Status.READING) { +// Thread.sleep(10); +// } +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// // Wait for task to exit +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(2, task.getFireDepartmentMetrics().thrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); +// Assert.assertEquals( +// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// } +// +// @Test(timeout = 60_000L) +// public void testRunOnNothing() throws Exception +// { +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// final KinesisIndexTask task = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future = runTask(task); +// +// // Wait for task to exit +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(0, task.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// +// // Check published metadata +// Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); +// } +// +// @Test(timeout = 60_000L) +// public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception +// { +// handoffConditionTimeout = 5_000; +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// final KinesisIndexTask task = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future = runTask(task); +// +// // Wait for task to exit +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); +// Assert.assertEquals( +// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// } +// +// @Test(timeout = 60_000L) +// public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exception +// { +// doHandoff = false; +// handoffConditionTimeout = 100; +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// final KinesisIndexTask task = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future = runTask(task); +// +// // Wait for task to exit +// Assert.assertEquals(TaskStatus.Status.FAILED, future.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); +// Assert.assertEquals( +// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// } +// +// @Test(timeout = 60_000L) +// public void testReportParseExceptions() throws Exception +// { +// reportParseExceptions = true; +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// final KinesisIndexTask task = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 7L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future = runTask(task); +// +// // Wait for task to exit +// Assert.assertEquals(TaskStatus.Status.FAILED, future.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// +// // Check published metadata +// Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); +// Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); +// } +// +// @Test(timeout = 60_000L) +// public void testRunReplicas() throws Exception +// { +// final KinesisIndexTask task1 = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// final KinesisIndexTask task2 = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future1 = runTask(task1); +// final ListenableFuture future2 = runTask(task2); +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// // Wait for tasks to exit +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task2.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); +// +// // Check published segments & metadata +// SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); +// Assert.assertEquals( +// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// } +// +// @Test(timeout = 60_000L) +// public void testRunConflicting() throws Exception +// { +// final KinesisIndexTask task1 = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// final KinesisIndexTask task2 = createTask( +// null, +// new KinesisIOConfig( +// "sequence1", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 3L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 8L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// // Run first task +// final ListenableFuture future1 = runTask(task1); +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); +// +// // Run second task +// final ListenableFuture future2 = runTask(task2); +// Assert.assertEquals(TaskStatus.Status.FAILED, future2.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(2, task2.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); +// +// // Check published segments & metadata, should all be from the first task +// SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); +// Assert.assertEquals( +// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// } +// +// @Test(timeout = 60_000L) +// public void testRunConflictingWithoutTransactions() throws Exception +// { +// final KinesisIndexTask task1 = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// false, +// false, +// null, +// false +// ), +// null, +// null +// ); +// final KinesisIndexTask task2 = createTask( +// null, +// new KinesisIOConfig( +// "sequence1", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 3L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 8L)), +// kafkaServer.consumerProperties(), +// false, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// // Run first task +// final ListenableFuture future1 = runTask(task1); +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); +// +// // Check published segments & metadata +// SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); +// Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); +// +// // Run second task +// final ListenableFuture future2 = runTask(task2); +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(2, task2.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); +// +// // Check published segments & metadata +// SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1); +// SegmentDescriptor desc4 = SD(task2, "2013/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); +// Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc3)); +// Assert.assertEquals(ImmutableList.of("f"), readSegmentDim1(desc4)); +// } +// +// @Test(timeout = 60_000L) +// public void testRunOneTaskTwoPartitions() throws Exception +// { +// final KinesisIndexTask task = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L, 1, 0L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L, 1, 2L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future = runTask(task); +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// // Wait for tasks to exit +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(5, task.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// +// // Check published segments & metadata +// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); +// SegmentDescriptor desc3 = SD(task, "2011/P1D", 1); +// SegmentDescriptor desc4 = SD(task, "2012/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); +// Assert.assertEquals( +// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L, 1, 2L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); +// Assert.assertEquals(ImmutableList.of("g"), readSegmentDim1(desc4)); +// +// // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically +// Assert.assertEquals( +// ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), +// ImmutableSet.of(readSegmentDim1(desc2), readSegmentDim1(desc3)) +// ); +// } +// +// @Test(timeout = 60_000L) +// public void testRunTwoTasksTwoPartitions() throws Exception +// { +// final KinesisIndexTask task1 = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// final KinesisIndexTask task2 = createTask( +// null, +// new KinesisIOConfig( +// "sequence1", +// new KinesisPartitions("topic0", ImmutableMap.of(1, 0L)), +// new KinesisPartitions("topic0", ImmutableMap.of(1, 1L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future1 = runTask(task1); +// final ListenableFuture future2 = runTask(task2); +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// // Wait for tasks to exit +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(1, task2.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task2.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); +// +// // Check published segments & metadata +// SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); +// SegmentDescriptor desc3 = SD(task2, "2012/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); +// Assert.assertEquals( +// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L, 1, 1L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// Assert.assertEquals(ImmutableList.of("g"), readSegmentDim1(desc3)); +// } +// +// @Test(timeout = 60_000L) +// public void testRestore() throws Exception +// { +// final KinesisIndexTask task1 = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future1 = runTask(task1); +// +// // Insert some data, but not enough for the task to finish +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : Iterables.limit(RECORDS, 4)) { +// kafkaProducer.send(record).get(); +// } +// } +// +// while (countEvents(task1) != 2) { +// Thread.sleep(25); +// } +// +// Assert.assertEquals(2, countEvents(task1)); +// +// // Stop without publishing segment +// task1.stopGracefully(); +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); +// +// // Start a new task +// final KinesisIndexTask task2 = createTask( +// task1.getId(), +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future2 = runTask(task2); +// +// // Insert remaining data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : Iterables.skip(RECORDS, 4)) { +// kafkaProducer.send(record).get(); +// } +// } +// +// // Wait for task to exit +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(2, task1.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(1, task2.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task2.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); +// +// // Check published segments & metadata +// SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); +// Assert.assertEquals( +// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// } +// +// @Test(timeout = 60_000L) +// public void testRunWithPauseAndResume() throws Exception +// { +// final KinesisIndexTask task = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future = runTask(task); +// +// // Insert some data, but not enough for the task to finish +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : Iterables.limit(RECORDS, 4)) { +// kafkaProducer.send(record).get(); +// } +// } +// +// while (countEvents(task) != 2) { +// Thread.sleep(25); +// } +// +// Assert.assertEquals(2, countEvents(task)); +// Assert.assertEquals(KinesisIndexTask.Status.READING, task.getStatus()); +// +// Map currentOffsets = objectMapper.readValue( +// task.pause(0).getEntity().toString(), +// new TypeReference>() +// { +// } +// ); +// Assert.assertEquals(KinesisIndexTask.Status.PAUSED, task.getStatus()); +// +// // Insert remaining data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : Iterables.skip(RECORDS, 4)) { +// kafkaProducer.send(record).get(); +// } +// } +// +// try { +// future.get(10, TimeUnit.SECONDS); +// Assert.fail("Task completed when it should have been paused"); +// } +// catch (TimeoutException e) { +// // carry on.. +// } +// +// Assert.assertEquals(currentOffsets, task.getCurrentOffsets()); +// +// task.resume(); +// +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// Assert.assertEquals(task.getEndOffsets(), task.getCurrentOffsets()); +// +// // Check metrics +// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); +// Assert.assertEquals( +// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// } +// +// @Test(timeout = 60_000L) +// public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception +// { +// final KinesisIndexTask task = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 1L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 3L)), +// kafkaServer.consumerProperties(), +// true, +// true, +// null, +// false +// ), +// null, +// null +// ); +// +// final ListenableFuture future = runTask(task); +// +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// while (task.getStatus() != KinesisIndexTask.Status.PAUSED) { +// Thread.sleep(25); +// } +// +// // reached the end of the assigned offsets and paused instead of publishing +// Assert.assertEquals(task.getEndOffsets(), task.getCurrentOffsets()); +// Assert.assertEquals(KinesisIndexTask.Status.PAUSED, task.getStatus()); +// +// Assert.assertEquals(ImmutableMap.of(0, 3L), task.getEndOffsets()); +// Map newEndOffsets = ImmutableMap.of(0, 4L); +// task.setEndOffsets(newEndOffsets, false); +// Assert.assertEquals(newEndOffsets, task.getEndOffsets()); +// Assert.assertEquals(KinesisIndexTask.Status.PAUSED, task.getStatus()); +// task.resume(); +// +// while (task.getStatus() != KinesisIndexTask.Status.PAUSED) { +// Thread.sleep(25); +// } +// +// // reached the end of the updated offsets and paused +// Assert.assertEquals(newEndOffsets, task.getCurrentOffsets()); +// Assert.assertEquals(KinesisIndexTask.Status.PAUSED, task.getStatus()); +// +// // try again but with resume flag == true +// newEndOffsets = ImmutableMap.of(0, 6L); +// task.setEndOffsets(newEndOffsets, true); +// Assert.assertEquals(newEndOffsets, task.getEndOffsets()); +// Assert.assertNotEquals(KinesisIndexTask.Status.PAUSED, task.getStatus()); +// +// while (task.getStatus() != KinesisIndexTask.Status.PAUSED) { +// Thread.sleep(25); +// } +// +// Assert.assertEquals(newEndOffsets, task.getCurrentOffsets()); +// Assert.assertEquals(KinesisIndexTask.Status.PAUSED, task.getStatus()); +// +// task.resume(); +// +// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(4, task.getFireDepartmentMetrics().processed()); +// Assert.assertEquals(1, task.getFireDepartmentMetrics().unparseable()); +// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2009/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2010/P1D", 0); +// SegmentDescriptor desc3 = SD(task, "2011/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); +// Assert.assertEquals( +// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 6L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("b"), readSegmentDim1(desc1)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc2)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc3)); +// } +// +// @Test(timeout = 30_000L) +// public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception +// { +// final KinesisIndexTask task = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// null +// ); +// +// runTask(task); +// +// while (!task.getStatus().equals(KinesisIndexTask.Status.READING)) { +// Thread.sleep(2000); +// } +// +// task.pause(0); +// +// while (!task.getStatus().equals(KinesisIndexTask.Status.PAUSED)) { +// Thread.sleep(25); +// } +// } +// +// @Test(timeout = 30_000L) +// public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAvailable() throws Exception +// { +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : RECORDS) { +// kafkaProducer.send(record).get(); +// } +// } +// +// final KinesisIndexTask task = createTask( +// null, +// new KinesisIOConfig( +// "sequence0", +// new KinesisPartitions("topic0", ImmutableMap.of(0, 200L)), +// new KinesisPartitions("topic0", ImmutableMap.of(0, 500L)), +// kafkaServer.consumerProperties(), +// true, +// false, +// null, +// false +// ), +// null, +// true +// ); +// +// runTask(task); +// +// while (!task.getStatus().equals(KinesisIndexTask.Status.READING)) { +// Thread.sleep(2000); +// } +// +// int i = 0; +// while(i++ < 5) { +// Assert.assertEquals(task.getStatus(), KinesisIndexTask.Status.READING); +// // Offset should not be reset +// Assert.assertTrue(task.getCurrentOffsets().get(0) == 200L); +// Thread.sleep(2000); +// } +// } +// +// private ListenableFuture runTask(final Task task) +// { +// try { +// taskStorage.insert(task, TaskStatus.running(task.getId())); +// } +// catch (EntryExistsException e) { +// // suppress +// } +// taskLockbox.syncFromStorage(); +// final TaskToolbox toolbox = toolboxFactory.build(task); +// synchronized (runningTasks) { +// runningTasks.add(task); +// } +// return taskExec.submit( +// new Callable() +// { +// @Override +// public TaskStatus call() throws Exception +// { +// try { +// if (task.isReady(toolbox.getTaskActionClient())) { +// return task.run(toolbox); +// } else { +// throw new ISE("Task is not ready"); +// } +// } +// catch (Exception e) { +// log.warn(e, "Task failed"); +// return TaskStatus.failure(task.getId()); +// } +// } +// } +// ); +// } +// +// private TaskLock getLock(final Task task, final Interval interval) +// { +// return Iterables.find( +// taskLockbox.findLocksForTask(task), +// new Predicate() +// { +// @Override +// public boolean apply(TaskLock lock) +// { +// return lock.getInterval().contains(interval); +// } +// } +// ); +// } +// +// private KinesisIndexTask createTask( +// final String taskId, +// final KinesisIOConfig ioConfig, +// final Integer maxRowsPerSegment, +// final Boolean resetOffsetAutomatically +// ) +// { +// final KinesisTuningConfig tuningConfig = new KinesisTuningConfig( +// 1000, +// maxRowsPerSegment, +// new Period("P1Y"), +// null, +// null, +// null, +// buildV9Directly, +// reportParseExceptions, +// handoffConditionTimeout, +// resetOffsetAutomatically +// ); +// return new KinesisIndexTask( +// taskId, +// null, +// DATA_SCHEMA, +// tuningConfig, +// ioConfig, +// null, +// null +// ); +// } +// +// private QueryRunnerFactoryConglomerate makeTimeseriesOnlyConglomerate() +// { +// return new DefaultQueryRunnerFactoryConglomerate( +// ImmutableMap., QueryRunnerFactory>of( +// TimeseriesQuery.class, +// new TimeseriesQueryRunnerFactory( +// new TimeseriesQueryQueryToolChest( +// new IntervalChunkingQueryRunnerDecorator(null, null, null) +// { +// @Override +// public QueryRunner decorate( +// QueryRunner delegate, QueryToolChest> toolChest +// ) +// { +// return delegate; +// } +// } +// ), +// new TimeseriesQueryEngine(), +// new QueryWatcher() +// { +// @Override +// public void registerQuery(Query query, ListenableFuture future) +// { +// // do nothing +// } +// } +// ) +// ) +// ); +// } +// +// private void makeToolboxFactory() throws IOException +// { +// directory = tempFolder.newFolder(); +// final TestUtils testUtils = new TestUtils(); +// final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); +// for (Module module : new KinesisIndexingServiceModule().getJacksonModules()) { +// objectMapper.registerModule(module); +// } +// final TaskConfig taskConfig = new TaskConfig( +// new File(directory, "taskBaseDir").getPath(), +// null, +// null, +// 50000, +// null, +// false, +// null, +// null +// ); +// final TestDerbyConnector derbyConnector = derby.getConnector(); +// derbyConnector.createDataSourceTable(); +// derbyConnector.createPendingSegmentsTable(); +// derbyConnector.createSegmentTable(); +// derbyConnector.createRulesTable(); +// derbyConnector.createConfigTable(); +// derbyConnector.createTaskTables(); +// derbyConnector.createAuditTable(); +// taskStorage = new MetadataTaskStorage( +// derbyConnector, +// new TaskStorageConfig(null), +// new SQLMetadataStorageActionHandlerFactory( +// derbyConnector, +// derby.metadataTablesConfigSupplier().get(), +// objectMapper +// ) +// ); +// metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( +// testUtils.getTestObjectMapper(), +// derby.metadataTablesConfigSupplier().get(), +// derbyConnector +// ); +// taskLockbox = new TaskLockbox(taskStorage); +// final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( +// taskLockbox, +// metadataStorageCoordinator, +// emitter, +// new SupervisorManager(null) +// ); +// final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( +// taskStorage, +// taskActionToolbox +// ); +// final SegmentHandoffNotifierFactory handoffNotifierFactory = new SegmentHandoffNotifierFactory() +// { +// @Override +// public SegmentHandoffNotifier createSegmentHandoffNotifier(String dataSource) +// { +// return new SegmentHandoffNotifier() +// { +// @Override +// public boolean registerSegmentHandoffCallback( +// SegmentDescriptor descriptor, Executor exec, Runnable handOffRunnable +// ) +// { +// if (doHandoff) { +// // Simulate immediate handoff +// exec.execute(handOffRunnable); +// } +// return true; +// } +// +// @Override +// public void start() +// { +// //Noop +// } +// +// @Override +// public void close() +// { +// //Noop +// } +// }; +// } +// }; +// final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig(); +// dataSegmentPusherConfig.storageDirectory = getSegmentDirectory(); +// final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig, objectMapper); +// toolboxFactory = new TaskToolboxFactory( +// taskConfig, +// taskActionClientFactory, +// emitter, +// dataSegmentPusher, +// new TestDataSegmentKiller(), +// null, // DataSegmentMover +// null, // DataSegmentArchiver +// new TestDataSegmentAnnouncer(), +// EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), +// handoffNotifierFactory, +// makeTimeseriesOnlyConglomerate(), +// MoreExecutors.sameThreadExecutor(), // queryExecutorService +// EasyMock.createMock(MonitorScheduler.class), +// new SegmentLoaderFactory( +// new SegmentLoaderLocalCacheManager( +// null, +// new SegmentLoaderConfig() +// { +// @Override +// public List getLocations() +// { +// return Lists.newArrayList(); +// } +// }, testUtils.getTestObjectMapper() +// ) +// ), +// testUtils.getTestObjectMapper(), +// testUtils.getTestIndexMerger(), +// testUtils.getTestIndexIO(), +// MapCache.create(1024), +// new CacheConfig(), +// testUtils.getTestIndexMergerV9() +// ); +// } +// +// private void destroyToolboxFactory() +// { +// toolboxFactory = null; +// taskStorage = null; +// taskLockbox = null; +// metadataStorageCoordinator = null; +// } +// +// private Set publishedDescriptors() throws IOException +// { +// return FluentIterable.from( +// metadataStorageCoordinator.getUsedSegmentsForInterval( +// DATA_SCHEMA.getDataSource(), +// new Interval("0000/3000") +// ) +// ).transform( +// new Function() +// { +// @Override +// public SegmentDescriptor apply(DataSegment input) +// { +// return input.toDescriptor(); +// } +// } +// ).toSet(); +// } +// +// private File getSegmentDirectory() +// { +// return new File(directory, "segments"); +// } +// +// private List readSegmentDim1(final SegmentDescriptor descriptor) throws IOException +// { +// File indexZip = new File( +// String.format( +// "%s/%s/%s_%s/%s/%d/index.zip", +// getSegmentDirectory(), +// DATA_SCHEMA.getDataSource(), +// descriptor.getInterval().getStart(), +// descriptor.getInterval().getEnd(), +// descriptor.getVersion(), +// descriptor.getPartitionNumber() +// ) +// ); +// File outputLocation = new File( +// directory, +// String.format( +// "%s_%s_%s_%s", +// descriptor.getInterval().getStart(), +// descriptor.getInterval().getEnd(), +// descriptor.getVersion(), +// descriptor.getPartitionNumber() +// ) +// ); +// outputLocation.mkdir(); +// CompressionUtils.unzip( +// Files.asByteSource(indexZip), +// outputLocation, +// Predicates.alwaysFalse(), +// false +// ); +// IndexIO indexIO = new TestUtils().getTestIndexIO(); +// QueryableIndex index = indexIO.loadIndex(outputLocation); +// DictionaryEncodedColumn dim1 = index.getColumn("dim1").getDictionaryEncoding(); +// List values = Lists.newArrayList(); +// for (int i = 0; i < dim1.length(); i++) { +// int id = dim1.getSingleValueRow(i); +// String value = dim1.lookupName(id); +// values.add(value); +// } +// return values; +// } +// +// public long countEvents(final Task task) throws Exception +// { +// // Do a query. +// TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() +// .dataSource(DATA_SCHEMA.getDataSource()) +// .aggregators( +// ImmutableList.of( +// new LongSumAggregatorFactory("rows", "rows") +// ) +// ).granularity(Granularities.ALL) +// .intervals("0000/3000") +// .build(); +// +// ArrayList> results = Sequences.toList( +// task.getQueryRunner(query).run(query, ImmutableMap.of()), +// Lists.>newArrayList() +// ); +// +// return results.isEmpty() ? 0 : results.get(0).getValue().getLongMetric("rows"); +// } +// +// private static byte[] JB(String timestamp, String dim1, String dim2, double met1) +// { +// try { +// return new ObjectMapper().writeValueAsBytes( +// ImmutableMap.of("timestamp", timestamp, "dim1", dim1, "dim2", dim2, "met1", met1) +// ); +// } +// catch (Exception e) { +// throw Throwables.propagate(e); +// } +// } +// +// private SegmentDescriptor SD(final Task task, final String intervalString, final int partitionNum) +// { +// final Interval interval = new Interval(intervalString); +// return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum); +// } +//} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java new file mode 100644 index 000000000000..16d3def45953 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java @@ -0,0 +1,120 @@ +///* +// * Licensed to Metamarkets Group Inc. (Metamarkets) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. Metamarkets licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, +// * software distributed under the License is distributed on an +// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// * KIND, either express or implied. See the License for the +// * specific language governing permissions and limitations +// * under the License. +// */ +// +//package org.apache.druid.indexing.kinesis; +// +//import com.fasterxml.jackson.databind.Module; +//import com.fasterxml.jackson.databind.ObjectMapper; +//import org.apache.druid.jackson.DefaultObjectMapper; +//import org.apache.druid.segment.IndexSpec; +//import org.apache.druid.segment.indexing.TuningConfig; +//import org.joda.time.Period; +//import org.junit.Assert; +//import org.junit.Test; +// +//import java.io.File; +// +//public class KinesisTuningConfigTest +//{ +// private final ObjectMapper mapper; +// +// public KinesisTuningConfigTest() +// { +// mapper = new DefaultObjectMapper(); +// mapper.registerModules((Iterable) new KinesisIndexingServiceModule().getJacksonModules()); +// } +// +// @Test +// public void testSerdeWithDefaults() throws Exception +// { +// String jsonStr = "{\"type\": \"kinesis\"}"; +// +// KinesisTuningConfig config = (KinesisTuningConfig) mapper.readValue( +// mapper.writeValueAsString( +// mapper.readValue( +// jsonStr, +// TuningConfig.class +// ) +// ), +// TuningConfig.class +// ); +// +// Assert.assertNotNull(config.getBasePersistDirectory()); +// Assert.assertEquals(75000, config.getMaxRowsInMemory()); +// Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment()); +// Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); +// Assert.assertEquals(0, config.getMaxPendingPersists()); +// Assert.assertEquals(new IndexSpec(), config.getIndexSpec()); +// Assert.assertEquals(true, config.getBuildV9Directly()); +// Assert.assertEquals(false, config.isReportParseExceptions()); +// Assert.assertEquals(0, config.getHandoffConditionTimeout()); +// } +// +// @Test +// public void testSerdeWithNonDefaults() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"basePersistDirectory\": \"/tmp/xxx\",\n" +// + " \"maxRowsInMemory\": 100,\n" +// + " \"maxRowsPerSegment\": 100,\n" +// + " \"intermediatePersistPeriod\": \"PT1H\",\n" +// + " \"maxPendingPersists\": 100,\n" +// + " \"buildV9Directly\": true,\n" +// + " \"reportParseExceptions\": true,\n" +// + " \"handoffConditionTimeout\": 100\n" +// + "}"; +// +// KinesisTuningConfig config = (KinesisTuningConfig) mapper.readValue( +// mapper.writeValueAsString( +// mapper.readValue( +// jsonStr, +// TuningConfig.class +// ) +// ), +// TuningConfig.class +// ); +// +// Assert.assertEquals(new File("/tmp/xxx"), config.getBasePersistDirectory()); +// Assert.assertEquals(100, config.getMaxRowsInMemory()); +// Assert.assertEquals(100, config.getMaxRowsPerSegment()); +// Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); +// Assert.assertEquals(100, config.getMaxPendingPersists()); +// Assert.assertEquals(true, config.getBuildV9Directly()); +// Assert.assertEquals(true, config.isReportParseExceptions()); +// Assert.assertEquals(100, config.getHandoffConditionTimeout()); +// } +// +// @Test +// public void testCopyOf() throws Exception +// { +// KinesisTuningConfig original = new KinesisTuningConfig(1, 2, new Period("PT3S"), new File("/tmp/xxx"), 4, new IndexSpec(), true, true, 5L, null); +// KinesisTuningConfig copy = KinesisTuningConfig.copyOf(original); +// +// Assert.assertEquals(1, copy.getMaxRowsInMemory()); +// Assert.assertEquals(2, copy.getMaxRowsPerSegment()); +// Assert.assertEquals(new Period("PT3S"), copy.getIntermediatePersistPeriod()); +// Assert.assertEquals(new File("/tmp/xxx"), copy.getBasePersistDirectory()); +// Assert.assertEquals(4, copy.getMaxPendingPersists()); +// Assert.assertEquals(new IndexSpec(), copy.getIndexSpec()); +// Assert.assertEquals(true, copy.getBuildV9Directly()); +// Assert.assertEquals(true, copy.isReportParseExceptions()); +// Assert.assertEquals(5L, copy.getHandoffConditionTimeout()); +// } +//} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java new file mode 100644 index 000000000000..8f9da95347d8 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java @@ -0,0 +1,161 @@ +///* +// * Licensed to Metamarkets Group Inc. (Metamarkets) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. Metamarkets licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, +// * software distributed under the License is distributed on an +// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// * KIND, either express or implied. See the License for the +// * specific language governing permissions and limitations +// * under the License. +// */ +// +//package org.apache.druid.indexing.kinesis.supervisor; +// +//import com.fasterxml.jackson.databind.JsonMappingException; +//import com.fasterxml.jackson.databind.Module; +//import com.fasterxml.jackson.databind.ObjectMapper; +//import com.google.common.collect.ImmutableMap; +//import org.apache.druid.indexing.kinesis.KinesisIndexingServiceModule; +//import org.apache.druid.jackson.DefaultObjectMapper; +//import org.hamcrest.CoreMatchers; +//import org.joda.time.Duration; +//import org.junit.Assert; +//import org.junit.Rule; +//import org.junit.Test; +//import org.junit.rules.ExpectedException; +// +//public class KinesisSupervisorIOConfigTest +//{ +// private final ObjectMapper mapper; +// +// public KinesisSupervisorIOConfigTest() +// { +// mapper = new DefaultObjectMapper(); +// mapper.registerModules((Iterable) new KinesisIndexingServiceModule().getJacksonModules()); +// } +// +// @Rule +// public final ExpectedException exception = ExpectedException.none(); +// +// @Test +// public void testSerdeWithDefaults() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"topic\": \"my-topic\",\n" +// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"}\n" +// + "}"; +// +// KinesisSupervisorIOConfig config = mapper.readValue( +// mapper.writeValueAsString( +// mapper.readValue( +// jsonStr, +// KinesisSupervisorIOConfig.class +// ) +// ), KinesisSupervisorIOConfig.class +// ); +// +// Assert.assertEquals("my-topic", config.getStream()); +// Assert.assertEquals(1, (int) config.getReplicas()); +// Assert.assertEquals(1, (int) config.getTaskCount()); +// Assert.assertEquals(Duration.standardMinutes(60), config.getTaskDuration()); +// Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties()); +// Assert.assertEquals(Duration.standardSeconds(5), config.getStartDelay()); +// Assert.assertEquals(Duration.standardSeconds(30), config.getPeriod()); +// Assert.assertEquals(false, config.isUseEarliestOffset()); +// Assert.assertEquals(Duration.standardMinutes(30), config.getCompletionTimeout()); +// Assert.assertFalse("lateMessageRejectionPeriod", config.getLateMessageRejectionPeriod().isPresent()); +// Assert.assertFalse("skipOffsetGaps", config.isSkipOffsetGaps()); +// } +// +// @Test +// public void testSerdeWithNonDefaults() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"topic\": \"my-topic\",\n" +// + " \"replicas\": 3,\n" +// + " \"taskCount\": 9,\n" +// + " \"taskDuration\": \"PT30M\",\n" +// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" +// + " \"startDelay\": \"PT1M\",\n" +// + " \"period\": \"PT10S\",\n" +// + " \"useEarliestOffset\": true,\n" +// + " \"completionTimeout\": \"PT45M\",\n" +// + " \"lateMessageRejectionPeriod\": \"PT1H\",\n" +// + " \"skipOffsetGaps\": true\n" +// + "}"; +// +// KinesisSupervisorIOConfig config = mapper.readValue( +// mapper.writeValueAsString( +// mapper.readValue( +// jsonStr, +// KinesisSupervisorIOConfig.class +// ) +// ), KinesisSupervisorIOConfig.class +// ); +// +// Assert.assertEquals("my-topic", config.getStream()); +// Assert.assertEquals(3, (int) config.getReplicas()); +// Assert.assertEquals(9, (int) config.getTaskCount()); +// Assert.assertEquals(Duration.standardMinutes(30), config.getTaskDuration()); +// Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties()); +// Assert.assertEquals(Duration.standardMinutes(1), config.getStartDelay()); +// Assert.assertEquals(Duration.standardSeconds(10), config.getPeriod()); +// Assert.assertEquals(true, config.isUseEarliestOffset()); +// Assert.assertEquals(Duration.standardMinutes(45), config.getCompletionTimeout()); +// Assert.assertEquals(Duration.standardHours(1), config.getLateMessageRejectionPeriod().get()); +// Assert.assertTrue("skipOffsetGaps", config.isSkipOffsetGaps()); +// } +// +// @Test +// public void testTopicRequired() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"}\n" +// + "}"; +// +// exception.expect(JsonMappingException.class); +// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); +// exception.expectMessage(CoreMatchers.containsString("topic")); +// mapper.readValue(jsonStr, KinesisSupervisorIOConfig.class); +// } +// +// @Test +// public void testConsumerPropertiesRequired() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"topic\": \"my-topic\"\n" +// + "}"; +// +// exception.expect(JsonMappingException.class); +// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); +// exception.expectMessage(CoreMatchers.containsString("consumerProperties")); +// mapper.readValue(jsonStr, KinesisSupervisorIOConfig.class); +// } +// +// @Test +// public void testBootstrapServersRequired() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"topic\": \"my-topic\",\n" +// + " \"consumerProperties\": {}\n" +// + "}"; +// +// exception.expect(JsonMappingException.class); +// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); +// exception.expectMessage(CoreMatchers.containsString("bootstrap.servers")); +// mapper.readValue(jsonStr, KinesisSupervisorIOConfig.class); +// } +//} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java new file mode 100644 index 000000000000..5471dfc92e24 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -0,0 +1,1863 @@ +///* +// * Licensed to Metamarkets Group Inc. (Metamarkets) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. Metamarkets licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, +// * software distributed under the License is distributed on an +// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// * KIND, either express or implied. See the License for the +// * specific language governing permissions and limitations +// * under the License. +// */ +// +//package org.apache.druid.indexing.kinesis.supervisor; +// +//import com.fasterxml.jackson.databind.ObjectMapper; +//import com.google.common.base.Charsets; +//import com.google.common.base.Optional; +//import com.google.common.collect.ImmutableList; +//import com.google.common.collect.ImmutableMap; +//import com.google.common.util.concurrent.Futures; +//import com.google.common.util.concurrent.ListenableFuture; +//import org.apache.druid.data.input.impl.DimensionSchema; +//import org.apache.druid.data.input.impl.DimensionsSpec; +//import org.apache.druid.data.input.impl.JSONParseSpec; +//import org.apache.druid.data.input.impl.JSONPathFieldSpec; +//import org.apache.druid.data.input.impl.JSONPathSpec; +//import org.apache.druid.data.input.impl.StringDimensionSchema; +//import org.apache.druid.data.input.impl.StringInputRowParser; +//import org.apache.druid.data.input.impl.TimestampSpec; +//import org.apache.druid.indexing.common.TaskInfoProvider; +//import org.apache.druid.indexing.common.TaskLocation; +//import org.apache.druid.indexing.common.TaskStatus; +//import org.apache.druid.indexing.common.task.RealtimeIndexTask; +//import org.apache.druid.indexing.common.task.Task; +//import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata; +//import org.apache.druid.indexing.kinesis.KinesisIOConfig; +//import org.apache.druid.indexing.kinesis.KinesisIndexTask; +//import org.apache.druid.indexing.kinesis.KinesisIndexTaskClient; +//import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; +//import org.apache.druid.indexing.kinesis.KinesisPartitions; +//import org.apache.druid.indexing.kinesis.KinesisTuningConfig; +//import org.apache.druid.indexing.kinesis.test.TestBroker; +//import org.apache.druid.indexing.overlord.DataSourceMetadata; +//import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +//import org.apache.druid.indexing.overlord.TaskMaster; +//import org.apache.druid.indexing.overlord.TaskQueue; +//import org.apache.druid.indexing.overlord.TaskRunner; +//import org.apache.druid.indexing.overlord.TaskRunnerListener; +//import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; +//import org.apache.druid.indexing.overlord.TaskStorage; +//import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +//import org.apache.druid.jackson.DefaultObjectMapper; +//import org.apache.druid.java.util.common.ISE; +//import org.apache.druid.java.util.common.granularity.Granularities; +//import org.apache.druid.query.aggregation.AggregatorFactory; +//import org.apache.druid.query.aggregation.CountAggregatorFactory; +//import org.apache.druid.segment.indexing.DataSchema; +//import org.apache.druid.segment.indexing.RealtimeIOConfig; +//import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +//import org.apache.druid.segment.realtime.FireDepartment; +//import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; +//import org.apache.druid.server.metrics.NoopServiceEmitter; +//import org.apache.curator.test.TestingCluster; +//import org.apache.kafka.clients.producer.KafkaProducer; +//import org.apache.kafka.clients.producer.ProducerRecord; +//import org.easymock.Capture; +//import org.easymock.CaptureType; +//import org.easymock.EasyMock; +//import org.easymock.EasyMockSupport; +//import org.joda.time.DateTime; +//import org.joda.time.Duration; +//import org.joda.time.Interval; +//import org.joda.time.Period; +//import org.junit.After; +//import org.junit.Assert; +//import org.junit.Before; +//import org.junit.Rule; +//import org.junit.Test; +//import org.junit.rules.TemporaryFolder; +//import org.junit.runner.RunWith; +//import org.junit.runners.Parameterized; +// +//import java.io.File; +//import java.util.ArrayList; +//import java.util.Collection; +//import java.util.Collections; +//import java.util.List; +//import java.util.Map; +//import java.util.concurrent.Executor; +// +//import static org.easymock.EasyMock.anyObject; +//import static org.easymock.EasyMock.anyString; +//import static org.easymock.EasyMock.capture; +//import static org.easymock.EasyMock.eq; +//import static org.easymock.EasyMock.expect; +//import static org.easymock.EasyMock.expectLastCall; +//import static org.easymock.EasyMock.replay; +//import static org.easymock.EasyMock.reset; +// +//@RunWith(Parameterized.class) +//public class KinesisSupervisorTest extends EasyMockSupport +//{ +// private static final ObjectMapper objectMapper = new DefaultObjectMapper(); +// private static final String KAFKA_TOPIC = "testTopic"; +// private static final String DATASOURCE = "testDS"; +// private static final int NUM_PARTITIONS = 3; +// private static final int TEST_CHAT_THREADS = 3; +// private static final long TEST_CHAT_RETRIES = 9L; +// private static final Period TEST_HTTP_TIMEOUT = new Period("PT10S"); +// private static final Period TEST_SHUTDOWN_TIMEOUT = new Period("PT80S"); +// +// private int numThreads; +// private TestingCluster zkServer; +// private TestBroker kafkaServer; +// private KinesisSupervisor supervisor; +// private String kafkaHost; +// private DataSchema dataSchema; +// private KinesisSupervisorTuningConfig tuningConfig; +// private TaskStorage taskStorage; +// private TaskMaster taskMaster; +// private TaskRunner taskRunner; +// private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; +// private KinesisIndexTaskClient taskClient; +// private TaskQueue taskQueue; +// +// @Rule +// public final TemporaryFolder tempFolder = new TemporaryFolder(); +// +// @Parameterized.Parameters(name = "numThreads = {0}") +// public static Iterable constructorFeeder() +// { +// return ImmutableList.of(new Object[]{1}, new Object[]{8}); +// } +// +// public KinesisSupervisorTest(int numThreads) +// { +// this.numThreads = numThreads; +// } +// +// @Before +// public void setUp() throws Exception +// { +// taskStorage = createMock(TaskStorage.class); +// taskMaster = createMock(TaskMaster.class); +// taskRunner = createMock(TaskRunner.class); +// indexerMetadataStorageCoordinator = createMock(IndexerMetadataStorageCoordinator.class); +// taskClient = createMock(KinesisIndexTaskClient.class); +// taskQueue = createMock(TaskQueue.class); +// +// zkServer = new TestingCluster(1); +// zkServer.start(); +// +// kafkaServer = new TestBroker( +// zkServer.getConnectString(), +// tempFolder.newFolder(), +// 1, +// ImmutableMap.of("num.partitions", String.valueOf(NUM_PARTITIONS)) +// ); +// kafkaServer.start(); +// kafkaHost = String.format("localhost:%d", kafkaServer.getPort()); +// +// dataSchema = getDataSchema(DATASOURCE); +// tuningConfig = new KinesisSupervisorTuningConfig( +// 1000, +// 50000, +// new Period("P1Y"), +// new File("/test"), +// null, +// null, +// true, +// false, +// null, +// null, +// numThreads, +// TEST_CHAT_THREADS, +// TEST_CHAT_RETRIES, +// TEST_HTTP_TIMEOUT, +// TEST_SHUTDOWN_TIMEOUT +// ); +// } +// +// @After +// public void tearDown() throws Exception +// { +// kafkaServer.close(); +// kafkaServer = null; +// +// zkServer.stop(); +// zkServer = null; +// +// supervisor = null; +// } +// +// @Test +// public void testNoInitialState() throws Exception +// { +// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// addSomeEvents(1); +// +// Capture captured = Capture.newInstance(); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// KinesisIndexTask task = captured.getValue(); +// Assert.assertEquals(dataSchema, task.getDataSchema()); +// Assert.assertEquals(KinesisTuningConfig.copyOf(tuningConfig), task.getTuningConfig()); +// +// KinesisIOConfig taskConfig = task.getIOConfig(); +// Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); +// Assert.assertEquals("myCustomValue", taskConfig.getConsumerProperties().get("myCustomKey")); +// Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); +// Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); +// Assert.assertFalse("pauseAfterRead", taskConfig.isPauseAfterRead()); +// Assert.assertFalse("minimumMessageTime", taskConfig.getMinimumMessageTime().isPresent()); +// Assert.assertFalse("skipOffsetGaps", taskConfig.isSkipOffsetGaps()); +// +// Assert.assertEquals(KAFKA_TOPIC, taskConfig.getStartPartitions().getStream()); +// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); +// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); +// +// Assert.assertEquals(KAFKA_TOPIC, taskConfig.getEndPartitions().getStream()); +// Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)); +// Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)); +// } +// +// @Test +// public void testSkipOffsetGaps() throws Exception +// { +// supervisor = getSupervisor(1, 1, true, "PT1H", null, true); +// addSomeEvents(1); +// +// Capture captured = Capture.newInstance(); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// KinesisIndexTask task = captured.getValue(); +// KinesisIOConfig taskConfig = task.getIOConfig(); +// +// Assert.assertTrue("skipOffsetGaps", taskConfig.isSkipOffsetGaps()); +// } +// +// @Test +// public void testMultiTask() throws Exception +// { +// supervisor = getSupervisor(1, 2, true, "PT1H", null, false); +// addSomeEvents(1); +// +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true).times(2); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// KinesisIndexTask task1 = captured.getValues().get(0); +// Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); +// Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); +// Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(Long.MAX_VALUE, (long) task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2)); +// Assert.assertEquals(Long.MAX_VALUE, (long) task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(2)); +// +// KinesisIndexTask task2 = captured.getValues().get(1); +// Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); +// Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); +// Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1)); +// Assert.assertEquals(Long.MAX_VALUE, (long) task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(1)); +// } +// +// @Test +// public void testReplicas() throws Exception +// { +// supervisor = getSupervisor(2, 1, true, "PT1H", null, false); +// addSomeEvents(1); +// +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true).times(2); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// KinesisIndexTask task1 = captured.getValues().get(0); +// Assert.assertEquals(3, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); +// Assert.assertEquals(3, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); +// Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1)); +// Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2)); +// +// KinesisIndexTask task2 = captured.getValues().get(1); +// Assert.assertEquals(3, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); +// Assert.assertEquals(3, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); +// Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1)); +// Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2)); +// } +// +// @Test +// public void testLateMessageRejectionPeriod() throws Exception +// { +// supervisor = getSupervisor(2, 1, true, "PT1H", new Period("PT1H"), false); +// addSomeEvents(1); +// +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true).times(2); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// KinesisIndexTask task1 = captured.getValues().get(0); +// KinesisIndexTask task2 = captured.getValues().get(1); +// +// Assert.assertTrue( +// "minimumMessageTime", +// task1.getIOConfig().getMinimumMessageTime().get().plusMinutes(59).isBeforeNow() +// ); +// Assert.assertTrue( +// "minimumMessageTime", +// task1.getIOConfig().getMinimumMessageTime().get().plusMinutes(61).isAfterNow() +// ); +// Assert.assertEquals( +// task1.getIOConfig().getMinimumMessageTime().get(), +// task2.getIOConfig().getMinimumMessageTime().get() +// ); +// } +// +// @Test +// /** +// * Test generating the starting offsets from the partition high water marks in Kafka. +// */ +// public void testLatestOffset() throws Exception +// { +// supervisor = getSupervisor(1, 1, false, "PT1H", null, false); +// addSomeEvents(1100); +// +// Capture captured = Capture.newInstance(); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// KinesisIndexTask task = captured.getValue(); +// Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1)); +// Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2)); +// } +// +// @Test +// /** +// * Test generating the starting offsets from the partition data stored in druid_dataSource which contains the +// * offsets of the last built segments. +// */ +// public void testDatasourceMetadata() throws Exception +// { +// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// addSomeEvents(100); +// +// Capture captured = Capture.newInstance(); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// new KinesisPartitions(KAFKA_TOPIC, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) +// ) +// ).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// KinesisIndexTask task = captured.getValue(); +// KinesisIOConfig taskConfig = task.getIOConfig(); +// Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); +// Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); +// Assert.assertEquals(30L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); +// } +// +// @Test(expected = ISE.class) +// public void testBadMetadataOffsets() throws Exception +// { +// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// addSomeEvents(1); +// +// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// new KinesisPartitions(KAFKA_TOPIC, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) +// ) +// ).anyTimes(); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// } +// +// @Test +// public void testKillIncompatibleTasks() throws Exception +// { +// supervisor = getSupervisor(2, 1, true, "PT1H", null, false); +// addSomeEvents(1); +// +// Task id1 = createKafkaIndexTask( // unexpected # of partitions (kill) +// "id1", +// DATASOURCE, +// "index_kinesis_testDS__some_other_sequenceName", +// new KinesisPartitions("topic", ImmutableMap.of(0, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, 10L)), +// null +// ); +// +// Task id2 = createKafkaIndexTask( // correct number of partitions and ranges (don't kill) +// "id2", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 333L)), +// null +// ); +// +// Task id3 = createKafkaIndexTask( // unexpected range on partition 2 (kill) +// "id3", +// DATASOURCE, +// "index_kinesis_testDS__some_other_sequenceName", +// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 1L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 330L)), +// null +// ); +// +// Task id4 = createKafkaIndexTask( // different datasource (don't kill) +// "id4", +// "other-datasource", +// "index_kinesis_testDS_d927edff33c4b3f", +// new KinesisPartitions("topic", ImmutableMap.of(0, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, 10L)), +// null +// ); +// +// Task id5 = new RealtimeIndexTask( // non KinesisIndexTask (don't kill) +// "id5", +// null, +// new FireDepartment( +// dataSchema, +// new RealtimeIOConfig(null, null, null), +// null +// ), +// null +// ); +// +// List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); +// +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.NOT_STARTED)) +// .anyTimes(); +// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); +// expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(false)); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// taskQueue.shutdown("id3"); +// +// expect(taskQueue.add(anyObject(Task.class))).andReturn(true); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// } +// +// @Test +// public void testKillBadPartitionAssignment() throws Exception +// { +// supervisor = getSupervisor(1, 2, true, "PT1H", null, false); +// addSomeEvents(1); +// +// Task id1 = createKafkaIndexTask( +// "id1", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null +// ); +// Task id2 = createKafkaIndexTask( +// "id2", +// DATASOURCE, +// "sequenceName-1", +// new KinesisPartitions("topic", ImmutableMap.of(1, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE)), +// null +// ); +// Task id3 = createKafkaIndexTask( +// "id3", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null +// ); +// Task id4 = createKafkaIndexTask( +// "id4", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE)), +// null +// ); +// Task id5 = createKafkaIndexTask( +// "id5", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(1, 0L, 2, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null +// ); +// +// List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); +// +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); +// expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).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(taskStorage.getTask("id4")).andReturn(Optional.of(id3)).anyTimes(); +// expect(taskStorage.getTask("id5")).andReturn(Optional.of(id3)).anyTimes(); +// expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.NOT_STARTED)) +// .anyTimes(); +// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); +// expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); +// expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// taskQueue.shutdown("id4"); +// taskQueue.shutdown("id5"); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// } +// +// @Test +// public void testRequeueTaskWhenFailed() throws Exception +// { +// supervisor = getSupervisor(2, 2, true, "PT1H", null, false); +// addSomeEvents(1); +// +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.NOT_STARTED)) +// .anyTimes(); +// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true).times(4); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// List tasks = captured.getValues(); +// +// // test that running the main loop again checks the status of the tasks that were created and does nothing if they +// // are all still running +// reset(taskStorage); +// expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); +// for (Task task : tasks) { +// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); +// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); +// } +// replay(taskStorage); +// +// supervisor.runInternal(); +// verifyAll(); +// +// // test that a task failing causes a new task to be re-queued with the same parameters +// Capture aNewTaskCapture = Capture.newInstance(); +// List imStillAlive = tasks.subList(0, 3); +// KinesisIndexTask iHaveFailed = (KinesisIndexTask) tasks.get(3); +// reset(taskStorage); +// reset(taskQueue); +// expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes(); +// for (Task task : imStillAlive) { +// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); +// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); +// } +// expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); +// expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); +// expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); +// replay(taskStorage); +// replay(taskQueue); +// +// supervisor.runInternal(); +// verifyAll(); +// +// Assert.assertNotEquals(iHaveFailed.getId(), aNewTaskCapture.getValue().getId()); +// Assert.assertEquals( +// iHaveFailed.getIOConfig().getBaseSequenceName(), +// ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getBaseSequenceName() +// ); +// } +// +// @Test +// public void testRequeueAdoptedTaskWhenFailed() throws Exception +// { +// supervisor = getSupervisor(2, 1, true, "PT1H", null, false); +// addSomeEvents(1); +// +// DateTime now = DateTime.now(); +// Task id1 = createKafkaIndexTask( +// "id1", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// now +// ); +// +// List existingTasks = ImmutableList.of(id1); +// +// Capture captured = Capture.newInstance(); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); +// expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); +// expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); +// expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// // check that replica tasks are created with the same minimumMessageTime as tasks inherited from another supervisor +// Assert.assertEquals(now, ((KinesisIndexTask) captured.getValue()).getIOConfig().getMinimumMessageTime().get()); +// +// // test that a task failing causes a new task to be re-queued with the same parameters +// String runningTaskId = captured.getValue().getId(); +// Capture aNewTaskCapture = Capture.newInstance(); +// KinesisIndexTask iHaveFailed = (KinesisIndexTask) existingTasks.get(0); +// reset(taskStorage); +// reset(taskQueue); +// reset(taskClient); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes(); +// expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); +// expect(taskStorage.getStatus(runningTaskId)).andReturn(Optional.of(TaskStatus.running(runningTaskId))).anyTimes(); +// expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); +// expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); +// expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); +// expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes(); +// expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); +// replay(taskStorage); +// replay(taskQueue); +// replay(taskClient); +// +// supervisor.runInternal(); +// verifyAll(); +// +// Assert.assertNotEquals(iHaveFailed.getId(), aNewTaskCapture.getValue().getId()); +// Assert.assertEquals( +// iHaveFailed.getIOConfig().getBaseSequenceName(), +// ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getBaseSequenceName() +// ); +// +// // check that failed tasks are recreated with the same minimumMessageTime as the task it replaced, even if that +// // task came from another supervisor +// Assert.assertEquals(now, ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getMinimumMessageTime().get()); +// } +// +// @Test +// public void testQueueNextTasksOnSuccess() throws Exception +// { +// supervisor = getSupervisor(2, 2, true, "PT1H", null, false); +// addSomeEvents(1); +// +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.NOT_STARTED)) +// .anyTimes(); +// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true).times(4); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// List tasks = captured.getValues(); +// +// reset(taskStorage); +// expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); +// for (Task task : tasks) { +// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); +// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); +// } +// replay(taskStorage); +// +// supervisor.runInternal(); +// verifyAll(); +// +// // test that a task succeeding causes a new task to be re-queued with the next offset range and causes any replica +// // tasks to be shutdown +// Capture newTasksCapture = Capture.newInstance(CaptureType.ALL); +// Capture shutdownTaskIdCapture = Capture.newInstance(); +// List imStillRunning = tasks.subList(1, 4); +// KinesisIndexTask iAmSuccess = (KinesisIndexTask) tasks.get(0); +// reset(taskStorage); +// reset(taskQueue); +// reset(taskClient); +// expect(taskStorage.getActiveTasks()).andReturn(imStillRunning).anyTimes(); +// for (Task task : imStillRunning) { +// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); +// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); +// } +// expect(taskStorage.getStatus(iAmSuccess.getId())).andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId()))); +// expect(taskStorage.getTask(iAmSuccess.getId())).andReturn(Optional.of((Task) iAmSuccess)).anyTimes(); +// expect(taskQueue.add(capture(newTasksCapture))).andReturn(true).times(2); +// expect(taskClient.stopAsync(capture(shutdownTaskIdCapture), eq(false))).andReturn(Futures.immediateFuture(true)); +// replay(taskStorage); +// replay(taskQueue); +// replay(taskClient); +// +// supervisor.runInternal(); +// verifyAll(); +// +// // make sure we killed the right task (sequenceName for replicas are the same) +// Assert.assertTrue(shutdownTaskIdCapture.getValue().contains(iAmSuccess.getIOConfig().getBaseSequenceName())); +// } +// +// @Test +// public void testBeginPublishAndQueueNextTasks() throws Exception +// { +// final TaskLocation location = new TaskLocation("testHost", 1234); +// +// supervisor = getSupervisor(2, 2, true, "PT1M", null, false); +// addSomeEvents(100); +// +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true).times(4); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// List tasks = captured.getValues(); +// Collection workItems = new ArrayList<>(); +// for (Task task : tasks) { +// workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); +// } +// +// reset(taskStorage, taskRunner, taskClient, taskQueue); +// captured = Capture.newInstance(CaptureType.ALL); +// expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); +// for (Task task : tasks) { +// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); +// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); +// } +// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); +// expect(taskClient.getStatusAsync(anyString())) +// .andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)) +// .anyTimes(); +// expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) +// .andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2))) +// .andReturn(Futures.immediateFuture(DateTime.now())); +// expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) +// .andReturn(Futures.immediateFuture(DateTime.now())) +// .times(2); +// expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); +// expect( +// taskClient.setEndOffsetsAsync( +// EasyMock.contains("sequenceName-0"), +// EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), +// EasyMock.eq(true) +// ) +// ).andReturn(Futures.immediateFuture(true)).times(2); +// expect(taskQueue.add(capture(captured))).andReturn(true).times(2); +// +// replay(taskStorage, taskRunner, taskClient, taskQueue); +// +// supervisor.runInternal(); +// verifyAll(); +// +// for (Task task : captured.getValues()) { +// KinesisIndexTask kinesisIndexTask = (KinesisIndexTask) task; +// Assert.assertEquals(dataSchema, kinesisIndexTask.getDataSchema()); +// Assert.assertEquals(KinesisTuningConfig.copyOf(tuningConfig), kinesisIndexTask.getTuningConfig()); +// +// KinesisIOConfig taskConfig = kinesisIndexTask.getIOConfig(); +// Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); +// Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); +// Assert.assertFalse("pauseAfterRead", taskConfig.isPauseAfterRead()); +// +// Assert.assertEquals(KAFKA_TOPIC, taskConfig.getStartPartitions().getStream()); +// Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); +// Assert.assertEquals(35L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); +// } +// } +// +// @Test +// public void testDiscoverExistingPublishingTask() throws Exception +// { +// final TaskLocation location = new TaskLocation("testHost", 1234); +// +// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// addSomeEvents(1); +// +// Task task = createKafkaIndexTask( +// "id1", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null +// ); +// +// Collection workItems = new ArrayList<>(); +// workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); +// +// Capture captured = Capture.newInstance(); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); +// expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); +// expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.PUBLISHING)); +// expect(taskClient.getCurrentOffsetsAsync("id1", false)) +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); +// expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); +// expect(taskQueue.add(capture(captured))).andReturn(true); +// +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// SupervisorReport report = supervisor.getStatus(); +// verifyAll(); +// +// Assert.assertEquals(DATASOURCE, report.getId()); +// Assert.assertTrue(report.getPayload() instanceof KinesisSupervisorReport.KinesisSupervisorReportPayload); +// +// KinesisSupervisorReport.KinesisSupervisorReportPayload payload = (KinesisSupervisorReport.KinesisSupervisorReportPayload) +// report.getPayload(); +// +// Assert.assertEquals(DATASOURCE, payload.getDataSource()); +// Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); +// Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); +// Assert.assertEquals(1, (int) payload.getReplicas()); +// Assert.assertEquals(KAFKA_TOPIC, payload.getTopic()); +// Assert.assertEquals(0, payload.getActiveTasks().size()); +// Assert.assertEquals(1, payload.getPublishingTasks().size()); +// +// TaskReportData publishingReport = payload.getPublishingTasks().get(0); +// +// Assert.assertEquals("id1", publishingReport.getId()); +// Assert.assertEquals(ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), publishingReport.getStartingOffsets()); +// Assert.assertEquals(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), publishingReport.getCurrentOffsets()); +// +// KinesisIndexTask capturedTask = captured.getValue(); +// Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); +// Assert.assertEquals(KinesisTuningConfig.copyOf(tuningConfig), capturedTask.getTuningConfig()); +// +// KinesisIOConfig capturedTaskConfig = capturedTask.getIOConfig(); +// Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); +// Assert.assertEquals("myCustomValue", capturedTaskConfig.getConsumerProperties().get("myCustomKey")); +// Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); +// Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); +// Assert.assertFalse("pauseAfterRead", capturedTaskConfig.isPauseAfterRead()); +// +// // check that the new task was created with starting offsets matching where the publishing task finished +// Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getStartPartitions().getStream()); +// Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(20L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); +// Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); +// +// Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getEndPartitions().getStream()); +// Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)); +// Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)); +// } +// +// @Test +// public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() throws Exception +// { +// final TaskLocation location = new TaskLocation("testHost", 1234); +// +// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// addSomeEvents(1); +// +// Task task = createKafkaIndexTask( +// "id1", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null +// ); +// +// Collection workItems = new ArrayList<>(); +// workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); +// +// Capture captured = Capture.newInstance(); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); +// expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); +// expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.PUBLISHING)); +// expect(taskClient.getCurrentOffsetsAsync("id1", false)) +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 2, 30L))); +// expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); +// expect(taskQueue.add(capture(captured))).andReturn(true); +// +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// SupervisorReport report = supervisor.getStatus(); +// verifyAll(); +// +// Assert.assertEquals(DATASOURCE, report.getId()); +// Assert.assertTrue(report.getPayload() instanceof KinesisSupervisorReport.KinesisSupervisorReportPayload); +// +// KinesisSupervisorReport.KinesisSupervisorReportPayload payload = (KinesisSupervisorReport.KinesisSupervisorReportPayload) +// report.getPayload(); +// +// Assert.assertEquals(DATASOURCE, payload.getDataSource()); +// Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); +// Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); +// Assert.assertEquals(1, (int) payload.getReplicas()); +// Assert.assertEquals(KAFKA_TOPIC, payload.getTopic()); +// Assert.assertEquals(0, payload.getActiveTasks().size()); +// Assert.assertEquals(1, payload.getPublishingTasks().size()); +// +// TaskReportData publishingReport = payload.getPublishingTasks().get(0); +// +// Assert.assertEquals("id1", publishingReport.getId()); +// Assert.assertEquals(ImmutableMap.of(0, 0L, 2, 0L), publishingReport.getStartingOffsets()); +// Assert.assertEquals(ImmutableMap.of(0, 10L, 2, 30L), publishingReport.getCurrentOffsets()); +// +// KinesisIndexTask capturedTask = captured.getValue(); +// Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); +// Assert.assertEquals(KinesisTuningConfig.copyOf(tuningConfig), capturedTask.getTuningConfig()); +// +// KinesisIOConfig capturedTaskConfig = capturedTask.getIOConfig(); +// Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); +// Assert.assertEquals("myCustomValue", capturedTaskConfig.getConsumerProperties().get("myCustomKey")); +// Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); +// Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); +// Assert.assertFalse("pauseAfterRead", capturedTaskConfig.isPauseAfterRead()); +// +// // check that the new task was created with starting offsets matching where the publishing task finished +// Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getStartPartitions().getStream()); +// Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(0L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); +// Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); +// +// Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getEndPartitions().getStream()); +// Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)); +// Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)); +// } +// +// @Test +// public void testDiscoverExistingPublishingAndReadingTask() throws Exception +// { +// final TaskLocation location1 = new TaskLocation("testHost", 1234); +// final TaskLocation location2 = new TaskLocation("testHost2", 145); +// final DateTime startTime = new DateTime(); +// +// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// addSomeEvents(1); +// +// Task id1 = createKafkaIndexTask( +// "id1", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null +// ); +// +// Task id2 = createKafkaIndexTask( +// "id2", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null +// ); +// +// Collection workItems = new ArrayList<>(); +// workItems.add(new TestTaskRunnerWorkItem(id1.getId(), null, location1)); +// workItems.add(new TestTaskRunnerWorkItem(id2.getId(), null, location2)); +// +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2)).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.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); +// expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.PUBLISHING)); +// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); +// expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); +// expect(taskClient.getCurrentOffsetsAsync("id1", false)) +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); +// expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); +// expect(taskClient.getCurrentOffsetsAsync("id2", false)) +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 40L, 1, 50L, 2, 60L))); +// +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// SupervisorReport report = supervisor.getStatus(); +// verifyAll(); +// +// Assert.assertEquals(DATASOURCE, report.getId()); +// Assert.assertTrue(report.getPayload() instanceof KinesisSupervisorReport.KinesisSupervisorReportPayload); +// +// KinesisSupervisorReport.KinesisSupervisorReportPayload payload = (KinesisSupervisorReport.KinesisSupervisorReportPayload) +// report.getPayload(); +// +// Assert.assertEquals(DATASOURCE, payload.getDataSource()); +// Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); +// Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); +// Assert.assertEquals(1, (int) payload.getReplicas()); +// Assert.assertEquals(KAFKA_TOPIC, payload.getTopic()); +// Assert.assertEquals(1, payload.getActiveTasks().size()); +// Assert.assertEquals(1, payload.getPublishingTasks().size()); +// +// TaskReportData activeReport = payload.getActiveTasks().get(0); +// TaskReportData publishingReport = payload.getPublishingTasks().get(0); +// +// Assert.assertEquals("id2", activeReport.getId()); +// Assert.assertEquals(startTime, activeReport.getStartTime()); +// Assert.assertEquals(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), activeReport.getStartingOffsets()); +// Assert.assertEquals(ImmutableMap.of(0, 40L, 1, 50L, 2, 60L), activeReport.getCurrentOffsets()); +// +// Assert.assertEquals("id1", publishingReport.getId()); +// Assert.assertEquals(ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), publishingReport.getStartingOffsets()); +// Assert.assertEquals(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), publishingReport.getCurrentOffsets()); +// } +// +// @Test +// public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception +// { +// supervisor = getSupervisor(2, 2, true, "PT1H", null, false); +// addSomeEvents(1); +// +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true).times(4); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// List tasks = captured.getValues(); +// +// reset(taskStorage, taskClient, taskQueue); +// expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); +// for (Task task : tasks) { +// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); +// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); +// expect(taskClient.getStatusAsync(task.getId())) +// .andReturn(Futures.immediateFuture(KinesisIndexTask.Status.NOT_STARTED)); +// expect(taskClient.getStartTimeAsync(task.getId())) +// .andReturn(Futures.immediateFailedFuture(new RuntimeException())); +// taskQueue.shutdown(task.getId()); +// } +// replay(taskStorage, taskClient, taskQueue); +// +// supervisor.runInternal(); +// verifyAll(); +// } +// +// @Test +// public void testKillUnresponsiveTasksWhilePausing() throws Exception +// { +// final TaskLocation location = new TaskLocation("testHost", 1234); +// +// supervisor = getSupervisor(2, 2, true, "PT1M", null, false); +// addSomeEvents(100); +// +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true).times(4); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// List tasks = captured.getValues(); +// Collection workItems = new ArrayList<>(); +// for (Task task : tasks) { +// workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); +// } +// +// reset(taskStorage, taskRunner, taskClient, taskQueue); +// captured = Capture.newInstance(CaptureType.ALL); +// expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); +// for (Task task : tasks) { +// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); +// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); +// } +// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); +// expect(taskClient.getStatusAsync(anyString())) +// .andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)) +// .anyTimes(); +// expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) +// .andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2))) +// .andReturn(Futures.immediateFuture(DateTime.now())); +// expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) +// .andReturn(Futures.immediateFuture(DateTime.now())) +// .times(2); +// expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) +// .andReturn(Futures.>immediateFailedFuture(new RuntimeException())).times(2); +// taskQueue.shutdown(EasyMock.contains("sequenceName-0")); +// expectLastCall().times(2); +// expect(taskQueue.add(capture(captured))).andReturn(true).times(2); +// +// replay(taskStorage, taskRunner, taskClient, taskQueue); +// +// supervisor.runInternal(); +// verifyAll(); +// +// for (Task task : captured.getValues()) { +// KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); +// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); +// } +// } +// +// @Test +// public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception +// { +// final TaskLocation location = new TaskLocation("testHost", 1234); +// +// supervisor = getSupervisor(2, 2, true, "PT1M", null, false); +// addSomeEvents(100); +// +// Capture captured = Capture.newInstance(CaptureType.ALL); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskQueue.add(capture(captured))).andReturn(true).times(4); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// List tasks = captured.getValues(); +// Collection workItems = new ArrayList<>(); +// for (Task task : tasks) { +// workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); +// } +// +// reset(taskStorage, taskRunner, taskClient, taskQueue); +// captured = Capture.newInstance(CaptureType.ALL); +// expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); +// for (Task task : tasks) { +// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); +// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); +// } +// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); +// expect(taskClient.getStatusAsync(anyString())) +// .andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)) +// .anyTimes(); +// expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) +// .andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2))) +// .andReturn(Futures.immediateFuture(DateTime.now())); +// expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) +// .andReturn(Futures.immediateFuture(DateTime.now())) +// .times(2); +// expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); +// expect( +// taskClient.setEndOffsetsAsync( +// EasyMock.contains("sequenceName-0"), +// EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), +// EasyMock.eq(true) +// ) +// ).andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); +// taskQueue.shutdown(EasyMock.contains("sequenceName-0")); +// expectLastCall().times(2); +// expect(taskQueue.add(capture(captured))).andReturn(true).times(2); +// +// replay(taskStorage, taskRunner, taskClient, taskQueue); +// +// supervisor.runInternal(); +// verifyAll(); +// +// for (Task task : captured.getValues()) { +// KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); +// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); +// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); +// } +// } +// +// @Test(expected = IllegalStateException.class) +// public void testStopNotStarted() throws Exception +// { +// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// supervisor.stop(false); +// } +// +// @Test +// public void testStop() throws Exception +// { +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// taskClient.close(); +// taskRunner.unregisterListener(String.format("KinesisSupervisor-%s", DATASOURCE)); +// replayAll(); +// +// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// supervisor.start(); +// supervisor.stop(false); +// +// verifyAll(); +// } +// +// @Test +// public void testStopGracefully() throws Exception +// { +// final TaskLocation location1 = new TaskLocation("testHost", 1234); +// final TaskLocation location2 = new TaskLocation("testHost2", 145); +// final DateTime startTime = new DateTime(); +// +// supervisor = getSupervisor(2, 1, true, "PT1H", null, false); +// addSomeEvents(1); +// +// Task id1 = createKafkaIndexTask( +// "id1", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null +// ); +// +// Task id2 = createKafkaIndexTask( +// "id2", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null +// ); +// +// Task id3 = createKafkaIndexTask( +// "id3", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null +// ); +// +// Collection workItems = new ArrayList<>(); +// workItems.add(new TestTaskRunnerWorkItem(id1.getId(), null, location1)); +// workItems.add(new TestTaskRunnerWorkItem(id2.getId(), null, location2)); +// +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(workItems).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 KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.PUBLISHING)); +// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); +// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); +// expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); +// expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); +// expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); +// +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// reset(taskRunner, taskClient, taskQueue); +// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); +// expect(taskClient.pauseAsync("id2")) +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); +// expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) +// .andReturn(Futures.immediateFuture(true)); +// taskQueue.shutdown("id3"); +// expectLastCall().times(2); +// +// replay(taskRunner, taskClient, taskQueue); +// +// supervisor.gracefulShutdownInternal(); +// verifyAll(); +// } +// +// @Test +// public void testResetNoTasks() throws Exception +// { +// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// reset(indexerMetadataStorageCoordinator); +// expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); +// replay(indexerMetadataStorageCoordinator); +// +// supervisor.resetInternal(null); +// verifyAll(); +// +// } +// +// @Test +// public void testResetDataSourceMetadata() throws Exception +// { +// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// Capture captureDataSource = EasyMock.newCapture(); +// Capture captureDataSourceMetadata = EasyMock.newCapture(); +// +// KinesisDataSourceMetadata kinesisDataSourceMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( +// KAFKA_TOPIC, +// ImmutableMap.of(0, 1000L, 1, 1000L, 2, 1000L) +// )); +// +// KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( +// KAFKA_TOPIC, +// ImmutableMap.of(1, 1000L, 2, 1000L) +// )); +// +// KinesisDataSourceMetadata expectedMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( +// KAFKA_TOPIC, +// ImmutableMap.of(0, 1000L) +// )); +// +// reset(indexerMetadataStorageCoordinator); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(kinesisDataSourceMetadata); +// expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata( +// EasyMock.capture(captureDataSource), +// EasyMock.capture(captureDataSourceMetadata) +// )).andReturn(true); +// replay(indexerMetadataStorageCoordinator); +// +// supervisor.resetInternal(resetMetadata); +// verifyAll(); +// +// Assert.assertEquals(captureDataSource.getValue(), DATASOURCE); +// Assert.assertEquals(captureDataSourceMetadata.getValue(), expectedMetadata); +// } +// +// @Test +// public void testResetNoDataSourceMetadata() throws Exception +// { +// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( +// KAFKA_TOPIC, +// ImmutableMap.of(1, 1000L, 2, 1000L) +// )); +// +// reset(indexerMetadataStorageCoordinator); +// // no DataSourceMetadata in metadata store +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null); +// replay(indexerMetadataStorageCoordinator); +// +// supervisor.resetInternal(resetMetadata); +// verifyAll(); +// } +// +// @Test +// public void testResetRunningTasks() throws Exception +// { +// final TaskLocation location1 = new TaskLocation("testHost", 1234); +// final TaskLocation location2 = new TaskLocation("testHost2", 145); +// final DateTime startTime = new DateTime(); +// +// supervisor = getSupervisor(2, 1, true, "PT1H", null, false); +// addSomeEvents(1); +// +// Task id1 = createKafkaIndexTask( +// "id1", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null +// ); +// +// Task id2 = createKafkaIndexTask( +// "id2", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null +// ); +// +// Task id3 = createKafkaIndexTask( +// "id3", +// DATASOURCE, +// "sequenceName-0", +// new KinesisPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null +// ); +// +// Collection workItems = new ArrayList<>(); +// workItems.add(new TestTaskRunnerWorkItem(id1.getId(), null, location1)); +// workItems.add(new TestTaskRunnerWorkItem(id2.getId(), null, location2)); +// +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(workItems).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 KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.PUBLISHING)); +// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); +// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); +// expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); +// expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); +// expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); +// +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// reset(taskQueue, indexerMetadataStorageCoordinator); +// expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); +// taskQueue.shutdown("id2"); +// taskQueue.shutdown("id3"); +// replay(taskQueue, indexerMetadataStorageCoordinator); +// +// supervisor.resetInternal(null); +// verifyAll(); +// } +// +// private void addSomeEvents(int numEventsPerPartition) throws Exception +// { +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (int i = 0; i < NUM_PARTITIONS; i++) { +// for (int j = 0; j < numEventsPerPartition; j++) { +// kafkaProducer.send( +// new ProducerRecord( +// KAFKA_TOPIC, +// i, +// null, +// String.format("event-%d", j).getBytes() +// ) +// ).get(); +// } +// } +// } +// } +// +// private KinesisSupervisor getSupervisor( +// int replicas, +// int taskCount, +// boolean useEarliestOffset, +// String duration, +// Period lateMessageRejectionPeriod, +// boolean skipOffsetGaps +// ) +// { +// KinesisSupervisorIOConfig kafkaSupervisorIOConfig = new KinesisSupervisorIOConfig( +// KAFKA_TOPIC, +// replicas, +// taskCount, +// new Period(duration), +// ImmutableMap.of("myCustomKey", "myCustomValue", "bootstrap.servers", kafkaHost), +// new Period("P1D"), +// new Period("PT30S"), +// useEarliestOffset, +// new Period("PT30M"), +// lateMessageRejectionPeriod, +// skipOffsetGaps +// ); +// +// KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory(null, null) +// { +// @Override +// public KinesisIndexTaskClient build( +// TaskInfoProvider taskInfoProvider, +// String dataSource, +// int numThreads, +// Duration httpTimeout, +// long numRetries +// ) +// { +// Assert.assertEquals(TEST_CHAT_THREADS, numThreads); +// Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout); +// Assert.assertEquals(TEST_CHAT_RETRIES, numRetries); +// return taskClient; +// } +// }; +// +// return new TestableKinesisSupervisor( +// taskStorage, +// taskMaster, +// indexerMetadataStorageCoordinator, +// taskClientFactory, +// objectMapper, +// new KinesisSupervisorSpec( +// dataSchema, +// tuningConfig, +// kafkaSupervisorIOConfig, +// null, +// taskStorage, +// taskMaster, +// indexerMetadataStorageCoordinator, +// taskClientFactory, +// objectMapper, +// new NoopServiceEmitter(), +// new DruidMonitorSchedulerConfig() +// ) +// ); +// } +// +// private DataSchema getDataSchema(String dataSource) +// { +// List dimensions = new ArrayList<>(); +// dimensions.add(StringDimensionSchema.create("dim1")); +// dimensions.add(StringDimensionSchema.create("dim2")); +// +// return new DataSchema( +// dataSource, +// objectMapper.convertValue( +// new StringInputRowParser( +// new JSONParseSpec( +// new TimestampSpec("timestamp", "iso", null), +// new DimensionsSpec( +// dimensions, +// null, +// null +// ), +// new JSONPathSpec(true, ImmutableList.of()), +// ImmutableMap.of() +// ), +// Charsets.UTF_8.name() +// ), +// Map.class +// ), +// new AggregatorFactory[]{new CountAggregatorFactory("rows")}, +// new UniformGranularitySpec( +// Granularities.HOUR, +// Granularities.NONE, +// ImmutableList.of() +// ), +// objectMapper +// ); +// } +// +// private KinesisIndexTask createKafkaIndexTask( +// String id, +// String dataSource, +// String sequenceName, +// KinesisPartitions startPartitions, +// KinesisPartitions endPartitions, +// DateTime minimumMessageTime +// ) +// { +// return new KinesisIndexTask( +// id, +// null, +// getDataSchema(dataSource), +// tuningConfig, +// new KinesisIOConfig( +// sequenceName, +// startPartitions, +// endPartitions, +// ImmutableMap.of(), +// true, +// false, +// minimumMessageTime, +// false +// ), +// ImmutableMap.of(), +// null +// ); +// } +// +// private static class TestTaskRunnerWorkItem extends TaskRunnerWorkItem +// { +// +// private TaskLocation location; +// +// public TestTaskRunnerWorkItem(String taskId, ListenableFuture result, TaskLocation location) +// { +// super(taskId, result); +// this.location = location; +// } +// +// @Override +// public TaskLocation getLocation() +// { +// return location; +// } +// } +// +// private static class TestableKinesisSupervisor extends KinesisSupervisor +// { +// public TestableKinesisSupervisor( +// TaskStorage taskStorage, +// TaskMaster taskMaster, +// IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, +// KinesisIndexTaskClientFactory taskClientFactory, +// ObjectMapper mapper, +// KinesisSupervisorSpec spec +// ) +// { +// super(taskStorage, taskMaster, indexerMetadataStorageCoordinator, taskClientFactory, mapper, spec); +// } +// +// @Override +// protected String generateSequenceName(int groupId) +// { +// return String.format("sequenceName-%d", groupId); +// } +// } +//} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java new file mode 100644 index 000000000000..1169f7c5380c --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java @@ -0,0 +1,120 @@ +///* +// * Licensed to Metamarkets Group Inc. (Metamarkets) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. Metamarkets licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, +// * software distributed under the License is distributed on an +// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// * KIND, either express or implied. See the License for the +// * specific language governing permissions and limitations +// * under the License. +// */ +// +//package org.apache.druid.indexing.kinesis.supervisor; +// +//import com.fasterxml.jackson.databind.Module; +//import com.fasterxml.jackson.databind.ObjectMapper; +//import org.apache.druid.indexing.kinesis.KinesisIndexingServiceModule; +//import org.apache.druid.jackson.DefaultObjectMapper; +//import org.apache.druid.segment.IndexSpec; +//import org.apache.druid.segment.indexing.TuningConfig; +//import org.joda.time.Duration; +//import org.joda.time.Period; +//import org.junit.Assert; +//import org.junit.Test; +// +//import java.io.File; +// +//public class KinesisSupervisorTuningConfigTest +//{ +// private final ObjectMapper mapper; +// +// public KinesisSupervisorTuningConfigTest() +// { +// mapper = new DefaultObjectMapper(); +// mapper.registerModules((Iterable) new KinesisIndexingServiceModule().getJacksonModules()); +// } +// +// @Test +// public void testSerdeWithDefaults() throws Exception +// { +// String jsonStr = "{\"type\": \"kinesis\"}"; +// +// KinesisSupervisorTuningConfig config = (KinesisSupervisorTuningConfig) mapper.readValue( +// mapper.writeValueAsString( +// mapper.readValue( +// jsonStr, +// TuningConfig.class +// ) +// ), +// TuningConfig.class +// ); +// +// Assert.assertNotNull(config.getBasePersistDirectory()); +// Assert.assertEquals(75000, config.getMaxRowsInMemory()); +// Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment()); +// Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); +// Assert.assertEquals(0, config.getMaxPendingPersists()); +// Assert.assertEquals(new IndexSpec(), config.getIndexSpec()); +// Assert.assertEquals(true, config.getBuildV9Directly()); +// Assert.assertEquals(false, config.isReportParseExceptions()); +// Assert.assertEquals(0, config.getHandoffConditionTimeout()); +// Assert.assertNull(config.getWorkerThreads()); +// Assert.assertNull(config.getChatThreads()); +// Assert.assertEquals(8L, (long) config.getChatRetries()); +// Assert.assertEquals(Duration.standardSeconds(10), config.getHttpTimeout()); +// Assert.assertEquals(Duration.standardSeconds(80), config.getShutdownTimeout()); +// } +// +// @Test +// public void testSerdeWithNonDefaults() throws Exception +// { +// String jsonStr = "{\n" +// + " \"type\": \"kinesis\",\n" +// + " \"basePersistDirectory\": \"/tmp/xxx\",\n" +// + " \"maxRowsInMemory\": 100,\n" +// + " \"maxRowsPerSegment\": 100,\n" +// + " \"intermediatePersistPeriod\": \"PT1H\",\n" +// + " \"maxPendingPersists\": 100,\n" +// + " \"buildV9Directly\": false,\n" +// + " \"reportParseExceptions\": true,\n" +// + " \"handoffConditionTimeout\": 100,\n" +// + " \"workerThreads\": 12,\n" +// + " \"chatThreads\": 13,\n" +// + " \"chatRetries\": 14,\n" +// + " \"httpTimeout\": \"PT15S\",\n" +// + " \"shutdownTimeout\": \"PT95S\"\n" +// + "}"; +// +// KinesisSupervisorTuningConfig config = (KinesisSupervisorTuningConfig) mapper.readValue( +// mapper.writeValueAsString( +// mapper.readValue( +// jsonStr, +// TuningConfig.class +// ) +// ), +// TuningConfig.class +// ); +// +// Assert.assertEquals(new File("/tmp/xxx"), config.getBasePersistDirectory()); +// Assert.assertEquals(100, config.getMaxRowsInMemory()); +// Assert.assertEquals(100, config.getMaxRowsPerSegment()); +// Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); +// Assert.assertEquals(100, config.getMaxPendingPersists()); +// Assert.assertEquals(false, config.getBuildV9Directly()); +// Assert.assertEquals(true, config.isReportParseExceptions()); +// Assert.assertEquals(100, config.getHandoffConditionTimeout()); +// Assert.assertEquals(12, (int) config.getWorkerThreads()); +// Assert.assertEquals(13, (int) config.getChatThreads()); +// Assert.assertEquals(14L, (long) config.getChatRetries()); +// Assert.assertEquals(Duration.standardSeconds(15), config.getHttpTimeout()); +// Assert.assertEquals(Duration.standardSeconds(95), config.getShutdownTimeout()); +// } +//} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestBroker.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestBroker.java new file mode 100644 index 000000000000..6ad8f8c2226c --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestBroker.java @@ -0,0 +1,121 @@ +///* +// * Licensed to Metamarkets Group Inc. (Metamarkets) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. Metamarkets licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, +// * software distributed under the License is distributed on an +// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// * KIND, either express or implied. See the License for the +// * specific language governing permissions and limitations +// * under the License. +// */ +// +//package org.apache.druid.indexing.kinesis.test; +// +//import com.google.common.collect.ImmutableMap; +//import com.google.common.collect.Maps; +//import kafka.server.KafkaConfig; +//import kafka.server.KafkaServer; +//import org.apache.kafka.clients.consumer.KafkaConsumer; +//import org.apache.kafka.clients.producer.KafkaProducer; +//import org.apache.kafka.common.serialization.ByteArrayDeserializer; +//import org.apache.kafka.common.serialization.ByteArraySerializer; +//import org.apache.kafka.common.utils.SystemTime; +//import scala.Some; +//import scala.collection.immutable.List$; +// +//import java.io.Closeable; +//import java.io.File; +//import java.io.IOException; +//import java.util.Map; +//import java.util.Properties; +//import java.util.Random; +// +//public class TestBroker implements Closeable +//{ +// private final static Random RANDOM = new Random(); +// +// private final String zookeeperConnect; +// private final File directory; +// private final int id; +// private final Map brokerProps; +// +// private volatile KafkaServer server; +// +// public TestBroker(String zookeeperConnect, File directory, int id, Map brokerProps) +// { +// this.zookeeperConnect = zookeeperConnect; +// this.directory = directory; +// this.id = id; +// this.brokerProps = brokerProps == null ? ImmutableMap.of() : brokerProps; +// } +// +// public void start() +// { +// final Properties props = new Properties(); +// props.setProperty("zookeeper.connect", zookeeperConnect); +// props.setProperty("zookeeper.session.timeout.ms", "30000"); +// props.setProperty("zookeeper.connection.timeout.ms", "30000"); +// props.setProperty("log.dirs", directory.toString()); +// props.setProperty("broker.id", String.valueOf(id)); +// props.setProperty("port", String.valueOf(new Random().nextInt(9999) + 10000)); +// props.putAll(brokerProps); +// +// final KafkaConfig config = new KafkaConfig(props); +// +// server = new KafkaServer(config, SystemTime.SYSTEM, Some.apply(String.format("TestingBroker[%d]-", id)), List$.MODULE$.empty()); +// server.startup(); +// } +// +// public int getPort() +// { +// return server.socketServer().config().port(); +// } +// +// public KafkaProducer newProducer() +// { +// return new KafkaProducer(producerProperties()); +// } +// +// public KafkaConsumer newConsumer() +// { +// return new KafkaConsumer(consumerProperties()); +// } +// +// public Map producerProperties() +// { +// final Map props = Maps.newHashMap(); +// props.put("bootstrap.servers", String.format("localhost:%d", getPort())); +// props.put("key.serializer", ByteArraySerializer.class.getName()); +// props.put("value.serializer", ByteArraySerializer.class.getName()); +// props.put("acks", "all"); +// return props; +// } +// +// public Map consumerProperties() +// { +// final Map props = Maps.newHashMap(); +// props.put("bootstrap.servers", String.format("localhost:%d", getPort())); +// props.put("key.deserializer", ByteArrayDeserializer.class.getName()); +// props.put("value.deserializer", ByteArrayDeserializer.class.getName()); +// props.put("group.id", String.valueOf(RANDOM.nextInt())); +// props.put("auto.offset.reset", "earliest"); +// return props; +// } +// +// @Override +// public void close() throws IOException +// { +// if (server != null) { +// server.shutdown(); +// server.awaitShutdown(); +// } +// } +//} diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml new file mode 100644 index 000000000000..d5c2e3e99dd1 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -0,0 +1,172 @@ + + + 4.0.0 + + org.apache.druid.extensions + druid-kinesis-indexing-service + druid-kinesis-indexing-service + druid-kinesis-indexing-service + + pom + + + org.apache.druid + druid + 0.13.0-SNAPSHOT + ../../pom.xml + + + + + 1.6.3 + 1.10.61 + + + + imply-kinesis-lib + kinesis-indexing-service + + + + + org.apache.druid + druid-api + ${project.parent.version} + provided + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + provided + + + org.apache.druid + druid-server + ${project.parent.version} + provided + + + io.netty + netty + 3.10.4.Final + provided + + + + com.fasterxml.jackson.dataformat + jackson-dataformat-cbor + ${jackson.version} + + + + com.amazonaws + amazon-kinesis-client + ${kinesis.version} + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.dataformat + jackson-dataformat-cbor + + + commons-codec + commons-codec + + + com.amazonaws + aws-java-sdk-cloudwatch + + + com.amazonaws + aws-java-sdk-dynamodb + + + + + + com.amazonaws + aws-java-sdk-sts + ${aws.version} + + + + + junit + junit + 4.12 + test + + + org.apache.druid + druid-server + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + test-jar + test + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.1 + + -Xlint:unchecked + 1.8 + 1.8 + + + + org.apache.maven.plugins + maven-assembly-plugin + 2.5.5 + + + maven-release-plugin + 2.5.3 + + + org.apache.maven.scm + maven-scm-provider-gitexe + 1.9.5 + + + + + org.apache.maven.plugins + maven-jar-plugin + 3.0.2 + + + + true + true + + + + + + + + diff --git a/pom.xml b/pom.xml index a2f5ecf557f7..809fd010e1e6 100644 --- a/pom.xml +++ b/pom.xml @@ -133,6 +133,7 @@ extensions-core/kafka-eight extensions-core/kafka-extraction-namespace extensions-core/kafka-indexing-service + extensions-core/kinesis-indexing-service extensions-core/mysql-metadata-storage extensions-core/postgresql-metadata-storage extensions-core/protobuf-extensions From 4c806fe392ba7b84f064be2de8ae940fdd62e013 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 19 Sep 2018 13:06:25 -0700 Subject: [PATCH 09/87] merged some changes from kafka supervisor race condition --- .../supervisor/SeekableStreamSupervisor.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index b92e149088af..8b2b543db11d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -813,12 +813,9 @@ public Boolean apply(SeekableStreamIndexTask.Status status) private void verifyAndMergeCheckpoints(final Collection taskGroupsToVerify) { - final List> futures = new ArrayList<>(); + final List> futures = new ArrayList<>(); for (TaskGroup taskGroup : taskGroupsToVerify) { - futures.add(workerExec.submit(() -> { - verifyAndMergeCheckpoints(taskGroup); - return true; - })); + futures.add(workerExec.submit(() -> verifyAndMergeCheckpoints(taskGroup))); } try { Futures.allAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); @@ -859,7 +856,8 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) final String taskId = taskIds.get(i); if (checkpoints == null) { try { - futures.get(i).get(1, TimeUnit.NANOSECONDS); + // catch the exception in failed futures + futures.get(i).get(); } catch (Exception e) { log.error(e, "Problem while getting checkpoints for task [%s], killing the task", taskId); From 11c53507ab0051a704e4934b340a190c3ddee698 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 20 Sep 2018 16:16:16 -0700 Subject: [PATCH 10/87] integrated kinesis-indexing-service with seekablestream --- .../kafka/KafkaDataSourceMetadata.java | 98 +- .../druid/indexing/kafka/KafkaIndexTask.java | 16 +- .../indexing/kafka/KafkaIndexTaskClient.java | 8 - .../druid/indexing/kafka/KafkaPartitions.java | 50 +- .../indexing/kafka/KafkaRecordSupplier.java | 28 +- .../indexing/kafka/KafkaTuningConfig.java | 1 + .../kafka/supervisor/KafkaSupervisor.java | 43 +- .../KafkaSupervisorReportPayload.java | 2 +- .../indexing/kafka/KafkaIndexTaskTest.java | 4 +- .../imply-kinesis-lib/pom.xml | 19 + .../kinesis/KinesisRecordSupplier.java | 87 +- .../druid/indexing/kinesis/KinesisRegion.java | 12 +- .../ConstructibleAWSCredentialsConfig.java | 30 +- .../druid/indexing/kinesis/common/Record.java | 46 - .../kinesis/common/RecordSupplier.java | 28 - .../kinesis/common/StreamPartition.java | 63 - .../kinesis-indexing-service/pom.xml | 19 + .../kinesis/KinesisDataSourceMetadata.java | 128 +- .../indexing/kinesis/KinesisIOConfig.java | 113 +- .../indexing/kinesis/KinesisIndexTask.java | 167 +- .../kinesis/KinesisIndexTaskClient.java | 387 +--- .../KinesisIndexTaskClientFactory.java | 29 +- .../kinesis/KinesisIndexingServiceModule.java | 12 +- .../indexing/kinesis/KinesisPartitions.java | 75 +- .../indexing/kinesis/KinesisTuningConfig.java | 361 +-- .../kinesis/common/SequenceNumberPlus.java | 28 - .../kinesis/supervisor/KinesisSupervisor.java | 2022 ++--------------- .../supervisor/KinesisSupervisorIOConfig.java | 133 +- .../KinesisSupervisorReportPayload.java | 132 +- .../supervisor/KinesisSupervisorSpec.java | 202 +- .../KinesisSupervisorTuningConfig.java | 34 +- .../kinesis/supervisor/TaskReportData.java | 110 - .../kinesis-indexing-service/pom.xml | 19 + .../SeekableStreamDataSourceMetadata.java | 95 +- .../SeekableStreamIndexTask.java | 23 +- .../SeekableStreamIndexTaskClient.java | 2 - .../SeekableStreamPartitions.java | 36 +- .../SeekableStreamTuningConfig.java | 12 + .../seekablestream/common/Record.java | 1 + .../seekablestream/common/RecordSupplier.java | 2 - .../common/SequenceNumberPlus.java | 16 +- .../supervisor/SeekableStreamSupervisor.java | 663 +++--- .../supervisor/TaskReportData.java | 8 +- 43 files changed, 1344 insertions(+), 4020 deletions(-) delete mode 100644 extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/Record.java delete mode 100644 extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/RecordSupplier.java delete mode 100644 extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/StreamPartition.java delete mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/common/SequenceNumberPlus.java delete mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/TaskReportData.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java index dfec33b0d10c..3d98a6021e98 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java @@ -21,13 +21,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.Maps; -import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; -import org.apache.druid.java.util.common.IAE; import java.util.Map; -import java.util.Objects; public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata { @@ -46,97 +42,11 @@ public KafkaPartitions getKafkaPartitions() return (KafkaPartitions) super.getSeekableStreamPartitions(); } - @Override - public DataSourceMetadata plus(DataSourceMetadata other) - { - KafkaPartitions kafkaPartitions = getKafkaPartitions(); - - if (!(other instanceof KafkaDataSourceMetadata)) { - throw new IAE( - "Expected instance of %s, got %s", - KafkaDataSourceMetadata.class.getCanonicalName(), - other.getClass().getCanonicalName() - ); - } - - final KafkaDataSourceMetadata that = (KafkaDataSourceMetadata) other; - - if (that.getKafkaPartitions().getTopic().equals(kafkaPartitions.getTopic())) { - // Same topic, merge offsets. - final Map newMap = Maps.newHashMap(); - - for (Map.Entry entry : kafkaPartitions.getPartitionOffsetMap().entrySet()) { - newMap.put(entry.getKey(), entry.getValue()); - } - - for (Map.Entry entry : that.getKafkaPartitions().getPartitionOffsetMap().entrySet()) { - newMap.put(entry.getKey(), entry.getValue()); - } - - return new KafkaDataSourceMetadata(new KafkaPartitions(kafkaPartitions.getTopic(), newMap)); - } else { - // Different topic, prefer "other". - return other; - } - } - - @Override - public DataSourceMetadata minus(DataSourceMetadata other) - { - KafkaPartitions kafkaPartitions = getKafkaPartitions(); - - if (!(other instanceof KafkaDataSourceMetadata)) { - throw new IAE( - "Expected instance of %s, got %s", - KafkaDataSourceMetadata.class.getCanonicalName(), - other.getClass().getCanonicalName() - ); - } - - final KafkaDataSourceMetadata that = (KafkaDataSourceMetadata) other; - - if (that.getKafkaPartitions().getTopic().equals(kafkaPartitions.getTopic())) { - // Same topic, remove partitions present in "that" from "this" - final Map newMap = Maps.newHashMap(); - - for (Map.Entry entry : kafkaPartitions.getPartitionOffsetMap().entrySet()) { - if (!that.getKafkaPartitions().getPartitionOffsetMap().containsKey(entry.getKey())) { - newMap.put(entry.getKey(), entry.getValue()); - } - } - - return new KafkaDataSourceMetadata(new KafkaPartitions(kafkaPartitions.getTopic(), newMap)); - } else { - // Different topic, prefer "this". - return this; - } - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - KafkaDataSourceMetadata that = (KafkaDataSourceMetadata) o; - return Objects.equals(getKafkaPartitions(), that.getKafkaPartitions()); - } - - @Override - public int hashCode() - { - return Objects.hash(getKafkaPartitions()); - } - - @Override - public String toString() + protected SeekableStreamDataSourceMetadata createConcretDataSourceMetaData( + String streamId, Map newMap + ) { - return "KafkaDataSourceMetadata{" + - "kafkaPartitions=" + getKafkaPartitions() + - '}'; + return new KafkaDataSourceMetadata(new KafkaPartitions(streamId, newMap)); } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 9d0157f1fcac..7ee651aa7c86 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -48,14 +48,12 @@ import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; -import org.apache.druid.utils.CircularBuffer; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import java.util.Map; import java.util.Properties; -import java.util.Random; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -64,8 +62,6 @@ public class KafkaIndexTask extends SeekableStreamIndexTask { private static final EmittingLogger log = new EmittingLogger(KafkaIndexTask.class); - private static final String TYPE = "index_kafka"; //TODO: figure something out about TYPE - private static final Random RANDOM = new Random(); static final long POLL_TIMEOUT_MILLIS = TimeUnit.MILLISECONDS.toMillis(100); static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15; @@ -74,7 +70,6 @@ public class KafkaIndexTask extends SeekableStreamIndexTask // This value can be tuned in some tests private long pollRetryMs = 30000; - @JsonCreator public KafkaIndexTask( @JsonProperty("id") String id, @@ -97,16 +92,9 @@ public KafkaIndexTask( context, chatHandlerProvider, authorizerMapper, - rowIngestionMetersFactory + rowIngestionMetersFactory, + "index_kafka" ); - - final CircularBuffer savedParseExceptions; - if (tuningConfig.getMaxSavedParseExceptions() > 0) { - savedParseExceptions = new CircularBuffer<>(tuningConfig.getMaxSavedParseExceptions()); - } else { - savedParseExceptions = null; - } - if (context != null && context.get(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED) != null && ((boolean) context.get(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED))) { runner = new IncrementalPublishingKafkaIndexTaskRunner( diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java index 04d254dbdc4b..3733d190c1e5 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.common.TaskInfoProvider; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; import org.apache.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; @@ -61,11 +60,4 @@ protected JavaType constructMapType(Class mapType) return mapper.getTypeFactory().constructMapType(mapType, Integer.class, Long.class); } - @Override - public Class getTaskClassType() - { - return KafkaIndexTask.class; - } - - } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java index a69ff0a6509d..e92bc89973a7 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java @@ -21,16 +21,15 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; -import org.apache.druid.java.util.common.StringUtils; import java.util.Map; -import java.util.Objects; public class KafkaPartitions extends SeekableStreamPartitions { + public static final long NO_END_SEQUENCE_NUMBER = Long.MAX_VALUE; + @JsonCreator public KafkaPartitions( @JsonProperty("topic") final String topic, @@ -42,17 +41,12 @@ public KafkaPartitions( partitionOffsetMap ); - // Validate partitionOffsetMap - for (Map.Entry entry : partitionOffsetMap.entrySet()) { - Preconditions.checkArgument( - entry.getValue() >= 0, - StringUtils.format( - "partition[%d] offset[%d] invalid", - entry.getKey(), - entry.getValue() - ) - ); - } + } + + @Override + public Long getNoEndSequenceNumber() + { + return Long.MAX_VALUE; } @JsonProperty @@ -67,32 +61,4 @@ public Map getPartitionOffsetMap() return getPartitionSequenceMap(); } - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - KafkaPartitions that = (KafkaPartitions) o; - return Objects.equals(getTopic(), that.getTopic()) && - Objects.equals(getPartitionOffsetMap(), that.getPartitionOffsetMap()); - } - - @Override - public int hashCode() - { - return Objects.hash(getTopic(), getPartitionOffsetMap()); - } - - @Override - public String toString() - { - return "KafkaPartitions{" + - "topic='" + getTopic() + '\'' + - ", partitionOffsetMap=" + getPartitionOffsetMap() + - '}'; - } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 0b5e99593995..6f68284d43cf 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -1,5 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.indexing.kafka; +import com.google.common.collect.ImmutableSet; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.common.Record; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; @@ -116,10 +136,12 @@ public Long position(StreamPartition partition) public Set getPartitionIds(String streamName) { final Map> topics = consumer.listTopics(); - if (topics == null || !topics.containsKey(streamName)) { - throw new ISE("Could not retrieve partitions for topic [%s]", streamName); + if (!topics.containsKey(streamName)) { + throw new ISE("Topic [%s] is not found in KafkaConsumer's list of topics", streamName); } - return topics.get(streamName).stream().map(PartitionInfo::partition).collect(Collectors.toSet()); + return topics == null + ? ImmutableSet.of() + : topics.get(streamName).stream().map(PartitionInfo::partition).collect(Collectors.toSet()); } @Override diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java index 279d7a5ae84b..d35af2f1f8b6 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java @@ -66,6 +66,7 @@ public KafkaTuningConfig( reportParseExceptions, handoffConditionTimeout, resetOffsetAutomatically, + false, segmentWriteOutMediumFactory, intermediateHandoffPeriod, logParseExceptions, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 533e791798fc..dab3e02b4ad7 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -57,6 +57,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -81,6 +82,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor private static final long MINIMUM_GET_OFFSET_PERIOD_MILLIS = 5000; private static final long INITIAL_GET_OFFSET_DELAY_MILLIS = 15000; private static final long INITIAL_EMIT_LAG_METRIC_DELAY_MILLIS = 25000; + private static final Long NOT_SET = -1L; private final ServiceEmitter emitter; private final DruidMonitorSchedulerConfig monitorSchedulerConfig; @@ -107,9 +109,8 @@ public KafkaSupervisor( mapper, spec, rowIngestionMetersFactory, - -1L, - Long.MAX_VALUE, - true + NOT_SET, + false ); this.spec = spec; @@ -212,10 +213,10 @@ protected SeekableStreamSupervisorReportPayload createReportPaylo numPartitions, ioConfig.getReplicas(), ioConfig.getTaskDuration().getMillis() / 1000, - includeOffsets ? latestOffsetsFromStream : null, + includeOffsets ? latestSequenceFromStream : null, includeOffsets ? partitionLag : null, includeOffsets ? partitionLag.values().stream().mapToLong(x -> Math.max(x, 0)).sum() : null, - includeOffsets ? offsetsLastUpdated : null, + includeOffsets ? sequenceLastUpdated : null, spec.isSuspended() ); } @@ -248,7 +249,7 @@ protected SeekableStreamIOConfig createIoConfig( } @Override - protected List> createIndexTask( + protected List> createIndexTasks( int replicas, String baseSequenceName, ObjectMapper sortingMapper, @@ -301,29 +302,47 @@ protected Map getLagPerPartition(Map currentOffset .collect( Collectors.toMap( Map.Entry::getKey, - e -> latestOffsetsFromStream != null - && latestOffsetsFromStream.get(e.getKey()) != null + e -> latestSequenceFromStream != null + && latestSequenceFromStream.get(e.getKey()) != null && e.getValue() != null - ? latestOffsetsFromStream.get(e.getKey()) - e.getValue() + ? latestSequenceFromStream.get(e.getKey()) - e.getValue() : Integer.MIN_VALUE ) ); } + @Override + protected KafkaDataSourceMetadata createDataSourceMetaData( + String topic, Map map + ) + { + return new KafkaDataSourceMetadata(new KafkaPartitions(topic, map)); + } + + @Override + protected Map createNewTaskEndPartitions(Set startPartitions) + { + Map endPartitions = new HashMap<>(); + for (int partition : startPartitions) { + endPartitions.put(partition, KafkaPartitions.NO_END_SEQUENCE_NUMBER); + } + return endPartitions; + } + private Runnable emitLag() { return () -> { try { Map highestCurrentOffsets = getHighestCurrentOffsets(); - if (latestOffsetsFromStream == null) { + if (latestSequenceFromStream == null) { throw new ISE("Latest offsets from Kafka have not been fetched"); } - if (!latestOffsetsFromStream.keySet().equals(highestCurrentOffsets.keySet())) { + if (!latestSequenceFromStream.keySet().equals(highestCurrentOffsets.keySet())) { log.warn( "Lag metric: Kafka partitions %s do not match task partitions %s", - latestOffsetsFromStream.keySet(), + latestSequenceFromStream.keySet(), highestCurrentOffsets.keySet() ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java index fed5f5e021b0..1eee604c9809 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java @@ -94,7 +94,7 @@ public String toString() (getLatestOffsets() != null ? ", latestOffsets=" + getLatestOffsets() : "") + (getMinimumLag() != null ? ", minimumLag=" + getMinimumLag() : "") + (getAggregateLag() != null ? ", aggregateLag=" + getAggregateLag() : "") + - (getOffsetsLastUpdated() != null ? ", offsetsLastUpdated=" + getOffsetsLastUpdated() : "") + + (getOffsetsLastUpdated() != null ? ", sequenceLastUpdated=" + getOffsetsLastUpdated() : "") + ", suspended=" + getSuspended() + '}'; } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 117f7703891f..0ba4c33b0c8f 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -605,7 +605,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception ) ); final ListenableFuture future = runTask(task); - while (task.getRunner().getStatus() != KafkaIndexTask.Status.PAUSED) { + while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { Thread.sleep(10); } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); @@ -613,7 +613,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets)); task.getRunner().setEndOffsets(currentOffsets, false); - while (task.getRunner().getStatus() != KafkaIndexTask.Status.PAUSED) { + while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { Thread.sleep(10); } diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/pom.xml b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/pom.xml index f23c1ba65c56..39487c7f5ed0 100644 --- a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/pom.xml +++ b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/pom.xml @@ -1,4 +1,23 @@ + + 4.0.0 diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index f21ddb92200b..d04cec5911ec 100644 --- a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.indexing.kinesis; import com.amazonaws.ClientConfiguration; @@ -16,15 +35,16 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.common.aws.AWSCredentialsUtils; import org.apache.druid.indexing.kinesis.aws.ConstructibleAWSCredentialsConfig; -import org.apache.druid.indexing.kinesis.common.Record; -import org.apache.druid.indexing.kinesis.common.RecordSupplier; -import org.apache.druid.indexing.kinesis.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.common.Record; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -40,7 +60,7 @@ import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; -public class KinesisRecordSupplier implements RecordSupplier +public class KinesisRecordSupplier implements RecordSupplier { private static final EmittingLogger log = new EmittingLogger(KinesisRecordSupplier.class); private static final long PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS = 3000; @@ -48,7 +68,7 @@ public class KinesisRecordSupplier implements RecordSupplier private class PartitionResource { - private final StreamPartition streamPartition; + private final StreamPartition streamPartition; private final IKinesisProxy kinesisProxy; private final ScheduledExecutorService scheduledExec; private final Object startLock = new Object(); @@ -58,7 +78,7 @@ private class PartitionResource private volatile boolean stopRequested; public PartitionResource( - StreamPartition streamPartition, + StreamPartition streamPartition, IKinesisProxy kinesisProxy, ScheduledExecutorService scheduledExec ) @@ -116,7 +136,7 @@ private Runnable getRecordRunnable() log.info("shardIterator[%s] has been closed and has no more records", streamPartition.getPartitionId()); // add an end-of-shard marker so caller knows this shard is closed - Record endOfShardRecord = new Record( + Record endOfShardRecord = new Record( streamPartition.getStreamName(), streamPartition.getPartitionId(), Record.END_OF_SHARD_MARKER, null ); @@ -145,7 +165,7 @@ private Runnable getRecordRunnable() data = Collections.singletonList(toByteArray(kinesisRecord.getData())); } - final Record record = new Record( + final Record record = new Record<>( streamPartition.getStreamName(), streamPartition.getPartitionId(), kinesisRecord.getSequenceNumber(), @@ -214,8 +234,8 @@ private void rescheduleRunnable(long delayMillis) private final ScheduledExecutorService scheduledExec; private final Map kinesisProxies = new ConcurrentHashMap<>(); - private final Map partitionResources = new ConcurrentHashMap<>(); - private final BlockingQueue records; + private final Map, PartitionResource> partitionResources = new ConcurrentHashMap<>(); + private final BlockingQueue> records; private volatile boolean checkPartitionsStarted = false; private volatile boolean closed = false; @@ -279,7 +299,7 @@ public KinesisRecordSupplier( } @Override - public void assign(Set collection) + public void assign(Set> collection) { checkIfClosed(); @@ -290,9 +310,9 @@ public void assign(Set collection) ) ); - for (Iterator> i = partitionResources.entrySet() - .iterator(); i.hasNext(); ) { - Map.Entry entry = i.next(); + for (Iterator, PartitionResource>> i = partitionResources.entrySet() + .iterator(); i.hasNext(); ) { + Map.Entry, PartitionResource> entry = i.next(); if (!collection.contains(entry.getKey())) { i.remove(); entry.getValue().stop(); @@ -301,45 +321,52 @@ public void assign(Set collection) } @Override - public void seek(StreamPartition partition, String sequenceNumber) + public void seek(StreamPartition partition, String sequenceNumber) { checkIfClosed(); seekInternal(partition, sequenceNumber, ShardIteratorType.AT_SEQUENCE_NUMBER); } @Override - public void seekAfter(StreamPartition partition, String sequenceNumber) + public void seekAfter(StreamPartition partition, String sequenceNumber) { checkIfClosed(); seekInternal(partition, sequenceNumber, ShardIteratorType.AFTER_SEQUENCE_NUMBER); } @Override - public void seekToEarliest(StreamPartition partition) + public void seekToEarliest(Set> partitions) { checkIfClosed(); - seekInternal(partition, null, ShardIteratorType.TRIM_HORIZON); + partitions.forEach(partition -> seekInternal(partition, null, ShardIteratorType.TRIM_HORIZON)); + } @Override - public void seekToLatest(StreamPartition partition) + public void seekToLatest(Set> partitions) { checkIfClosed(); - seekInternal(partition, null, ShardIteratorType.LATEST); + partitions.forEach(partition -> seekInternal(partition, null, ShardIteratorType.LATEST)); + } + + @Override + public Collection> getAssignment() + { + return null; } @Override - public Record poll(long timeout) + public Record poll(long timeout) { checkIfClosed(); if (checkPartitionsStarted) { - partitionResources.values().stream().forEach(PartitionResource::start); + partitionResources.values().forEach(PartitionResource::start); checkPartitionsStarted = false; } try { while (true) { - Record record = records.poll(timeout, TimeUnit.MILLISECONDS); + Record record = records.poll(timeout, TimeUnit.MILLISECONDS); if (record == null || partitionResources.containsKey(record.getStreamPartition())) { return record; } else if (log.isTraceEnabled()) { @@ -359,19 +386,25 @@ public Record poll(long timeout) } @Override - public String getLatestSequenceNumber(StreamPartition partition) throws TimeoutException + public String getLatestSequenceNumber(StreamPartition partition) throws TimeoutException { checkIfClosed(); return getSequenceNumberInternal(partition, ShardIteratorType.LATEST); } @Override - public String getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException + public String getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException { checkIfClosed(); return getSequenceNumberInternal(partition, ShardIteratorType.TRIM_HORIZON); } + @Override + public String position(StreamPartition partition) + { + return null; + } + @Override public Set getPartitionIds(String streamName) { @@ -412,7 +445,7 @@ private IKinesisProxy getKinesisProxy(String streamName) return kinesisProxies.get(streamName); } - private void seekInternal(StreamPartition partition, String sequenceNumber, ShardIteratorType iteratorEnum) + private void seekInternal(StreamPartition partition, String sequenceNumber, ShardIteratorType iteratorEnum) { PartitionResource resource = partitionResources.get(partition); if (resource == null) { @@ -432,7 +465,7 @@ private void seekInternal(StreamPartition partition, String sequenceNumber, Shar checkPartitionsStarted = true; } - private String getSequenceNumberInternal(StreamPartition partition, ShardIteratorType iteratorEnum) + private String getSequenceNumberInternal(StreamPartition partition, ShardIteratorType iteratorEnum) throws TimeoutException { long timeoutMillis = System.currentTimeMillis() + fetchSequenceNumberTimeout; diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java index dec9bce2f006..cd1a3a70c180 100644 --- a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java +++ b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java index 7b1f1b370ee7..3a244abde962 100644 --- a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java +++ b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.indexing.kinesis.aws; import org.apache.druid.common.aws.AWSCredentialsConfig; @@ -6,9 +25,9 @@ public class ConstructibleAWSCredentialsConfig extends AWSCredentialsConfig { - final private String accessKey; - final private String secretKey; - final private String fileSessionCredentials; + private final String accessKey; + private final String secretKey; + private final String fileSessionCredentials; public ConstructibleAWSCredentialsConfig(String accessKey, String secretKey) { @@ -35,5 +54,8 @@ public PasswordProvider getSecretKey() } @Override - public String getFileSessionCredentials() { return fileSessionCredentials; } + public String getFileSessionCredentials() + { + return fileSessionCredentials; + } } diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/Record.java b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/Record.java deleted file mode 100644 index 729a97ec4546..000000000000 --- a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/Record.java +++ /dev/null @@ -1,46 +0,0 @@ -package org.apache.druid.indexing.kinesis.common; - -import java.util.List; - -public class Record -{ - public static final String END_OF_SHARD_MARKER = "EOS"; - - private final String streamName; - private final String partitionId; - private final String sequenceNumber; - private final List data; - - public Record(String streamName, String partitionId, String sequenceNumber, List data) - { - this.streamName = streamName; - this.partitionId = partitionId; - this.sequenceNumber = sequenceNumber; - this.data = data; - } - - public String getStreamName() - { - return streamName; - } - - public String getPartitionId() - { - return partitionId; - } - - public String getSequenceNumber() - { - return sequenceNumber; - } - - public List getData() - { - return data; - } - - public StreamPartition getStreamPartition() - { - return StreamPartition.of(streamName, partitionId); - } -} diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/RecordSupplier.java b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/RecordSupplier.java deleted file mode 100644 index de112dc26c0d..000000000000 --- a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/RecordSupplier.java +++ /dev/null @@ -1,28 +0,0 @@ -package org.apache.druid.indexing.kinesis.common; - -import java.io.Closeable; -import java.util.Set; -import java.util.concurrent.TimeoutException; - -public interface RecordSupplier extends Closeable -{ - void assign(Set partitions); - - void seek(StreamPartition partition, String sequenceNumber); - - void seekAfter(StreamPartition partition, String sequenceNumber); - - void seekToEarliest(StreamPartition partition); - - void seekToLatest(StreamPartition partition); - - Record poll(long timeout); - - String getLatestSequenceNumber(StreamPartition partition) throws TimeoutException; - - String getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException; - - Set getPartitionIds(String streamName); - - void close(); -} diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/StreamPartition.java b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/StreamPartition.java deleted file mode 100644 index cc40709697fa..000000000000 --- a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/common/StreamPartition.java +++ /dev/null @@ -1,63 +0,0 @@ -package org.apache.druid.indexing.kinesis.common; - -public class StreamPartition -{ - private final String streamName; - private final String partitionId; - - public StreamPartition(String streamName, String partitionId) - { - this.streamName = streamName; - this.partitionId = partitionId; - } - - public String getStreamName() - { - return streamName; - } - - public String getPartitionId() - { - return partitionId; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - StreamPartition that = (StreamPartition) o; - - if (streamName != null ? !streamName.equals(that.streamName) : that.streamName != null) { - return false; - } - return !(partitionId != null ? !partitionId.equals(that.partitionId) : that.partitionId != null); - } - - @Override - public int hashCode() - { - int result = streamName != null ? streamName.hashCode() : 0; - result = 31 * result + (partitionId != null ? partitionId.hashCode() : 0); - return result; - } - - @Override - public String toString() - { - return "StreamPartition{" + - "streamName='" + streamName + '\'' + - ", partitionId='" + partitionId + '\'' + - '}'; - } - - public static StreamPartition of(String streamName, String partitionId) - { - return new StreamPartition(streamName, partitionId); - } -} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml index b2e62f212212..eb2eb5af9f10 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml @@ -1,4 +1,23 @@ + + 4.0.0 diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java index 4a58f9d2bf0e..d5b884ac5b28 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ @@ -21,133 +21,29 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.Maps; -import org.apache.druid.indexing.overlord.DataSourceMetadata; -import org.apache.druid.java.util.common.IAE; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; import java.util.Map; -import java.util.Objects; -public class KinesisDataSourceMetadata implements DataSourceMetadata +public class KinesisDataSourceMetadata extends SeekableStreamDataSourceMetadata { - private final KinesisPartitions kinesisPartitions; - @JsonCreator public KinesisDataSourceMetadata( @JsonProperty("partitions") KinesisPartitions kinesisPartitions ) { - this.kinesisPartitions = kinesisPartitions; + super(kinesisPartitions); } @JsonProperty("partitions") public KinesisPartitions getKinesisPartitions() { - return kinesisPartitions; - } - - @Override - public boolean isValidStart() - { - return true; - } - - @Override - public boolean matches(DataSourceMetadata other) - { - if (getClass() != other.getClass()) { - return false; - } - - return plus(other).equals(other.plus(this)); - } - - @Override - public DataSourceMetadata plus(DataSourceMetadata other) - { - if (!(other instanceof KinesisDataSourceMetadata)) { - throw new IAE( - "Expected instance of %s, got %s", - KinesisDataSourceMetadata.class.getCanonicalName(), - other.getClass().getCanonicalName() - ); - } - - final KinesisDataSourceMetadata that = (KinesisDataSourceMetadata) other; - - if (that.getKinesisPartitions().getStream().equals(kinesisPartitions.getStream())) { - // Same topic, merge sequence numbers. - final Map newMap = Maps.newHashMap(); - - for (Map.Entry entry : kinesisPartitions.getPartitionSequenceNumberMap().entrySet()) { - newMap.put(entry.getKey(), entry.getValue()); - } - - for (Map.Entry entry : that.getKinesisPartitions().getPartitionSequenceNumberMap().entrySet()) { - newMap.put(entry.getKey(), entry.getValue()); - } - - return new KinesisDataSourceMetadata(new KinesisPartitions(kinesisPartitions.getStream(), newMap)); - } else { - // Different topic, prefer "other". - return other; - } - } - - @Override - public DataSourceMetadata minus(DataSourceMetadata other) - { - if (!(other instanceof KinesisDataSourceMetadata)) { - throw new IAE( - "Expected instance of %s, got %s", - KinesisDataSourceMetadata.class.getCanonicalName(), - other.getClass().getCanonicalName() - ); - } - - final KinesisDataSourceMetadata that = (KinesisDataSourceMetadata) other; - - if (that.getKinesisPartitions().getStream().equals(kinesisPartitions.getStream())) { - // Same stream, remove partitions present in "that" from "this" - final Map newMap = Maps.newHashMap(); - - for (Map.Entry entry : kinesisPartitions.getPartitionSequenceNumberMap().entrySet()) { - if (!that.getKinesisPartitions().getPartitionSequenceNumberMap().containsKey(entry.getKey())) { - newMap.put(entry.getKey(), entry.getValue()); - } - } - - return new KinesisDataSourceMetadata(new KinesisPartitions(kinesisPartitions.getStream(), newMap)); - } else { - // Different stream, prefer "this". - return this; - } - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - KinesisDataSourceMetadata that = (KinesisDataSourceMetadata) o; - return Objects.equals(kinesisPartitions, that.kinesisPartitions); - } - - @Override - public int hashCode() - { - return Objects.hash(kinesisPartitions); + return (KinesisPartitions) super.getSeekableStreamPartitions(); } @Override - public String toString() + protected KinesisDataSourceMetadata createConcretDataSourceMetaData(String streamName, Map newMap) { - return "KinesisDataSourceMetadata{" + - "kinesisPartitions=" + kinesisPartitions + - '}'; + return new KinesisDataSourceMetadata(new KinesisPartitions(streamName, newMap)); } } diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java index fc2d9a127d5f..af8895d17242 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ @@ -21,27 +21,19 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import org.apache.druid.segment.indexing.IOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; import org.joda.time.DateTime; import java.util.Set; -public class KinesisIOConfig implements IOConfig +public class KinesisIOConfig extends SeekableStreamIOConfig { - private static final boolean DEFAULT_USE_TRANSACTION = true; private static final boolean DEFAULT_PAUSE_AFTER_READ = true; private static final int DEFAULT_RECORDS_PER_FETCH = 4000; private static final int DEFAULT_FETCH_DELAY_MILLIS = 0; - private final String baseSequenceName; - private final KinesisPartitions startPartitions; - private final KinesisPartitions endPartitions; - private final boolean useTransaction; private final boolean pauseAfterRead; - private final Optional minimumMessageTime; - private final Optional maximumMessageTime; private final String endpoint; private final Integer recordsPerFetch; private final Integer fetchDelayMillis; @@ -72,13 +64,16 @@ public KinesisIOConfig( @JsonProperty("deaggregate") boolean deaggregate ) { - this.baseSequenceName = Preconditions.checkNotNull(baseSequenceName, "baseSequenceName"); - this.startPartitions = Preconditions.checkNotNull(startPartitions, "startPartitions"); - this.endPartitions = Preconditions.checkNotNull(endPartitions, "endPartitions"); - this.useTransaction = useTransaction != null ? useTransaction : DEFAULT_USE_TRANSACTION; + super( + null, + baseSequenceName, + startPartitions, + endPartitions, + useTransaction, + minimumMessageTime, + maximumMessageTime + ); this.pauseAfterRead = pauseAfterRead != null ? pauseAfterRead : DEFAULT_PAUSE_AFTER_READ; - this.minimumMessageTime = Optional.fromNullable(minimumMessageTime); - this.maximumMessageTime = Optional.fromNullable(maximumMessageTime); this.endpoint = Preconditions.checkNotNull(endpoint, "endpoint"); this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : DEFAULT_RECORDS_PER_FETCH; this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : DEFAULT_FETCH_DELAY_MILLIS; @@ -89,41 +84,6 @@ public KinesisIOConfig( this.awsExternalId = awsExternalId; this.deaggregate = deaggregate; - Preconditions.checkArgument( - startPartitions.getStream().equals(endPartitions.getStream()), - "start stream and end stream must match" - ); - - Preconditions.checkArgument( - startPartitions.getPartitionSequenceNumberMap() - .keySet() - .equals(endPartitions.getPartitionSequenceNumberMap().keySet()), - "start partition set and end partition set must match" - ); - } - - @JsonProperty - public String getBaseSequenceName() - { - return baseSequenceName; - } - - @JsonProperty - public KinesisPartitions getStartPartitions() - { - return startPartitions; - } - - @JsonProperty - public KinesisPartitions getEndPartitions() - { - return endPartitions; - } - - @JsonProperty - public boolean isUseTransaction() - { - return useTransaction; } @JsonProperty @@ -132,18 +92,6 @@ public boolean isPauseAfterRead() return pauseAfterRead; } - @JsonProperty - public Optional getMinimumMessageTime() - { - return minimumMessageTime; - } - - @JsonProperty - public Optional getMaximumMessageTime() - { - return maximumMessageTime; - } - @JsonProperty public String getEndpoint() { @@ -174,6 +122,7 @@ public String getAwsSecretAccessKey() return awsSecretAccessKey; } + @Override @JsonProperty public Set getExclusiveStartSequenceNumberPartitions() { @@ -198,17 +147,31 @@ public boolean isDeaggregate() return deaggregate; } + @Override + @JsonProperty + public KinesisPartitions getStartPartitions() + { + return (KinesisPartitions) super.getStartPartitions(); + } + + @Override + @JsonProperty + public KinesisPartitions getEndPartitions() + { + return (KinesisPartitions) super.getEndPartitions(); + } + @Override public String toString() { return "KinesisIOConfig{" + - "baseSequenceName='" + baseSequenceName + '\'' + - ", startPartitions=" + startPartitions + - ", endPartitions=" + endPartitions + - ", useTransaction=" + useTransaction + + "baseSequenceName='" + getBaseSequenceName() + '\'' + + ", startPartitions=" + getStartPartitions() + + ", endPartitions=" + getEndPartitions() + + ", useTransaction=" + isUseTransaction() + ", pauseAfterRead=" + pauseAfterRead + - ", minimumMessageTime=" + minimumMessageTime + - ", maximumMessageTime=" + maximumMessageTime + + ", minimumMessageTime=" + getMinimumMessageTime() + + ", maximumMessageTime=" + getMaximumMessageTime() + ", endpoint='" + endpoint + '\'' + ", recordsPerFetch=" + recordsPerFetch + ", fetchDelayMillis=" + fetchDelayMillis + diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index d0295290f3df..3eedf7dc6996 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ @@ -27,18 +27,14 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Joiner; -import com.google.common.base.Optional; -import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.google.common.primitives.Ints; import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.discovery.LookupNodeService; @@ -51,17 +47,15 @@ import org.apache.druid.indexing.common.actions.ResetDataSourceMetadataAction; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.TaskResource; -import org.apache.druid.indexing.common.task.Tasks; -import org.apache.druid.indexing.kinesis.common.Record; -import org.apache.druid.indexing.kinesis.common.RecordSupplier; -import org.apache.druid.indexing.kinesis.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.common.Record; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -84,7 +78,6 @@ import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; -import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; @@ -95,7 +88,6 @@ import org.apache.druid.server.security.ResourceAction; import org.apache.druid.server.security.ResourceType; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.utils.CircularBuffer; import org.joda.time.DateTime; import javax.servlet.http.HttpServletRequest; @@ -117,7 +109,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.RejectedExecutionException; @@ -128,37 +119,23 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; -public class KinesisIndexTask extends AbstractTask implements ChatHandler +public class KinesisIndexTask extends SeekableStreamIndexTask { public static final long PAUSE_FOREVER = -1L; - public enum Status - { - NOT_STARTED, - STARTING, - READING, - PAUSED, - PUBLISHING - } - private static final EmittingLogger log = new EmittingLogger(KinesisIndexTask.class); - private static final String TYPE = "index_kinesis"; - private static final Random RANDOM = new Random(); + private static final long POLL_TIMEOUT = 100; private static final long POLL_RETRY_MS = 30000; private static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15; private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; - private final DataSchema dataSchema; - private final InputRowParser parser; - private final KinesisTuningConfig tuningConfig; - private final KinesisIOConfig ioConfig; private final AuthorizerMapper authorizerMapper; - private final Optional chatHandlerProvider; private final Map endOffsets = new ConcurrentHashMap<>(); private final Map lastOffsets = new ConcurrentHashMap<>(); - + private final KinesisIOConfig ioConfig; + private final KinesisTuningConfig tuningConfig; private ObjectMapper mapper; private volatile Appenderator appenderator = null; @@ -205,7 +182,6 @@ public enum Status private final Object statusLock = new Object(); private final RowIngestionMeters rowIngestionMeters; - private CircularBuffer savedParseExceptions; private IngestionState ingestionState; private volatile boolean pauseRequested = false; @@ -225,70 +201,24 @@ public KinesisIndexTask( ) { super( - id == null ? makeTaskId(dataSchema.getDataSource(), RANDOM.nextInt()) : id, - String.format("%s_%s", TYPE, dataSchema.getDataSource()), + id, taskResource, - dataSchema.getDataSource(), - context + dataSchema, + tuningConfig, + ioConfig, + context, + chatHandlerProvider, + authorizerMapper, + rowIngestionMetersFactory, + "index_kinesis" ); - this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); - this.parser = Preconditions.checkNotNull((InputRowParser) dataSchema.getParser(), "parser"); - this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); - this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); - this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); this.authorizerMapper = authorizerMapper; this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters(); this.ingestionState = IngestionState.NOT_STARTED; - if (tuningConfig.getMaxSavedParseExceptions() > 0) { - savedParseExceptions = new CircularBuffer(tuningConfig.getMaxSavedParseExceptions()); - } - this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionSequenceNumberMap()); - } - - private static String makeTaskId(String dataSource, int randomBits) - { - final StringBuilder suffix = new StringBuilder(8); - for (int i = 0; i < Ints.BYTES * 2; ++i) { - suffix.append((char) ('a' + ((randomBits >>> (i * 4)) & 0x0F))); - } - return Joiner.on("_").join(TYPE, dataSource, suffix); - } - - @Override - public int getPriority() - { - return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY); - } - - @Override - public String getType() - { - return TYPE; - } - - @Override - public boolean isReady(TaskActionClient taskActionClient) throws Exception - { - return true; - } - - @JsonProperty - public DataSchema getDataSchema() - { - return dataSchema; - } - - @JsonProperty - public KinesisTuningConfig getTuningConfig() - { - return tuningConfig; - } - - @JsonProperty("ioConfig") - public KinesisIOConfig getIOConfig() - { - return ioConfig; + this.ioConfig = ioConfig; + this.tuningConfig = tuningConfig; + this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionSequenceMap()); } @Override @@ -334,41 +264,41 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception try ( final Appenderator appenderator0 = newAppenderator(fireDepartmentMetrics, toolbox); final StreamAppenderatorDriver driver = newDriver(appenderator0, toolbox, fireDepartmentMetrics); - final RecordSupplier recordSupplier = getRecordSupplier() + final RecordSupplier recordSupplier = getRecordSupplier() ) { toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); toolbox.getDataSegmentServerAnnouncer().announce(); appenderator = appenderator0; - final String topic = ioConfig.getStartPartitions().getStream(); + final String topic = ioConfig.getStartPartitions().getId(); // Start up, set up initial offsets. final Object restoredMetadata = driver.startJob(); if (restoredMetadata == null) { - lastOffsets.putAll(ioConfig.getStartPartitions().getPartitionSequenceNumberMap()); + lastOffsets.putAll(ioConfig.getStartPartitions().getPartitionSequenceMap()); } else { final Map restoredMetadataMap = (Map) restoredMetadata; final KinesisPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( restoredMetadataMap.get(METADATA_NEXT_PARTITIONS), KinesisPartitions.class ); - lastOffsets.putAll(restoredNextPartitions.getPartitionSequenceNumberMap()); + lastOffsets.putAll(restoredNextPartitions.getPartitionSequenceMap()); // Sanity checks. - if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) { + if (!restoredNextPartitions.getId().equals(ioConfig.getStartPartitions().getId())) { throw new ISE( "WTF?! Restored stream[%s] but expected stream[%s]", - restoredNextPartitions.getStream(), - ioConfig.getStartPartitions().getStream() + restoredNextPartitions.getId(), + ioConfig.getStartPartitions().getId() ); } - if (!lastOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet())) { + if (!lastOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceMap().keySet())) { throw new ISE( "WTF?! Restored partitions[%s] but expected partitions[%s]", lastOffsets.keySet(), - ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet() + ioConfig.getStartPartitions().getPartitionSequenceMap().keySet() ); } } @@ -404,7 +334,7 @@ public Object getMetadata() { return ImmutableMap.of( METADATA_NEXT_PARTITIONS, new KinesisPartitions( - ioConfig.getStartPartitions().getStream(), + ioConfig.getStartPartitions().getId(), snapshot ) ); @@ -445,7 +375,7 @@ public void run() break; } - Record record = recordSupplier.poll(POLL_TIMEOUT); + Record record = recordSupplier.poll(POLL_TIMEOUT); if (record == null) { continue; @@ -593,7 +523,7 @@ public void run() ); // Sanity check, we should only be publishing things that match our desired end state. - if (!endOffsets.equals(finalPartitions.getPartitionSequenceNumberMap())) { + if (!endOffsets.equals(finalPartitions.getPartitionSequenceMap())) { throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata); } @@ -717,7 +647,7 @@ private void handleParseException(ParseException pe, Record record) if (tuningConfig.isLogParseExceptions()) { log.error( pe, - "Encountered parse exception on row from partition[%d] sequenceNumber[%s]", + "Encountered parse exception on row from partition[%s] sequenceNumber[%s]", record.getPartitionId(), record.getSequenceNumber() ); @@ -1074,7 +1004,7 @@ private boolean isPaused() private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) { final int maxRowsInMemoryPerPartition = (tuningConfig.getMaxRowsInMemory() / - ioConfig.getStartPartitions().getPartitionSequenceNumberMap().size()); + ioConfig.getStartPartitions().getPartitionSequenceMap().size()); return Appenderators.createRealtime( dataSchema, tuningConfig.withBasePersistDirectory(new File(toolbox.getPersistDir(), "persist")) @@ -1123,11 +1053,11 @@ private StreamAppenderatorDriver newDriver( ); } - private RecordSupplier getRecordSupplier() + private RecordSupplier getRecordSupplier() { int fetchThreads = tuningConfig.getFetchThreads() != null ? tuningConfig.getFetchThreads() - : Math.max(1, ioConfig.getStartPartitions().getPartitionSequenceNumberMap().size()); + : Math.max(1, ioConfig.getStartPartitions().getPartitionSequenceMap().size()); return new KinesisRecordSupplier( ioConfig.getEndpoint(), @@ -1180,7 +1110,7 @@ private Set assignPartitions(RecordSupplier recordSupplier, String topic } private void seekToStartingRecords( - RecordSupplier recordSupplier, + RecordSupplier recordSupplier, String topic, Set assignment, TaskToolbox toolbox @@ -1189,7 +1119,7 @@ private void seekToStartingRecords( // Seek to starting offsets. for (final String partition : assignment) { final String offset = lastOffsets.get(partition); - final StreamPartition streamPartition = StreamPartition.of(topic, partition); + final StreamPartition streamPartition = StreamPartition.of(topic, partition); if (!tuningConfig.isSkipSequenceNumberAvailabilityCheck()) { try { @@ -1200,7 +1130,7 @@ private void seekToStartingRecords( try { sendResetRequestAndWait( assignment.stream() - .collect(Collectors.toMap((x) -> new StreamPartition(topic, x), lastOffsets::get)), + .collect(Collectors.toMap((x) -> new StreamPartition<>(topic, x), lastOffsets::get)), toolbox ); } @@ -1290,7 +1220,10 @@ private boolean possiblyPause(Set assignment) throws InterruptedExceptio return false; } - private void sendResetRequestAndWait(Map outOfRangePartitions, TaskToolbox taskToolbox) + private void sendResetRequestAndWait( + Map, String> outOfRangePartitions, + TaskToolbox taskToolbox + ) throws IOException { Map partitionOffsetMap = outOfRangePartitions @@ -1302,7 +1235,7 @@ private void sendResetRequestAndWait(Map outOfRangePart new ResetDataSourceMetadataAction( getDataSource(), new KinesisDataSourceMetadata( - new KinesisPartitions(ioConfig.getStartPartitions().getStream(), partitionOffsetMap) + new KinesisPartitions(ioConfig.getStartPartitions().getId(), partitionOffsetMap) ) ) ); diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java index a7de074de1a8..8b0e3b5bcd80 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java @@ -1,32 +1,30 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ package org.apache.druid.indexing.kinesis; -import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Charsets; import com.google.common.base.Optional; import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.indexer.TaskLocation; @@ -35,11 +33,10 @@ import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.Request; @@ -49,7 +46,6 @@ import org.jboss.netty.channel.ChannelException; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; @@ -57,11 +53,9 @@ import java.io.IOException; import java.net.Socket; import java.net.URI; -import java.util.Collections; import java.util.Map; -import java.util.concurrent.Callable; -public class KinesisIndexTaskClient +public class KinesisIndexTaskClient extends SeekableStreamIndexTaskClient { public static class NoTaskLocationException extends RuntimeException { @@ -79,8 +73,8 @@ public TaskNotRunnableException(String message) } } + private static ObjectMapper mapper = new ObjectMapper(); public static final int MAX_RETRY_WAIT_SECONDS = 10; - private static final int MIN_RETRY_WAIT_SECONDS = 2; private static final EmittingLogger log = new EmittingLogger(KinesisIndexTaskClient.class); private static final String BASE_PATH = "/druid/worker/v1/chat"; @@ -104,6 +98,15 @@ public KinesisIndexTaskClient( long numRetries ) { + super( + httpClient, + jsonMapper, + taskInfoProvider, + dataSource, + numThreads, + httpTimeout, + numRetries + ); this.httpClient = httpClient; this.jsonMapper = jsonMapper; this.taskInfoProvider = taskInfoProvider; @@ -122,318 +125,18 @@ public KinesisIndexTaskClient( ); } - public void close() - { - executorService.shutdownNow(); - } - - public boolean stop(final String id, final boolean publish) - { - log.debug("Stop task[%s] publish[%s]", id, publish); - - try { - final FullResponseHolder response = submitRequest( - id, HttpMethod.POST, "stop", publish ? "publish=true" : null, true - ); - return response.getStatus().getCode() / 100 == 2; - } - catch (NoTaskLocationException e) { - return false; - } - catch (TaskNotRunnableException e) { - log.info("Task [%s] couldn't be stopped because it is no longer running", id); - return true; - } - catch (Exception e) { - log.warn(e, "Exception while stopping task [%s]", id); - return false; - } - } - - public boolean resume(final String id) - { - log.debug("Resume task[%s]", id); - - try { - final FullResponseHolder response = submitRequest(id, HttpMethod.POST, "resume", null, true); - return response.getStatus().getCode() / 100 == 2; - } - catch (NoTaskLocationException e) { - return false; - } - } - - public Map pause(final String id) - { - return pause(id, 0); - } - - public Map pause(final String id, final long timeout) - { - log.debug("Pause task[%s] timeout[%d]", id, timeout); - - try { - final FullResponseHolder response = submitRequest( - id, - HttpMethod.POST, - "pause", - timeout > 0 ? String.format("timeout=%d", timeout) : null, - true - ); - - if (response.getStatus().equals(HttpResponseStatus.OK)) { - log.info("Task [%s] paused successfully", id); - return jsonMapper.readValue(response.getContent(), new TypeReference>() {}); - } - - final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy(); - while (true) { - if (getStatus(id) == KinesisIndexTask.Status.PAUSED) { - return getCurrentOffsets(id, true); - } - - final Duration delay = retryPolicy.getAndIncrementRetryDelay(); - if (delay == null) { - log.error("Task [%s] failed to pause, aborting", id); - throw new ISE("Task [%s] failed to pause, aborting", id); - } else { - final long sleepTime = delay.getMillis(); - log.info( - "Still waiting for task [%s] to pause; will try again in [%s]", - id, - new Duration(sleepTime).toString() - ); - Thread.sleep(sleepTime); - } - } - } - catch (NoTaskLocationException e) { - log.error("Exception [%s] while pausing Task [%s]", e.getMessage(), id); - return ImmutableMap.of(); - } - catch (IOException | InterruptedException e) { - log.error("Exception [%s] while pausing Task [%s]", e.getMessage(), id); - throw Throwables.propagate(e); - } - } - - public KinesisIndexTask.Status getStatus(final String id) - { - log.debug("GetStatus task[%s]", id); - - try { - final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "status", null, true); - return jsonMapper.readValue(response.getContent(), KinesisIndexTask.Status.class); - } - catch (NoTaskLocationException e) { - return KinesisIndexTask.Status.NOT_STARTED; - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - - public DateTime getStartTime(final String id) + @Override + protected JavaType constructMapType(Class mapType) { - log.debug("GetStartTime task[%s]", id); - - try { - final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "time/start", null, true); - return response.getContent() == null || response.getContent().isEmpty() - ? null - : jsonMapper.readValue(response.getContent(), DateTime.class); - } - catch (NoTaskLocationException e) { - return null; - } - catch (IOException e) { - throw Throwables.propagate(e); - } + return mapper.getTypeFactory().constructMapType(mapType, String.class, String.class); } - public Map getCurrentOffsets(final String id, final boolean retry) - { - log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry); - - try { - final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "offsets/current", null, retry); - return jsonMapper.readValue(response.getContent(), new TypeReference>() {}); - } - catch (NoTaskLocationException e) { - return ImmutableMap.of(); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - - public Map getEndOffsets(final String id) - { - log.debug("GetEndOffsets task[%s]", id); - - try { - final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "offsets/end", null, true); - return jsonMapper.readValue(response.getContent(), new TypeReference>() {}); - } - catch (NoTaskLocationException e) { - return ImmutableMap.of(); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - - public boolean setEndOffsets(final String id, final Map endOffsets) - { - return setEndOffsets(id, endOffsets, false); - } - - public boolean setEndOffsets(final String id, final Map endOffsets, final boolean resume) - { - log.debug("SetEndOffsets task[%s] endOffsets[%s] resume[%s]", id, endOffsets, resume); - - try { - final FullResponseHolder response = submitRequest( - id, - HttpMethod.POST, - "offsets/end", - resume ? "resume=true" : null, - jsonMapper.writeValueAsBytes(endOffsets), - true - ); - return response.getStatus().getCode() / 100 == 2; - } - catch (NoTaskLocationException e) { - return false; - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - - public ListenableFuture stopAsync(final String id, final boolean publish) - { - return executorService.submit( - new Callable() - { - @Override - public Boolean call() throws Exception - { - return stop(id, publish); - } - } - ); - } - - public ListenableFuture resumeAsync(final String id) - { - return executorService.submit( - new Callable() - { - @Override - public Boolean call() throws Exception - { - return resume(id); - } - } - ); - } - - public ListenableFuture> pauseAsync(final String id) - { - return pauseAsync(id, 0); - } - - public ListenableFuture> pauseAsync(final String id, final long timeout) - { - return executorService.submit( - new Callable>() - { - @Override - public Map call() throws Exception - { - return pause(id, timeout); - } - } - ); - } - - public ListenableFuture getStatusAsync(final String id) - { - return executorService.submit( - new Callable() - { - @Override - public KinesisIndexTask.Status call() throws Exception - { - return getStatus(id); - } - } - ); - } - - public ListenableFuture getStartTimeAsync(final String id) - { - return executorService.submit( - new Callable() - { - @Override - public DateTime call() throws Exception - { - return getStartTime(id); - } - } - ); - } - - public ListenableFuture> getCurrentOffsetsAsync(final String id, final boolean retry) - { - return executorService.submit( - new Callable>() - { - @Override - public Map call() throws Exception - { - return getCurrentOffsets(id, retry); - } - } - ); - } - - public ListenableFuture> getEndOffsetsAsync(final String id) - { - return executorService.submit( - new Callable>() - { - @Override - public Map call() throws Exception - { - return getEndOffsets(id); - } - } - ); - } - - public ListenableFuture setEndOffsetsAsync(final String id, final Map endOffsets) + @Override + public void close() { - return setEndOffsetsAsync(id, endOffsets, false); + executorService.shutdownNow(); } - public ListenableFuture setEndOffsetsAsync( - final String id, final Map endOffsets, final boolean resume - ) - { - return executorService.submit( - new Callable() - { - @Override - public Boolean call() throws Exception - { - return setEndOffsets(id, endOffsets, resume); - } - } - ); - } @VisibleForTesting RetryPolicyFactory createRetryPolicyFactory() @@ -449,49 +152,13 @@ RetryPolicyFactory createRetryPolicyFactory() ); } + @Override @VisibleForTesting - void checkConnection(String host, int port) throws IOException + protected void checkConnection(String host, int port) throws IOException { new Socket(host, port).close(); } - public Map getMovingAverages(final String id) - { - log.debug("GetMovingAverages task[%s]", id); - - try { - final FullResponseHolder response = submitRequest( - id, - HttpMethod.GET, - "rowStats", - null, - true - ); - return response.getContent() == null || response.getContent().isEmpty() - ? Collections.emptyMap() - : jsonMapper.readValue(response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT); - } - catch (NoTaskLocationException e) { - return Collections.emptyMap(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - public ListenableFuture> getMovingAveragesAsync(final String id) - { - return executorService.submit( - new Callable>() - { - @Override - public Map call() - { - return getMovingAverages(id); - } - } - ); - } private FullResponseHolder submitRequest(String id, HttpMethod method, String pathSuffix, String query, boolean retry) { diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java index b16f22161680..986cb5f0408a 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ @@ -24,21 +24,22 @@ import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; -public class KinesisIndexTaskClientFactory +public class KinesisIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory { - private HttpClient httpClient; - private ObjectMapper mapper; - @Inject public KinesisIndexTaskClientFactory(@EscalatedGlobal HttpClient httpClient, @Json ObjectMapper mapper) { - this.httpClient = httpClient; - this.mapper = mapper; + super( + httpClient, + mapper + ); } + @Override public KinesisIndexTaskClient build( TaskInfoProvider taskInfoProvider, String dataSource, @@ -48,8 +49,8 @@ public KinesisIndexTaskClient build( ) { return new KinesisIndexTaskClient( - httpClient, - mapper, + getHttpClient(), + getMapper(), taskInfoProvider, dataSource, numThreads, diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java index c96736af45b0..8abf7f3778ed 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java index 9586f42e82f3..9f91edb394ea 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ @@ -21,79 +21,42 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import java.util.Map; -import java.util.Objects; -public class KinesisPartitions +public class KinesisPartitions extends SeekableStreamPartitions { public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER"; - private final String stream; - private final Map partitionSequenceNumberMap; - @JsonCreator public KinesisPartitions( @JsonProperty("stream") final String stream, @JsonProperty("partitionSequenceNumberMap") final Map partitionSequenceNumberMap ) { - this.stream = stream; - this.partitionSequenceNumberMap = ImmutableMap.copyOf(partitionSequenceNumberMap); + super( + stream, + partitionSequenceNumberMap + ); + } - // Validate partitionSequenceNumberMap - for (Map.Entry entry : partitionSequenceNumberMap.entrySet()) { - Preconditions.checkArgument( - entry.getValue() != null, - String.format( - "partition[%s] sequenceNumber[%s] invalid", - entry.getKey(), - entry.getValue() - ) - ); - } + @Override + public String getNoEndSequenceNumber() + { + return NO_END_SEQUENCE_NUMBER; } @JsonProperty public String getStream() { - return stream; + return getId(); } @JsonProperty public Map getPartitionSequenceNumberMap() { - return partitionSequenceNumberMap; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - KinesisPartitions that = (KinesisPartitions) o; - return Objects.equals(stream, that.stream) && - Objects.equals(partitionSequenceNumberMap, that.partitionSequenceNumberMap); + return getPartitionSequenceMap(); } - @Override - public int hashCode() - { - return Objects.hash(stream, partitionSequenceNumberMap); - } - - @Override - public String toString() - { - return "KinesisPartitions{" + - "stream='" + stream + '\'' + - ", partitionSequenceNumberMap=" + partitionSequenceNumberMap + - '}'; - } } diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java index 48a185f7bf7f..255122bb62cf 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ @@ -22,10 +22,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.indexing.TuningConfig; -import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; @@ -33,39 +32,20 @@ import java.io.File; import java.util.Objects; -public class KinesisTuningConfig implements TuningConfig, AppenderatorConfig +public class KinesisTuningConfig extends SeekableStreamTuningConfig { - private static final int DEFAULT_MAX_ROWS_PER_SEGMENT = 5_000_000; - private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false; - private static final boolean DEFAULT_SKIP_SEQUENCE_NUMBER_AVAILABILITY_CHECK = false; + private static final int DEFAULT_RECORD_BUFFER_SIZE = 10000; private static final int DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT = 5000; private static final int DEFAULT_RECORD_BUFFER_FULL_WAIT = 5000; private static final int DEFAULT_FETCH_SEQUENCE_NUMBER_TIMEOUT = 60000; - private final int maxRowsInMemory; - private final long maxBytesInMemory; - private final int maxRowsPerSegment; - private final Period intermediatePersistPeriod; - private final File basePersistDirectory; - private final int maxPendingPersists; - private final IndexSpec indexSpec; - private final boolean buildV9Directly; - private final boolean reportParseExceptions; - private final long handoffConditionTimeout; - private final boolean resetOffsetAutomatically; - private final boolean skipSequenceNumberAvailabilityCheck; + private final int recordBufferSize; private final int recordBufferOfferTimeout; private final int recordBufferFullWait; private final int fetchSequenceNumberTimeout; private final Integer fetchThreads; - @Nullable - private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; - - private final boolean logParseExceptions; - private final int maxParseExceptions; - private final int maxSavedParseExceptions; @JsonCreator public KinesisTuningConfig( @@ -92,31 +72,30 @@ public KinesisTuningConfig( @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions ) { + super( + maxRowsInMemory, + maxBytesInMemory, + maxRowsPerSegment, + null, + intermediatePersistPeriod, + basePersistDirectory, + maxPendingPersists, + indexSpec, + true, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + skipSequenceNumberAvailabilityCheck, + segmentWriteOutMediumFactory, + null, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions + ); // Cannot be a static because default basePersistDirectory is unique per-instance final RealtimeTuningConfig defaults = RealtimeTuningConfig.makeDefaultTuningConfig(basePersistDirectory); - this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; - this.maxBytesInMemory = maxBytesInMemory == null ? defaults.getMaxBytesInMemory() : maxBytesInMemory; - this.maxRowsPerSegment = maxRowsPerSegment == null ? DEFAULT_MAX_ROWS_PER_SEGMENT : maxRowsPerSegment; - this.intermediatePersistPeriod = intermediatePersistPeriod == null - ? defaults.getIntermediatePersistPeriod() - : intermediatePersistPeriod; - this.basePersistDirectory = defaults.getBasePersistDirectory(); - this.maxPendingPersists = maxPendingPersists == null ? defaults.getMaxPendingPersists() : maxPendingPersists; - this.indexSpec = indexSpec == null ? defaults.getIndexSpec() : indexSpec; - this.buildV9Directly = buildV9Directly == null ? defaults.getBuildV9Directly() : buildV9Directly; - this.reportParseExceptions = reportParseExceptions == null - ? defaults.isReportParseExceptions() - : reportParseExceptions; - this.handoffConditionTimeout = handoffConditionTimeout == null - ? defaults.getHandoffConditionTimeout() - : handoffConditionTimeout; - this.resetOffsetAutomatically = resetOffsetAutomatically == null - ? DEFAULT_RESET_OFFSET_AUTOMATICALLY - : resetOffsetAutomatically; - this.skipSequenceNumberAvailabilityCheck = skipSequenceNumberAvailabilityCheck == null - ? DEFAULT_SKIP_SEQUENCE_NUMBER_AVAILABILITY_CHECK - : skipSequenceNumberAvailabilityCheck; + this.recordBufferSize = recordBufferSize == null ? DEFAULT_RECORD_BUFFER_SIZE : recordBufferSize; this.recordBufferOfferTimeout = recordBufferOfferTimeout == null ? DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT @@ -127,137 +106,39 @@ public KinesisTuningConfig( this.fetchThreads = fetchThreads; // we handle this being null later Preconditions.checkArgument( - !this.resetOffsetAutomatically || !this.skipSequenceNumberAvailabilityCheck, + !resetOffsetAutomatically || !skipSequenceNumberAvailabilityCheck, "resetOffsetAutomatically cannot be used if skipSequenceNumberAvailabilityCheck=true" ); - - this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; - - if (this.reportParseExceptions) { - this.maxParseExceptions = 0; - this.maxSavedParseExceptions = maxSavedParseExceptions == null ? 0 : Math.min(1, maxSavedParseExceptions); - } else { - this.maxParseExceptions = maxParseExceptions == null ? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS : maxParseExceptions; - this.maxSavedParseExceptions = maxSavedParseExceptions == null - ? TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS - : maxSavedParseExceptions; - } - this.logParseExceptions = logParseExceptions == null ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS : logParseExceptions; } - public static KinesisTuningConfig copyOf(KinesisTuningConfig config) + @Override + public KinesisTuningConfig copyOf() { return new KinesisTuningConfig( - config.maxRowsInMemory, - config.maxBytesInMemory, - config.maxRowsPerSegment, - config.intermediatePersistPeriod, - config.basePersistDirectory, - config.maxPendingPersists, - config.indexSpec, - config.buildV9Directly, - config.reportParseExceptions, - config.handoffConditionTimeout, - config.resetOffsetAutomatically, - config.skipSequenceNumberAvailabilityCheck, - config.recordBufferSize, - config.recordBufferOfferTimeout, - config.recordBufferFullWait, - config.fetchSequenceNumberTimeout, - config.fetchThreads, - config.segmentWriteOutMediumFactory, - config.logParseExceptions, - config.maxParseExceptions, - config.maxSavedParseExceptions + getMaxRowsInMemory(), + getMaxBytesInMemory(), + getMaxRowsPerSegment(), + getIntermediatePersistPeriod(), + getBasePersistDirectory(), + getMaxPendingPersists(), + getIndexSpec(), + true, + isReportParseExceptions(), + getHandoffConditionTimeout(), + isResetOffsetAutomatically(), + isSkipSequenceNumberAvailabilityCheck(), + getRecordBufferSize(), + getRecordBufferOfferTimeout(), + getRecordBufferFullWait(), + getFetchSequenceNumberTimeout(), + getFetchThreads(), + getSegmentWriteOutMediumFactory(), + isLogParseExceptions(), + getMaxParseExceptions(), + getMaxSavedParseExceptions() ); } - @Override - @JsonProperty - public int getMaxRowsInMemory() - { - return maxRowsInMemory; - } - - @Override - @JsonProperty - public long getMaxBytesInMemory() - { - return maxBytesInMemory; - } - - @JsonProperty - public int getMaxRowsPerSegment() - { - return maxRowsPerSegment; - } - - @Override - @JsonProperty - public Period getIntermediatePersistPeriod() - { - return intermediatePersistPeriod; - } - - @Override - @JsonProperty - public File getBasePersistDirectory() - { - return basePersistDirectory; - } - - @Nullable - @Override - public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() - { - return segmentWriteOutMediumFactory; - } - - @Override - @JsonProperty - public int getMaxPendingPersists() - { - return maxPendingPersists; - } - - @Override - @JsonProperty - public IndexSpec getIndexSpec() - { - return indexSpec; - } - - @JsonProperty - public boolean getBuildV9Directly() - { - return buildV9Directly; - } - - @Override - @JsonProperty - public boolean isReportParseExceptions() - { - return reportParseExceptions; - } - - @JsonProperty - public long getHandoffConditionTimeout() - { - return handoffConditionTimeout; - } - - @JsonProperty - public boolean isResetOffsetAutomatically() - { - return resetOffsetAutomatically; - } - - @JsonProperty - public boolean isSkipSequenceNumberAvailabilityCheck() - { - return skipSequenceNumberAvailabilityCheck; - } - @JsonProperty public int getRecordBufferSize() { @@ -288,48 +169,31 @@ public Integer getFetchThreads() return fetchThreads; } - @JsonProperty - public boolean isLogParseExceptions() - { - return logParseExceptions; - } - - @JsonProperty - public int getMaxParseExceptions() - { - return maxParseExceptions; - } - - @JsonProperty - public int getMaxSavedParseExceptions() - { - return maxSavedParseExceptions; - } - + @Override public KinesisTuningConfig withBasePersistDirectory(File dir) { return new KinesisTuningConfig( - maxRowsInMemory, - maxBytesInMemory, - maxRowsPerSegment, - intermediatePersistPeriod, - dir, - maxPendingPersists, - indexSpec, - buildV9Directly, - reportParseExceptions, - handoffConditionTimeout, - resetOffsetAutomatically, - skipSequenceNumberAvailabilityCheck, - recordBufferSize, - recordBufferOfferTimeout, - recordBufferFullWait, - fetchSequenceNumberTimeout, - fetchThreads, - segmentWriteOutMediumFactory, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions + getMaxRowsInMemory(), + getMaxBytesInMemory(), + getMaxRowsPerSegment(), + getIntermediatePersistPeriod(), + getBasePersistDirectory(), + getMaxPendingPersists(), + getIndexSpec(), + true, + isReportParseExceptions(), + getHandoffConditionTimeout(), + isResetOffsetAutomatically(), + isSkipSequenceNumberAvailabilityCheck(), + getRecordBufferSize(), + getRecordBufferOfferTimeout(), + getRecordBufferFullWait(), + getFetchSequenceNumberTimeout(), + getFetchThreads(), + getSegmentWriteOutMediumFactory(), + isLogParseExceptions(), + getMaxParseExceptions(), + getMaxSavedParseExceptions() ); } @@ -337,26 +201,26 @@ public KinesisTuningConfig withMaxRowsInMemory(int rows) { return new KinesisTuningConfig( rows, - maxBytesInMemory, - maxRowsPerSegment, - intermediatePersistPeriod, - basePersistDirectory, - maxPendingPersists, - indexSpec, - buildV9Directly, - reportParseExceptions, - handoffConditionTimeout, - resetOffsetAutomatically, - skipSequenceNumberAvailabilityCheck, - recordBufferSize, - recordBufferOfferTimeout, - recordBufferFullWait, - fetchSequenceNumberTimeout, - fetchThreads, - segmentWriteOutMediumFactory, - logParseExceptions, - maxParseExceptions, - maxSavedParseExceptions + getMaxBytesInMemory(), + getMaxRowsPerSegment(), + getIntermediatePersistPeriod(), + getBasePersistDirectory(), + getMaxPendingPersists(), + getIndexSpec(), + true, + isReportParseExceptions(), + getHandoffConditionTimeout(), + isResetOffsetAutomatically(), + isSkipSequenceNumberAvailabilityCheck(), + getRecordBufferSize(), + getRecordBufferOfferTimeout(), + getRecordBufferFullWait(), + getFetchSequenceNumberTimeout(), + getFetchThreads(), + getSegmentWriteOutMediumFactory(), + isLogParseExceptions(), + getMaxParseExceptions(), + getMaxSavedParseExceptions() ); } @@ -425,27 +289,26 @@ public int hashCode() public String toString() { return "KinesisTuningConfig{" + - "maxRowsInMemory=" + maxRowsInMemory + - ", maxBytesInMemory=" + maxBytesInMemory + - ", maxRowsPerSegment=" + maxRowsPerSegment + - ", intermediatePersistPeriod=" + intermediatePersistPeriod + - ", basePersistDirectory=" + basePersistDirectory + - ", maxPendingPersists=" + maxPendingPersists + - ", indexSpec=" + indexSpec + - ", buildV9Directly=" + buildV9Directly + - ", reportParseExceptions=" + reportParseExceptions + - ", handoffConditionTimeout=" + handoffConditionTimeout + - ", resetOffsetAutomatically=" + resetOffsetAutomatically + - ", skipSequenceNumberAvailabilityCheck=" + skipSequenceNumberAvailabilityCheck + + "maxRowsInMemory=" + getMaxRowsInMemory() + + ", maxBytesInMemory=" + getMaxBytesInMemory() + + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + + ", basePersistDirectory=" + getBasePersistDirectory() + + ", maxPendingPersists=" + getMaxPendingPersists() + + ", indexSpec=" + getIndexSpec() + + ", reportParseExceptions=" + isReportParseExceptions() + + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + + ", skipSequenceNumberAvailabilityCheck=" + isSkipSequenceNumberAvailabilityCheck() + ", recordBufferSize=" + recordBufferSize + ", recordBufferOfferTimeout=" + recordBufferOfferTimeout + ", recordBufferFullWait=" + recordBufferFullWait + ", fetchSequenceNumberTimeout=" + fetchSequenceNumberTimeout + ", fetchThreads=" + fetchThreads + - ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + - ", logParseExceptions=" + logParseExceptions + - ", maxParseExceptions=" + maxParseExceptions + - ", maxSavedParseExceptions=" + maxSavedParseExceptions + + ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() + + ", logParseExceptions=" + isLogParseExceptions() + + ", maxParseExceptions=" + getMaxParseExceptions() + + ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + '}'; } } diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/common/SequenceNumberPlus.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/common/SequenceNumberPlus.java deleted file mode 100644 index 3f173498ba7a..000000000000 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/common/SequenceNumberPlus.java +++ /dev/null @@ -1,28 +0,0 @@ -package org.apache.druid.indexing.kinesis.common; - -public class SequenceNumberPlus -{ - private final String sequenceNumber; - private final boolean exclusive; - - private SequenceNumberPlus(String sequenceNumber, boolean exclusive) - { - this.sequenceNumber = sequenceNumber; - this.exclusive = exclusive; - } - - public String get() - { - return sequenceNumber; - } - - public boolean isExclusive() - { - return exclusive; - } - - public static SequenceNumberPlus of(String sequenceNumber, boolean exclusive) - { - return new SequenceNumberPlus(sequenceNumber, exclusive); - } -} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 98b0d2432854..a5373e2d8ffd 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -1,105 +1,61 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ package org.apache.druid.indexing.kinesis.supervisor; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; import com.google.common.base.Joiner; -import com.google.common.base.Optional; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.primitives.Ints; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import org.apache.commons.codec.digest.DigestUtils; -import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata; import org.apache.druid.indexing.kinesis.KinesisIOConfig; import org.apache.druid.indexing.kinesis.KinesisIndexTask; -import org.apache.druid.indexing.kinesis.KinesisIndexTaskClient; import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; import org.apache.druid.indexing.kinesis.KinesisPartitions; import org.apache.druid.indexing.kinesis.KinesisRecordSupplier; import org.apache.druid.indexing.kinesis.KinesisTuningConfig; -import org.apache.druid.indexing.kinesis.common.Record; -import org.apache.druid.indexing.kinesis.common.RecordSupplier; -import org.apache.druid.indexing.kinesis.common.SequenceNumberPlus; -import org.apache.druid.indexing.kinesis.common.StreamPartition; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; -import org.apache.druid.indexing.overlord.TaskQueue; -import org.apache.druid.indexing.overlord.TaskRunner; -import org.apache.druid.indexing.overlord.TaskRunnerListener; -import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorage; -import org.apache.druid.indexing.overlord.supervisor.Supervisor; -import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.metadata.EntryExistsException; -import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; import org.joda.time.DateTime; import javax.annotation.Nullable; -import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingDeque; +import java.util.TreeMap; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; /** * Supervisor responsible for managing the KafkaIndexTasks for a single dataSource. At a high level, the class accepts a @@ -109,1427 +65,66 @@ * tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of * Kafka offsets. */ -public class KinesisSupervisor implements Supervisor +public class KinesisSupervisor extends SeekableStreamSupervisor { private static final EmittingLogger log = new EmittingLogger(KinesisSupervisor.class); - private static final Random RANDOM = new Random(); - private static final long MAX_RUN_FREQUENCY_MILLIS = 1000; // prevent us from running too often in response to events private static final String NOT_SET = ""; - private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120; - - // Internal data structures - // -------------------------------------------------------- - - /** - * A TaskGroup is the main data structure used by KinesisSupervisor to organize and monitor Kafka partitions and - * indexing tasks. All the tasks in a TaskGroup should always be doing the same thing (reading the same partitions and - * starting from the same offset) and if [replicas] is configured to be 1, a TaskGroup will contain a single task (the - * exception being if the supervisor started up and discovered and adopted some already running tasks). At any given - * 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]). - */ - private static class TaskGroup - { - // 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 - // this task group has completed successfully, at which point this will be destroyed and a new task group will be - // created with new starting offsets. This allows us to create replacement tasks for failed tasks that process the - // same offsets, even if the values in [partitionGroups] has been changed. - final Map partitionOffsets; - - final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); - final Optional minimumMessageTime; - final Optional maximumMessageTime; - final Set exclusiveStartSequenceNumberPartitions; - - DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action - - public TaskGroup( - Map partitionOffsets, - Optional minimumMessageTime, - Optional maximumMessageTime, - Set exclusiveStartSequenceNumberPartitions - ) - { - this.partitionOffsets = partitionOffsets; - this.minimumMessageTime = minimumMessageTime; - this.maximumMessageTime = maximumMessageTime; - this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions != null - ? exclusiveStartSequenceNumberPartitions - : new HashSet<>(); - } - - Set taskIds() - { - return tasks.keySet(); - } - } - - private static class TaskData - { - TaskStatus status; - DateTime startTime; - } - - // Map<{group ID}, {actively reading task group}>; see documentation for TaskGroup class - private final ConcurrentHashMap taskGroups = new ConcurrentHashMap<>(); - - // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [taskGroups] to here so - // we can monitor its status while we queue new tasks to read the next range of offsets. This is a list since we could - // have multiple sets of tasks publishing at once if time-to-publish > taskDuration. - // Map<{group ID}, List<{pending completion task groups}>> - private final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); - - // 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 - // offset value from the metadata store, and if it can't find it there, from Kafka. Once a task begins - // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet- - // completed task, which will cause the next set of tasks to begin reading from where the previous task left - // off. If that previous task now fails, we will set the offset in [partitionGroups] back to NOT_SET which will - // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to - // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task - // failures during publishing. - // Map<{group ID}, Map<{partition ID}, {startingOffset}>> - private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); - // -------------------------------------------------------- - - private final TaskStorage taskStorage; - private final TaskMaster taskMaster; - private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; - private final KinesisIndexTaskClient taskClient; - private final ObjectMapper sortingMapper; - private final KinesisSupervisorSpec spec; - private final ServiceEmitter emitter; - private final DruidMonitorSchedulerConfig monitorSchedulerConfig; - private final String dataSource; - private final KinesisSupervisorIOConfig ioConfig; - private final KinesisSupervisorTuningConfig tuningConfig; - private final KinesisTuningConfig taskTuningConfig; - private final String supervisorId; - private final TaskInfoProvider taskInfoProvider; - private final long futureTimeoutInSeconds; // how long to wait for async operations to complete - private final RowIngestionMetersFactory rowIngestionMetersFactory; - - private final ExecutorService exec; - private final ScheduledExecutorService scheduledExec; - private final ListeningExecutorService workerExec; - private final BlockingQueue notices = new LinkedBlockingDeque<>(); - private final Object stopLock = new Object(); - private final Object stateChangeLock = new Object(); - - private boolean listenerRegistered = false; - private long lastRunTime; - - private volatile DateTime firstRunTime; - private volatile DateTime earlyPublishTime = null; - private volatile RecordSupplier recordSupplier; - - private volatile boolean started = false; - private volatile boolean stopped = false; - - private final ScheduledExecutorService metricEmittingExec; - // used while reporting lag - private final Map lastCurrentOffsets = new HashMap<>(); - - private final List partitionIds = new CopyOnWriteArrayList<>(); - private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); - - public KinesisSupervisor( - final TaskStorage taskStorage, - final TaskMaster taskMaster, - final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, - final KinesisIndexTaskClientFactory taskClientFactory, - final ObjectMapper mapper, - final KinesisSupervisorSpec spec, - final RowIngestionMetersFactory rowIngestionMetersFactory - ) - { - this.taskStorage = taskStorage; - this.taskMaster = taskMaster; - this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; - this.sortingMapper = mapper.copy().configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); - this.spec = spec; - this.emitter = spec.getEmitter(); - this.monitorSchedulerConfig = spec.getMonitorSchedulerConfig(); - this.rowIngestionMetersFactory = rowIngestionMetersFactory; - - this.dataSource = spec.getDataSchema().getDataSource(); - this.ioConfig = spec.getIoConfig(); - this.tuningConfig = spec.getTuningConfig(); - this.taskTuningConfig = KinesisTuningConfig.copyOf(this.tuningConfig); - this.supervisorId = String.format("KinesisSupervisor-%s", dataSource); - this.exec = Execs.singleThreaded(supervisorId); - this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d"); - this.metricEmittingExec = Execs.scheduledSingleThreaded(supervisorId + "-Emitter-%d"); - - int workerThreads = (this.tuningConfig.getWorkerThreads() != null - ? this.tuningConfig.getWorkerThreads() - : Math.min(10, this.ioConfig.getTaskCount())); - this.workerExec = MoreExecutors.listeningDecorator(Execs.multiThreaded(workerThreads, supervisorId + "-Worker-%d")); - log.info("Created worker pool with [%d] threads for dataSource [%s]", workerThreads, this.dataSource); - - this.taskInfoProvider = new TaskInfoProvider() - { - @Override - public TaskLocation getTaskLocation(final String id) - { - Preconditions.checkNotNull(id, "id"); - Optional taskRunner = taskMaster.getTaskRunner(); - if (taskRunner.isPresent()) { - Optional item = Iterables.tryFind( - taskRunner.get().getRunningTasks(), new Predicate() - { - @Override - public boolean apply(TaskRunnerWorkItem taskRunnerWorkItem) - { - return id.equals(taskRunnerWorkItem.getTaskId()); - } - } - ); - - if (item.isPresent()) { - return item.get().getLocation(); - } - } else { - log.error("Failed to get task runner because I'm not the leader!"); - } - - return TaskLocation.unknown(); - } - - @Override - public Optional getTaskStatus(String id) - { - return taskStorage.getStatus(id); - } - }; - - this.futureTimeoutInSeconds = Math.max( - MINIMUM_FUTURE_TIMEOUT_IN_SECONDS, - tuningConfig.getChatRetries() * (tuningConfig.getHttpTimeout().getStandardSeconds() - + KinesisIndexTaskClient.MAX_RETRY_WAIT_SECONDS) - ); - - int chatThreads = (this.tuningConfig.getChatThreads() != null - ? this.tuningConfig.getChatThreads() - : Math.min(10, this.ioConfig.getTaskCount() * this.ioConfig.getReplicas())); - this.taskClient = taskClientFactory.build( - taskInfoProvider, - dataSource, - chatThreads, - this.tuningConfig.getHttpTimeout(), - this.tuningConfig.getChatRetries() - ); - log.info( - "Created taskClient with dataSource[%s] chatThreads[%d] httpTimeout[%s] chatRetries[%d]", - dataSource, - chatThreads, - this.tuningConfig.getHttpTimeout(), - this.tuningConfig.getChatRetries() - ); - } - - @Override - public void start() - { - synchronized (stateChangeLock) { - Preconditions.checkState(!started, "already started"); - Preconditions.checkState(!exec.isShutdown(), "already stopped"); - - try { - setupRecordSupplier(); - - exec.submit( - new Runnable() - { - @Override - public void run() - { - try { - while (!Thread.currentThread().isInterrupted()) { - final Notice notice = notices.take(); - - try { - notice.handle(); - } - catch (Throwable e) { - log.makeAlert(e, "KinesisSupervisor[%s] failed to handle notice", dataSource) - .addData("noticeClass", notice.getClass().getSimpleName()) - .emit(); - } - } - } - catch (InterruptedException e) { - log.info("KinesisSupervisor[%s] interrupted, exiting", dataSource); - } - } - } - ); - firstRunTime = DateTime.now().plus(ioConfig.getStartDelay()); - scheduledExec.scheduleAtFixedRate( - buildRunTask(), - ioConfig.getStartDelay().getMillis(), - Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS), - TimeUnit.MILLISECONDS - ); - -// TODO: Implement this for Kinesis which uses approximate time from latest instead of offset lag -/* - metricEmittingExec.scheduleAtFixedRate( - computeAndEmitLag(taskClient), - ioConfig.getStartDelay().getMillis() + 10000, // wait for tasks to start up - Math.max(monitorSchedulerConfig.getEmitterPeriod().getMillis(), 60 * 1000), - TimeUnit.MILLISECONDS - ); -*/ - started = true; - log.info( - "Started KinesisSupervisor[%s], first run in [%s], with spec: [%s]", - dataSource, - ioConfig.getStartDelay(), - spec.toString() - ); - } - catch (Exception e) { - if (recordSupplier != null) { - recordSupplier.close(); - } - log.makeAlert(e, "Exception starting KinesisSupervisor[%s]", dataSource) - .emit(); - throw Throwables.propagate(e); - } - } - } - - @Override - public void stop(boolean stopGracefully) - { - synchronized (stateChangeLock) { - Preconditions.checkState(started, "not started"); - - log.info("Beginning shutdown of KinesisSupervisor[%s]", dataSource); - - try { - scheduledExec.shutdownNow(); // stop recurring executions - metricEmittingExec.shutdownNow(); - recordSupplier.close(); // aborts any in-flight sequenceNumber fetches - - Optional taskRunner = taskMaster.getTaskRunner(); - if (taskRunner.isPresent()) { - taskRunner.get().unregisterListener(supervisorId); - } - - // Stopping gracefully will synchronize the end offsets of the tasks and signal them to publish, and will block - // until the tasks have acknowledged or timed out. We want this behavior when we're explicitly shut down through - // the API, but if we shut down for other reasons (e.g. we lose leadership) we want to just stop and leave the - // tasks as they are. - synchronized (stopLock) { - if (stopGracefully) { - log.info("Posting GracefulShutdownNotice, signalling managed tasks to complete and publish"); - notices.add(new GracefulShutdownNotice()); - } else { - log.info("Posting ShutdownNotice"); - notices.add(new ShutdownNotice()); - } - - long shutdownTimeoutMillis = tuningConfig.getShutdownTimeout().getMillis(); - long endTime = System.currentTimeMillis() + shutdownTimeoutMillis; - while (!stopped) { - long sleepTime = endTime - System.currentTimeMillis(); - if (sleepTime <= 0) { - log.info("Timed out while waiting for shutdown (timeout [%,dms])", shutdownTimeoutMillis); - stopped = true; - break; - } - stopLock.wait(sleepTime); - } - } - log.info("Shutdown notice handled"); - - taskClient.close(); - workerExec.shutdownNow(); - exec.shutdownNow(); - started = false; - - log.info("KinesisSupervisor[%s] has stopped", dataSource); - } - catch (Exception e) { - log.makeAlert(e, "Exception stopping KinesisSupervisor[%s]", dataSource) - .emit(); - } - } - } - - @Override - public SupervisorReport getStatus() - { - return generateReport(true); - } - - @Override - public Map> getStats() - { - try { - return getCurrentTotalStats(); - } - catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - log.error(ie, "getStats() interrupted."); - throw new RuntimeException(ie); - } - catch (ExecutionException | TimeoutException eete) { - throw new RuntimeException(eete); - } - } - - @Override - public void reset(DataSourceMetadata dataSourceMetadata) - { - log.info("Posting ResetNotice"); - notices.add(new ResetNotice(dataSourceMetadata)); - } - - @Override - public void checkpoint( - @Nullable Integer taskGroupId, - @Deprecated String baseSequenceName, - DataSourceMetadata previousCheckPoint, - DataSourceMetadata currentCheckPoint - ) - { - // not supported right now - } - - public void possiblyRegisterListener() - { - // getTaskRunner() sometimes fails if the task queue is still being initialized so retry later until we succeed - - if (listenerRegistered) { - return; - } - - Optional taskRunner = taskMaster.getTaskRunner(); - if (taskRunner.isPresent()) { - taskRunner.get().registerListener( - new TaskRunnerListener() - { - @Override - public String getListenerId() - { - return supervisorId; - } - - @Override - public void locationChanged(final String taskId, final TaskLocation newLocation) - { - // do nothing - } - - @Override - public void statusChanged(String taskId, TaskStatus status) - { - notices.add(new RunNotice()); - } - }, MoreExecutors.sameThreadExecutor() - ); - - listenerRegistered = true; - } - } - - private interface Notice - { - void handle() throws ExecutionException, InterruptedException, TimeoutException; - } - - private class RunNotice implements Notice - { - @Override - public void handle() throws ExecutionException, InterruptedException, TimeoutException - { - long nowTime = System.currentTimeMillis(); - if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { - return; - } - lastRunTime = nowTime; - - runInternal(); - } - } - - private class GracefulShutdownNotice extends ShutdownNotice - { - @Override - public void handle() throws InterruptedException, ExecutionException, TimeoutException - { - gracefulShutdownInternal(); - super.handle(); - } - } - - private class ShutdownNotice implements Notice - { - @Override - public void handle() throws InterruptedException, ExecutionException, TimeoutException - { - recordSupplier.close(); - - synchronized (stopLock) { - stopped = true; - stopLock.notifyAll(); - } - } - } - - private class ResetNotice implements Notice - { - final DataSourceMetadata dataSourceMetadata; - - ResetNotice(DataSourceMetadata dataSourceMetadata) - { - this.dataSourceMetadata = dataSourceMetadata; - } - - @Override - public void handle() - { - log.makeAlert("Resetting dataSource [%s]", dataSource).emit(); - resetInternal(dataSourceMetadata); - } - } - - @VisibleForTesting - void resetInternal(DataSourceMetadata dataSourceMetadata) - { - if (dataSourceMetadata == null) { - // Reset everything - boolean result = indexerMetadataStorageCoordinator.deleteDataSourceMetadata(dataSource); - log.info("Reset dataSource[%s] - dataSource metadata entry deleted? [%s]", dataSource, result); - killTaskGroupForPartitions(partitionIds); - } else if (!(dataSourceMetadata instanceof KinesisDataSourceMetadata)) { - throw new IAE("Expected KinesisDataSourceMetadata but found instance of [%s]", dataSourceMetadata.getClass()); - } else { - // Reset only the partitions in dataSourceMetadata if it has not been reset yet - final KinesisDataSourceMetadata resetKafkaMetadata = (KinesisDataSourceMetadata) dataSourceMetadata; - - if (resetKafkaMetadata.getKinesisPartitions().getStream().equals(ioConfig.getStream())) { - // metadata can be null - final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); - if (metadata != null && !(metadata instanceof KinesisDataSourceMetadata)) { - throw new IAE( - "Expected KinesisDataSourceMetadata from metadata store but found instance of [%s]", - metadata.getClass() - ); - } - final KinesisDataSourceMetadata currentMetadata = (KinesisDataSourceMetadata) metadata; - - // defend against consecutive reset requests from replicas - // as well as the case where the metadata store do not have an entry for the reset partitions - boolean doReset = false; - for (Map.Entry resetPartitionOffset : resetKafkaMetadata.getKinesisPartitions() - .getPartitionSequenceNumberMap() - .entrySet()) { - final String partitionOffsetInMetadataStore = currentMetadata == null - ? null - : currentMetadata.getKinesisPartitions() - .getPartitionSequenceNumberMap() - .get(resetPartitionOffset.getKey()); - final TaskGroup partitionTaskGroup = taskGroups.get(getTaskGroupIdForPartition(resetPartitionOffset.getKey())); - if (partitionOffsetInMetadataStore != null || - (partitionTaskGroup != null && partitionTaskGroup.partitionOffsets.get(resetPartitionOffset.getKey()) - .equals(resetPartitionOffset.getValue()))) { - doReset = true; - break; - } - } - - if (!doReset) { - return; - } - - boolean metadataUpdateSuccess = false; - if (currentMetadata == null) { - metadataUpdateSuccess = true; - } else { - final DataSourceMetadata newMetadata = currentMetadata.minus(resetKafkaMetadata); - try { - metadataUpdateSuccess = indexerMetadataStorageCoordinator.resetDataSourceMetadata(dataSource, newMetadata); - } - catch (IOException e) { - log.error("Resetting DataSourceMetadata failed [%s]", e.getMessage()); - Throwables.propagate(e); - } - } - if (metadataUpdateSuccess) { - killTaskGroupForPartitions( - resetKafkaMetadata.getKinesisPartitions() - .getPartitionSequenceNumberMap() - .keySet() - ); - } else { - throw new ISE("Unable to reset metadata"); - } - } else { - log.warn( - "Reset metadata topic [%s] and supervisor's topic [%s] do not match", - resetKafkaMetadata.getKinesisPartitions().getStream(), - ioConfig.getStream() - ); - } - } - } - - private void killTaskGroupForPartitions(Collection partitions) - { - for (String partition : partitions) { - TaskGroup taskGroup = taskGroups.get(getTaskGroupIdForPartition(partition)); - if (taskGroup != null) { - // kill all tasks in this task group - for (String taskId : taskGroup.tasks.keySet()) { - log.info("Reset dataSource[%s] - killing task [%s]", dataSource, taskId); - killTask(taskId); - } - } - partitionGroups.remove(getTaskGroupIdForPartition(partition)); - taskGroups.remove(getTaskGroupIdForPartition(partition)); - } - } - - @VisibleForTesting - void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException - { - // Prepare for shutdown by 1) killing all tasks that haven't been assigned to a worker yet, and 2) causing all - // running tasks to begin publishing by setting their startTime to a very long time ago so that the logic in - // checkTaskDuration() will be triggered. This is better than just telling these tasks to publish whatever they - // have, as replicas that are supposed to publish the same segment may not have read the same set of offsets. - for (TaskGroup taskGroup : taskGroups.values()) { - for (Map.Entry entry : taskGroup.tasks.entrySet()) { - if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) { - killTask(entry.getKey()); - } else { - entry.getValue().startTime = new DateTime(0); - } - } - } - - checkTaskDuration(); - } - - @VisibleForTesting - void runInternal() throws ExecutionException, InterruptedException, TimeoutException - { - possiblyRegisterListener(); - updatePartitionDataFromKinesis(); - discoverTasks(); - updateTaskStatus(); - checkTaskDuration(); - checkPendingCompletionTasks(); - checkCurrentTaskState(); - - if (!spec.isSuspended()) { - log.info("[%s] supervisor is running.", dataSource); - createNewTasks(); - } else { - log.info("[%s] supervisor is suspended.", dataSource); - gracefulShutdownInternal(); - } - - if (log.isDebugEnabled()) { - log.debug(generateReport(true).toString()); - } else { - log.info(generateReport(false).toString()); - } - } - - @VisibleForTesting - String generateSequenceName(int groupId) - { - StringBuilder sb = new StringBuilder(); - Map startPartitions = taskGroups.get(groupId).partitionOffsets; - - for (Map.Entry entry : startPartitions.entrySet()) { - sb.append(String.format("+%s(%s)", entry.getKey(), entry.getValue())); - } - String partitionOffsetStr = sb.toString().substring(1); - - Optional minimumMessageTime = taskGroups.get(groupId).minimumMessageTime; - Optional maximumMessageTime = taskGroups.get(groupId).maximumMessageTime; - String minMsgTimeStr = (minimumMessageTime.isPresent() ? String.valueOf(minimumMessageTime.get().getMillis()) : ""); - String maxMsgTimeStr = (maximumMessageTime.isPresent() ? String.valueOf(maximumMessageTime.get().getMillis()) : ""); - - String dataSchema, tuningConfig; - try { - dataSchema = sortingMapper.writeValueAsString(spec.getDataSchema()); - tuningConfig = sortingMapper.writeValueAsString(taskTuningConfig); - } - catch (JsonProcessingException e) { - throw Throwables.propagate(e); - } - - String hashCode = DigestUtils.sha1Hex(dataSchema + tuningConfig + partitionOffsetStr + minMsgTimeStr + maxMsgTimeStr) - .substring(0, 15); - - return Joiner.on("_").join("index_kinesis", dataSource, hashCode); - } - - private static String getRandomId() - { - final StringBuilder suffix = new StringBuilder(8); - for (int i = 0; i < Ints.BYTES * 2; ++i) { - suffix.append((char) ('a' + ((RANDOM.nextInt() >>> (i * 4)) & 0x0F))); - } - return suffix.toString(); - } - - private RecordSupplier setupRecordSupplier() - { - if (recordSupplier == null) { - recordSupplier = new KinesisRecordSupplier( - ioConfig.getEndpoint(), - ioConfig.getAwsAccessKeyId(), - ioConfig.getAwsSecretAccessKey(), - ioConfig.getRecordsPerFetch(), - ioConfig.getFetchDelayMillis(), - 1, - ioConfig.getAwsAssumedRoleArn(), - ioConfig.getAwsExternalId(), - ioConfig.isDeaggregate(), - taskTuningConfig.getRecordBufferSize(), - taskTuningConfig.getRecordBufferOfferTimeout(), - taskTuningConfig.getRecordBufferFullWait(), - taskTuningConfig.getFetchSequenceNumberTimeout() - ); - } - - return recordSupplier; - } - - private void updatePartitionDataFromKinesis() - { - Set partitionIds = recordSupplier.getPartitionIds(ioConfig.getStream()); - - if (partitionIds == null) { - log.warn("Could not fetch partition IDs for stream[%s]", ioConfig.getStream()); - return; - } - - log.debug("Found [%d] Kinesis partitions for stream [%s]", partitionIds.size(), ioConfig.getStream()); - - Set closedPartitions = getOffsetsFromMetadataStorage() - .entrySet() - .stream() - .filter(x -> Record.END_OF_SHARD_MARKER.equals(x.getValue())) - .map(Map.Entry::getKey) - .collect(Collectors.toSet()); - - boolean initialPartitionDiscovery = this.partitionIds.isEmpty(); - for (String partition : partitionIds) { - if (closedPartitions.contains(partition)) { - continue; - } - - if (!initialPartitionDiscovery && !this.partitionIds.contains(partition)) { - subsequentlyDiscoveredPartitions.add(partition); - - if (earlyPublishTime == null) { - for (TaskGroup taskGroup : taskGroups.values()) { - if (!taskGroup.taskIds().isEmpty()) { - // a new partition was added and we are managing active tasks - set an early publish time 2 minutes in the - // future to give things time to settle - - earlyPublishTime = DateTime.now().plusMinutes(2); - log.info("New partition discovered - requesting early publish in 2 minutes [%s]", earlyPublishTime); - break; - } - } - } - } - - int taskGroupId = getTaskGroupIdForPartition(partition); - partitionGroups.putIfAbsent(taskGroupId, new ConcurrentHashMap<>()); - - ConcurrentHashMap partitionMap = partitionGroups.get(taskGroupId); - - // 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 - // offset value from the metadata store, and if it can't find it there, from Kafka. Once a task begins - // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet- - // completed task, which will cause the next set of tasks to begin reading from where the previous task left - // off. If that previous task now fails, we will set the offset in [partitionGroups] back to NOT_SET which will - // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to - // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task - // failures during publishing. - if (partitionMap.putIfAbsent(partition, NOT_SET) == null) { - log.info( - "New partition [%s] discovered for topic [%s], added to task group [%d]", - partition, - ioConfig.getStream(), - taskGroupId - ); - } - } - } - - private void discoverTasks() throws ExecutionException, InterruptedException, TimeoutException - { - int taskCount = 0; - List futureTaskIds = Lists.newArrayList(); - List> futures = Lists.newArrayList(); - List tasks = taskStorage.getActiveTasks(); - - for (Task task : tasks) { - if (!(task instanceof KinesisIndexTask) || !dataSource.equals(task.getDataSource())) { - continue; - } - - taskCount++; - final KinesisIndexTask kinesisTask = (KinesisIndexTask) task; - final String taskId = task.getId(); - - // Determine which task group this task belongs to based on one of the partitions handled by this task. If we - // later determine that this task is actively reading, we will make sure that it matches our current partition - // allocation (getTaskGroupIdForPartition(partition) should return the same value for every partition being read - // by this task) and kill it if it is not compatible. If the task is instead found to be in the publishing - // state, we will permit it to complete even if it doesn't match our current partition allocation to support - // seamless schema migration. - - Iterator it = kinesisTask.getIOConfig() - .getStartPartitions() - .getPartitionSequenceNumberMap() - .keySet() - .iterator(); - final Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null); - - if (taskGroupId != null) { - // check to see if we already know about this task, either in [taskGroups] or in [pendingCompletionTaskGroups] - // and if not add it to taskGroups or pendingCompletionTaskGroups (if status = PUBLISHING) - TaskGroup taskGroup = taskGroups.get(taskGroupId); - if (!isTaskInPendingCompletionGroups(taskId) && (taskGroup == null || !taskGroup.tasks.containsKey(taskId))) { - - futureTaskIds.add(taskId); - futures.add( - Futures.transform( - taskClient.getStatusAsync(taskId), new Function() - { - @Override - public Boolean apply(KinesisIndexTask.Status status) - { - if (status == KinesisIndexTask.Status.PUBLISHING) { - addDiscoveredTaskToPendingCompletionTaskGroups( - taskGroupId, - taskId, - kinesisTask.getIOConfig() - .getStartPartitions() - .getPartitionSequenceNumberMap() - ); - - // update partitionGroups with the publishing task's offsets (if they are greater than what is - // existing) so that the next tasks will start reading from where this task left off - Map publishingTaskCurrentOffsets = taskClient.getCurrentOffsets(taskId, true); - - for (Map.Entry entry : publishingTaskCurrentOffsets.entrySet()) { - String partition = entry.getKey(); - String offset = entry.getValue(); - ConcurrentHashMap partitionOffsets = partitionGroups.get( - getTaskGroupIdForPartition(partition) - ); - - boolean succeeded; - do { - succeeded = true; - String previousOffset = partitionOffsets.putIfAbsent(partition, offset); - if (previousOffset != null && previousOffset.compareTo(offset) < 0) { - succeeded = partitionOffsets.replace(partition, previousOffset, offset); - } - } while (!succeeded); - } - - } else { - for (String partition : kinesisTask.getIOConfig() - .getStartPartitions() - .getPartitionSequenceNumberMap() - .keySet()) { - if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { - log.warn( - "Stopping task [%s] which does not match the expected partition allocation", - taskId - ); - try { - stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - } - catch (InterruptedException | ExecutionException | TimeoutException e) { - log.warn(e, "Exception while stopping task"); - } - return false; - } - } - - if (taskGroups.putIfAbsent( - taskGroupId, - new TaskGroup( - ImmutableMap.copyOf( - kinesisTask.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap() - ), - kinesisTask.getIOConfig().getMinimumMessageTime(), - kinesisTask.getIOConfig().getMaximumMessageTime(), - kinesisTask.getIOConfig().getExclusiveStartSequenceNumberPartitions() - ) - ) == null) { - log.debug("Created new task group [%d]", taskGroupId); - } - - if (!isTaskCurrent(taskGroupId, taskId)) { - log.info( - "Stopping task [%s] which does not match the expected parameters and ingestion spec", - taskId - ); - try { - stopTask(taskId, false).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - } - catch (InterruptedException | ExecutionException | TimeoutException e) { - log.warn(e, "Exception while stopping task"); - } - return false; - } else { - taskGroups.get(taskGroupId).tasks.putIfAbsent(taskId, new TaskData()); - } - } - return true; - } - }, workerExec - ) - ); - } - } - } - - List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - for (int i = 0; i < results.size(); i++) { - if (results.get(i) == null) { - String taskId = futureTaskIds.get(i); - log.warn("Task [%s] failed to return status, killing task", taskId); - killTask(taskId); - } - } - log.debug("Found [%d] Kafka indexing tasks for dataSource [%s]", taskCount, dataSource); - } - - private void addDiscoveredTaskToPendingCompletionTaskGroups( - int groupId, - String taskId, - Map startingPartitions - ) - { - pendingCompletionTaskGroups.putIfAbsent(groupId, Lists.newCopyOnWriteArrayList()); - - CopyOnWriteArrayList taskGroupList = pendingCompletionTaskGroups.get(groupId); - for (TaskGroup taskGroup : taskGroupList) { - if (taskGroup.partitionOffsets.equals(startingPartitions)) { - if (taskGroup.tasks.putIfAbsent(taskId, new TaskData()) == null) { - log.info("Added discovered task [%s] to existing pending task group", taskId); - } - return; - } - } - - log.info("Creating new pending completion task group for discovered task [%s]", taskId); - - // reading the minimumMessageTime & maximumMessageTime from the publishing task and setting it here is not necessary as this task cannot - // change to a state where it will read any more events - TaskGroup newTaskGroup = new TaskGroup(ImmutableMap.copyOf(startingPartitions), Optional.absent(), Optional.absent(), null); - - newTaskGroup.tasks.put(taskId, new TaskData()); - newTaskGroup.completionTimeout = DateTime.now().plus(ioConfig.getCompletionTimeout()); - - taskGroupList.add(newTaskGroup); - } - - private void updateTaskStatus() throws ExecutionException, InterruptedException, TimeoutException - { - final List> futures = Lists.newArrayList(); - final List futureTaskIds = Lists.newArrayList(); - - // update status (and startTime if unknown) of current tasks in taskGroups - for (TaskGroup group : taskGroups.values()) { - for (Map.Entry entry : group.tasks.entrySet()) { - final String taskId = entry.getKey(); - final TaskData taskData = entry.getValue(); - - if (taskData.startTime == null) { - futureTaskIds.add(taskId); - futures.add( - Futures.transform( - taskClient.getStartTimeAsync(taskId), new Function() - { - @Nullable - @Override - public Boolean apply(@Nullable DateTime startTime) - { - if (startTime == null) { - return false; - } - - taskData.startTime = startTime; - long millisRemaining = ioConfig.getTaskDuration().getMillis() - (System.currentTimeMillis() - - taskData.startTime.getMillis()); - if (millisRemaining > 0) { - scheduledExec.schedule( - buildRunTask(), - millisRemaining + MAX_RUN_FREQUENCY_MILLIS, - TimeUnit.MILLISECONDS - ); - } - - return true; - } - }, workerExec - ) - ); - } - - taskData.status = taskStorage.getStatus(taskId).get(); - } - } - - // update status of pending completion tasks in pendingCompletionTaskGroups - for (List taskGroups : pendingCompletionTaskGroups.values()) { - for (TaskGroup group : taskGroups) { - for (Map.Entry entry : group.tasks.entrySet()) { - entry.getValue().status = taskStorage.getStatus(entry.getKey()).get(); - } - } - } - - List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - for (int i = 0; i < results.size(); i++) { - // false means the task hasn't started running yet and that's okay; null means it should be running but the HTTP - // request threw an exception so kill the task - if (results.get(i) == null) { - String taskId = futureTaskIds.get(i); - log.warn("Task [%s] failed to return start time, killing task", taskId); - killTask(taskId); - } - } - } - - private void checkTaskDuration() - throws InterruptedException, ExecutionException, TimeoutException - { - final List>> futures = Lists.newArrayList(); - final List futureGroupIds = Lists.newArrayList(); - - for (Map.Entry entry : taskGroups.entrySet()) { - Integer groupId = entry.getKey(); - TaskGroup group = entry.getValue(); - - // find the longest running task from this group - DateTime earliestTaskStart = DateTime.now(); - for (TaskData taskData : group.tasks.values()) { - if (earliestTaskStart.isAfter(taskData.startTime)) { - earliestTaskStart = taskData.startTime; - } - } - - boolean doEarlyPublish = false; - if (earlyPublishTime != null && (earlyPublishTime.isBeforeNow() || earlyPublishTime.isEqualNow())) { - log.info("Early publish requested - signalling tasks to publish"); - - earlyPublishTime = null; - doEarlyPublish = true; - } - - // if this task has run longer than the configured duration, signal all tasks in the group to persist - if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow() || doEarlyPublish) { - log.info("Task group [%d] has run for [%s]", groupId, ioConfig.getTaskDuration()); - futureGroupIds.add(groupId); - futures.add(signalTasksToFinish(groupId)); - } - } - - List> results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - for (int j = 0; j < results.size(); j++) { - Integer groupId = futureGroupIds.get(j); - TaskGroup group = taskGroups.get(groupId); - Map endOffsets = results.get(j); - - if (endOffsets != null) { - // set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion - group.completionTimeout = DateTime.now().plus(ioConfig.getCompletionTimeout()); - pendingCompletionTaskGroups.putIfAbsent(groupId, Lists.newCopyOnWriteArrayList()); - pendingCompletionTaskGroups.get(groupId).add(group); - - // set endOffsets as the next startOffsets - for (Map.Entry entry : endOffsets.entrySet()) { - partitionGroups.get(groupId).put(entry.getKey(), entry.getValue()); - } - } else { - log.warn( - "All tasks in group [%s] failed to transition to publishing state, killing tasks [%s]", - groupId, - group.taskIds() - ); - for (String id : group.taskIds()) { - killTask(id); - } - } - - // remove this task group from the list of current task groups now that it has been handled - taskGroups.remove(groupId); - } - } - - private ListenableFuture> signalTasksToFinish(final int groupId) - { - final TaskGroup taskGroup = taskGroups.get(groupId); - - // 1) Check if any task completed (in which case we're done) and kill unassigned tasks - Iterator> i = taskGroup.tasks.entrySet().iterator(); - while (i.hasNext()) { - Map.Entry taskEntry = i.next(); - String taskId = taskEntry.getKey(); - TaskData task = taskEntry.getValue(); - - if (task.status.isSuccess()) { - // If any task in this group has already completed, stop the rest of the tasks in the group and return. - // This will cause us to create a new set of tasks next cycle that will start from the offsets in - // 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) - return Futures.transform( - stopTasksInGroup(taskGroup), new Function>() - { - @Nullable - @Override - public Map apply(@Nullable Object input) - { - return null; - } - } - ); - } - - if (task.status.isRunnable()) { - if (taskInfoProvider.getTaskLocation(taskId).equals(TaskLocation.unknown())) { - log.info("Killing task [%s] which hasn't been assigned to a worker", taskId); - killTask(taskId); - i.remove(); - } - } - } - - // 2) Pause running tasks - final List>> pauseFutures = Lists.newArrayList(); - final List pauseTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); - for (final String taskId : pauseTaskIds) { - pauseFutures.add(taskClient.pauseAsync(taskId)); - } - - return Futures.transform( - Futures.successfulAsList(pauseFutures), new Function>, Map>() - { - @Nullable - @Override - public Map apply(List> input) - { - // 3) Build a map of the highest offset read by any task in the group for each partition - final Map endOffsets = new HashMap<>(); - for (int i = 0; i < input.size(); i++) { - Map result = input.get(i); - - if (result == null || result.isEmpty()) { // kill tasks that didn't return a value - String taskId = pauseTaskIds.get(i); - log.warn("Task [%s] failed to respond to [pause] in a timely manner, killing task", taskId); - killTask(taskId); - taskGroup.tasks.remove(taskId); - - } else { // otherwise build a map of the highest offsets seen - for (Map.Entry offset : result.entrySet()) { - if (!endOffsets.containsKey(offset.getKey()) - || endOffsets.get(offset.getKey()).compareTo(offset.getValue()) < 0) { - endOffsets.put(offset.getKey(), offset.getValue()); - } - } - } - } - - // 4) Set the end offsets for each task to the values from step 3 and resume the tasks. All the tasks should - // finish reading and start publishing within a short period, depending on how in sync the tasks were. - final List> setEndOffsetFutures = Lists.newArrayList(); - final List setEndOffsetTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); - - if (setEndOffsetTaskIds.isEmpty()) { - log.info("All tasks in taskGroup [%d] have failed, tasks will be re-created", groupId); - return null; - } - - log.info("Setting endOffsets for tasks in taskGroup [%d] to %s and resuming", groupId, endOffsets); - for (final String taskId : setEndOffsetTaskIds) { - setEndOffsetFutures.add(taskClient.setEndOffsetsAsync(taskId, endOffsets, true)); - } - - try { - List results = Futures.successfulAsList(setEndOffsetFutures) - .get(futureTimeoutInSeconds, TimeUnit.SECONDS); - for (int i = 0; i < results.size(); i++) { - if (results.get(i) == null || !results.get(i)) { - String taskId = setEndOffsetTaskIds.get(i); - log.warn("Task [%s] failed to respond to [set end offsets] in a timely manner, killing task", taskId); - killTask(taskId); - taskGroup.tasks.remove(taskId); - } - } - } - catch (Exception e) { - Throwables.propagate(e); - } - - if (taskGroup.tasks.isEmpty()) { - log.info("All tasks in taskGroup [%d] have failed, tasks will be re-created", groupId); - return null; - } - - return endOffsets; - } - }, workerExec - ); - } + private final KinesisSupervisorSpec spec; - /** - * Monitors [pendingCompletionTaskGroups] for tasks that have completed. If any task in a task group has completed, we - * can safely stop the rest of the tasks in that group. If a task group has exceeded its publishing timeout, then - * we need to stop all tasks in not only that task group but also 1) any subsequent task group that is also pending - * completion and 2) the current task group that is running, because the assumption that we have handled up to the - * starting offset for subsequent task groups is no longer valid, and subsequent tasks would fail as soon as they - * attempted to publish because of the contiguous range consistency check. - */ - private void checkPendingCompletionTasks() throws ExecutionException, InterruptedException, TimeoutException + public KinesisSupervisor( + final TaskStorage taskStorage, + final TaskMaster taskMaster, + final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + final KinesisIndexTaskClientFactory taskClientFactory, + final ObjectMapper mapper, + final KinesisSupervisorSpec spec, + final RowIngestionMetersFactory rowIngestionMetersFactory + ) { - List> futures = Lists.newArrayList(); - - for (Map.Entry> pendingGroupList : pendingCompletionTaskGroups.entrySet()) { - - boolean stopTasksInTaskGroup = false; - Integer groupId = pendingGroupList.getKey(); - CopyOnWriteArrayList taskGroupList = pendingGroupList.getValue(); - List toRemove = Lists.newArrayList(); - - for (TaskGroup group : taskGroupList) { - boolean foundSuccess = false, entireTaskGroupFailed = false; - - if (stopTasksInTaskGroup) { - // One of the earlier groups that was handling the same partition set timed out before the segments were - // published so stop any additional groups handling the same partition set that are pending completion. - futures.add(stopTasksInGroup(group)); - toRemove.add(group); - continue; - } - - Iterator> iTask = group.tasks.entrySet().iterator(); - while (iTask.hasNext()) { - Map.Entry task = iTask.next(); - - if (task.getValue().status.isFailure()) { - iTask.remove(); // remove failed task - if (group.tasks.isEmpty()) { - // if all tasks in the group have failed, just nuke all task groups with this partition set and restart - entireTaskGroupFailed = true; - break; - } - } - - if (task.getValue().status.isSuccess()) { - // If one of the pending completion tasks was successful, stop the rest of the tasks in the group as - // we no longer need them to publish their segment. - log.info("Task [%s] completed successfully, stopping tasks %s", task.getKey(), group.taskIds()); - futures.add(stopTasksInGroup(group)); - foundSuccess = true; - toRemove.add(group); // remove the TaskGroup from the list of pending completion task groups - break; // skip iterating the rest of the tasks in this group as they've all been stopped now - } - } - - if ((!foundSuccess && group.completionTimeout.isBeforeNow()) || entireTaskGroupFailed) { - if (entireTaskGroupFailed) { - log.warn("All tasks in group [%d] failed to publish, killing all tasks for these partitions", groupId); - } else { - log.makeAlert( - "No task in [%s] succeeded before the completion timeout elapsed [%s]!", - group.taskIds(), - ioConfig.getCompletionTimeout() - ).emit(); - } - - // reset partitions offsets for this task group so that they will be re-read from metadata storage - partitionGroups.remove(groupId); - - // stop all the tasks in this pending completion group - futures.add(stopTasksInGroup(group)); - - // set a flag so the other pending completion groups for this set of partitions will also stop - stopTasksInTaskGroup = true; - - // stop all the tasks in the currently reading task group and remove the bad task group - futures.add(stopTasksInGroup(taskGroups.remove(groupId))); - - toRemove.add(group); - } - } - - taskGroupList.removeAll(toRemove); - } + super( + StringUtils.format("KinesisSupervisor-%s", spec.getDataSchema().getDataSource()), + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + mapper, + spec, + rowIngestionMetersFactory, + NOT_SET, + true + ); - // wait for all task shutdowns to complete before returning - Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + this.spec = spec; } - private void checkCurrentTaskState() throws ExecutionException, InterruptedException, TimeoutException - { - List> futures = Lists.newArrayList(); - Iterator> iTaskGroups = taskGroups.entrySet().iterator(); - while (iTaskGroups.hasNext()) { - Map.Entry taskGroupEntry = iTaskGroups.next(); - Integer groupId = taskGroupEntry.getKey(); - TaskGroup taskGroup = taskGroupEntry.getValue(); - - // Iterate the list of known tasks in this group and: - // 1) Kill any tasks which are not "current" (have the partitions, starting offsets, and minimumMessageTime & maximumMessageTime - // (if applicable) in [taskGroups]) - // 2) Remove any tasks that have failed from the list - // 3) If any task completed successfully, stop all the tasks in this group and move to the next group - - log.debug("Task group [%d] pre-pruning: %s", groupId, taskGroup.taskIds()); - - Iterator> iTasks = taskGroup.tasks.entrySet().iterator(); - while (iTasks.hasNext()) { - Map.Entry task = iTasks.next(); - String taskId = task.getKey(); - TaskData taskData = task.getValue(); - - // stop and remove bad tasks from the task group - if (!isTaskCurrent(groupId, taskId)) { - log.info("Stopping task [%s] which does not match the expected offset range and ingestion spec", taskId); - futures.add(stopTask(taskId, false)); - iTasks.remove(); - continue; - } - - // remove failed tasks - if (taskData.status.isFailure()) { - iTasks.remove(); - continue; - } - - // check for successful tasks, and if we find one, stop all tasks in the group and remove the group so it can - // be recreated with the next set of offsets - if (taskData.status.isSuccess()) { - futures.add(stopTasksInGroup(taskGroup)); - iTaskGroups.remove(); - break; - } - } - log.debug("Task group [%d] post-pruning: %s", groupId, taskGroup.taskIds()); - } - - // wait for all task shutdowns to complete before returning - Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - } - void createNewTasks() + @Override + public void checkpoint( + @Nullable Integer taskGroupId, + @Deprecated String baseSequenceName, + DataSourceMetadata previousCheckPoint, + DataSourceMetadata currentCheckPoint + ) { - // check that there is a current task group for each group of partitions in [partitionGroups] - for (Integer groupId : partitionGroups.keySet()) { - if (!taskGroups.containsKey(groupId)) { - log.info( - "Creating new task group [%d] for partitions %s", - groupId, - partitionGroups.get(groupId).keySet() - ); - - Optional minimumMessageTime = (ioConfig.getLateMessageRejectionPeriod().isPresent() ? Optional.of( - DateTime.now().minus(ioConfig.getLateMessageRejectionPeriod().get()) - ) : Optional.absent()); - - Optional maximumMessageTime = (ioConfig.getEarlyMessageRejectionPeriod().isPresent() ? Optional.of( - DateTime.now().plus(ioConfig.getEarlyMessageRejectionPeriod().get()) - ) : Optional.absent()); - - try { - Map startingOffsets = generateStartingOffsetsForPartitionGroup(groupId); - - Map simpleStartingOffsets = startingOffsets - .entrySet().stream() - .filter(x -> x.getValue().get() != null) - .collect(Collectors.toMap(Map.Entry::getKey, x -> x.getValue().get())); - - Set exclusiveStartSequenceNumberPartitions = startingOffsets - .entrySet().stream() - .filter(x -> x.getValue().get() != null && x.getValue().isExclusive()) - .map(Map.Entry::getKey) - .collect(Collectors.toSet()); - - taskGroups.put( - groupId, - new TaskGroup( - simpleStartingOffsets, - minimumMessageTime, - maximumMessageTime, - exclusiveStartSequenceNumberPartitions - ) - ); - } - catch (TimeoutException e) { - log.warn( - e, - "Timeout while fetching sequence numbers - if you are reading from the latest sequence number, you need to write events to the stream before the sequence number can be determined" - ); - } - } - } - - // iterate through all the current task groups and make sure each one has the desired number of replica tasks - boolean createdTask = false; - for (Map.Entry entry : taskGroups.entrySet()) { - TaskGroup taskGroup = entry.getValue(); - Integer groupId = entry.getKey(); - - if (taskGroup.partitionOffsets == null || taskGroup.partitionOffsets - .values().stream().allMatch(x -> x == null || Record.END_OF_SHARD_MARKER.equals(x))) { - log.debug("Nothing to read in any partition for taskGroup [%d], skipping task creation", groupId); - continue; - } - - if (ioConfig.getReplicas() > taskGroup.tasks.size()) { - log.info( - "Number of tasks [%d] does not match configured numReplicas [%d] in task group [%d], creating more tasks", - taskGroup.tasks.size(), ioConfig.getReplicas(), groupId - ); - createKinesisTasksForGroup(groupId, ioConfig.getReplicas() - taskGroup.tasks.size()); - createdTask = true; - } - } - - if (createdTask && firstRunTime.isBeforeNow()) { - // Schedule a run event after a short delay to update our internal data structures with the new tasks that were - // just created. This is mainly for the benefit of the status API in situations where the run period is lengthy. - scheduledExec.schedule(buildRunTask(), 5000, TimeUnit.MILLISECONDS); - } + // not supported right now + throw new UnsupportedOperationException("kinesis supervisor does not yet support checkpoints"); } - private void createKinesisTasksForGroup(int groupId, int replicas) + @Override + protected SeekableStreamIOConfig createIoConfig( + int groupId, + Map startPartitions, + Map endPartitions, + String baseSequenceName, + DateTime minimumMessageTime, + DateTime maximumMessageTime, + Set exclusiveStartSequenceNumberPartitions, + SeekableStreamSupervisorIOConfig ioConfigg + ) { - Map startPartitions = taskGroups.get(groupId).partitionOffsets; - Map endPartitions = new HashMap<>(); - Set exclusiveStartSequenceNumberPartitions = taskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; - for (String partition : startPartitions.keySet()) { - endPartitions.put(partition, KinesisPartitions.NO_END_SEQUENCE_NUMBER); - } - - String sequenceName = generateSequenceName(groupId); - - DateTime minimumMessageTime = taskGroups.get(groupId).minimumMessageTime.orNull(); - DateTime maximumMessageTime = taskGroups.get(groupId).maximumMessageTime.orNull(); - - KinesisIOConfig kinesisIOConfig = new KinesisIOConfig( - sequenceName, + KinesisSupervisorIOConfig ioConfig = (KinesisSupervisorIOConfig) ioConfigg; + return new KinesisIOConfig( + baseSequenceName, new KinesisPartitions(ioConfig.getStream(), startPartitions), new KinesisPartitions(ioConfig.getStream(), endPartitions), true, @@ -1546,460 +141,137 @@ private void createKinesisTasksForGroup(int groupId, int replicas) ioConfig.getAwsExternalId(), ioConfig.isDeaggregate() ); + } + @Override + protected List> createIndexTasks( + int replicas, + String baseSequenceName, + ObjectMapper sortingMapper, + TreeMap> sequenceOffsets, + SeekableStreamIOConfig taskIoConfig, + SeekableStreamTuningConfig taskTuningConfig, + RowIngestionMetersFactory rowIngestionMetersFactory + ) + { + List> taskList = new ArrayList<>(); for (int i = 0; i < replicas; i++) { - String taskId = Joiner.on("_").join(sequenceName, getRandomId()); - KinesisIndexTask indexTask = new KinesisIndexTask( + String taskId = Joiner.on("_").join(baseSequenceName, getRandomId()); + taskList.add(new KinesisIndexTask( taskId, - new TaskResource(sequenceName, 1), + new TaskResource(baseSequenceName, 1), spec.getDataSchema(), - taskTuningConfig, - kinesisIOConfig, + (KinesisTuningConfig) taskTuningConfig, + (KinesisIOConfig) taskIoConfig, spec.getContext(), null, null, rowIngestionMetersFactory - ); - - Optional taskQueue = taskMaster.getTaskQueue(); - if (taskQueue.isPresent()) { - try { - taskQueue.get().add(indexTask); - } - catch (EntryExistsException e) { - log.error("Tried to add task [%s] but it already exists", indexTask.getId()); - } - } else { - log.error("Failed to get task queue because I'm not the leader!"); - } - } - } - - private ImmutableMap generateStartingOffsetsForPartitionGroup(int groupId) - throws TimeoutException - { - ImmutableMap.Builder builder = ImmutableMap.builder(); - for (Map.Entry entry : partitionGroups.get(groupId).entrySet()) { - String partition = entry.getKey(); - String offset = entry.getValue(); - - if (offset != null && !NOT_SET.equals(offset)) { - // if we are given a startingOffset (set by a previous task group which is pending completion) then use it - if (!Record.END_OF_SHARD_MARKER.equals(offset)) { - builder.put(partition, SequenceNumberPlus.of(offset, true)); - } - } else { - // if we don't have a startingOffset (first run or we had some previous failures and reset the offsets) then - // get the offset from metadata storage (if available) or Kafka (otherwise) - - SequenceNumberPlus offsetFromStorage = getOffsetFromStorageForPartition(partition); - - if (offsetFromStorage != null && !Record.END_OF_SHARD_MARKER.equals(offsetFromStorage.get())) { - builder.put(partition, offsetFromStorage); - } - } - } - return builder.build(); - } - - /** - * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't - * find any data, it will retrieve the latest or earliest Kinesis sequence number depending on the - * useEarliestSequenceNumber config. - */ - private SequenceNumberPlus getOffsetFromStorageForPartition(String partition) throws TimeoutException - { - Map metadataOffsets = getOffsetsFromMetadataStorage(); - String offset = metadataOffsets.get(partition); - - if (offset != null) { - log.debug("Getting sequence number [%s] from metadata storage for partition [%s]", offset, partition); - - if (!tuningConfig.isSkipSequenceNumberAvailabilityCheck()) { - final StreamPartition streamPartition = StreamPartition.of(ioConfig.getStream(), partition); - try { - String earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); - if (earliestSequenceNumber == null || earliestSequenceNumber.compareTo(offset) > 0) { - if (tuningConfig.isResetOffsetAutomatically()) { - resetInternal( - new KinesisDataSourceMetadata( - new KinesisPartitions(ioConfig.getStream(), ImmutableMap.of(partition, offset)) - ) - ); - throw new ISE( - "Previous sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) - automatically resetting offset", - offset, - partition, - earliestSequenceNumber - ); - - } else { - throw new ISE( - "Previous sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]). You can clear the previous sequenceNumber and start reading from a valid message by using the supervisor's reset API.", - offset, - partition, - earliestSequenceNumber - ); - } - } - } - catch (TimeoutException e) { - throw new ISE(e, "Timeout while fetching earliest sequence number for partition [%s]", partition); - } - } - - // Removed check comparing these offsets to the latest in Kinesis because it's potentially quite expensive - if no - // data has been written recently the call will block until timeout. - - return SequenceNumberPlus.of(offset, true); - - } else { - boolean useEarliestSequenceNumber = ioConfig.isUseEarliestSequenceNumber(); - if (subsequentlyDiscoveredPartitions.contains(partition)) { - log.info( - "Overriding useEarliestSequenceNumber and starting from beginning of newly discovered partition [%s] (which is probably from a split or merge)", - partition - ); - useEarliestSequenceNumber = true; - } - - offset = getOffsetFromKinesisForPartition(partition, useEarliestSequenceNumber); - log.info("Getting sequence number [%s] from Kinesis for partition [%s]", offset, partition); - return SequenceNumberPlus.of(offset, false); - } - } - - private Map getOffsetsFromMetadataStorage() - { - DataSourceMetadata dataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); - if (dataSourceMetadata != null && dataSourceMetadata instanceof KinesisDataSourceMetadata) { - KinesisPartitions partitions = ((KinesisDataSourceMetadata) dataSourceMetadata).getKinesisPartitions(); - if (partitions != null) { - if (!ioConfig.getStream().equals(partitions.getStream())) { - log.warn( - "Stream name in metadata storage [%s] doesn't match spec stream name [%s], ignoring stored sequence numbers", - partitions.getStream(), - ioConfig.getStream() - ); - return ImmutableMap.of(); - } else if (partitions.getPartitionSequenceNumberMap() != null) { - return partitions.getPartitionSequenceNumberMap(); - } - } + )); } - - return ImmutableMap.of(); + return taskList; } - private String getOffsetFromKinesisForPartition(String partition, boolean useEarliestSequenceNumber) - throws TimeoutException + @Override + protected RecordSupplier setupRecordSupplier() { - log.info( - "Fetching starting sequence number from Kinesis with useEarliestSequenceNumber=%s", - useEarliestSequenceNumber - ? "true. If there is no data in the stream, Kinesis will not return a sequence number and this call will fail with a timeout." - : "false. If there is no new data coming into the stream, Kinesis will not return a sequence number and this call will fail with a timeout." + KinesisSupervisorIOConfig ioConfig = spec.getIoConfig(); + KinesisTuningConfig taskTuningConfig = spec.getTuningConfig(); + return new KinesisRecordSupplier( + ioConfig.getEndpoint(), + ioConfig.getAwsAccessKeyId(), + ioConfig.getAwsSecretAccessKey(), + ioConfig.getRecordsPerFetch(), + ioConfig.getFetchDelayMillis(), + 1, + ioConfig.getAwsAssumedRoleArn(), + ioConfig.getAwsExternalId(), + ioConfig.isDeaggregate(), + taskTuningConfig.getRecordBufferSize(), + taskTuningConfig.getRecordBufferOfferTimeout(), + taskTuningConfig.getRecordBufferFullWait(), + taskTuningConfig.getFetchSequenceNumberTimeout() ); - - return useEarliestSequenceNumber - ? recordSupplier.getEarliestSequenceNumber(StreamPartition.of(ioConfig.getStream(), partition)) - : recordSupplier.getLatestSequenceNumber(StreamPartition.of(ioConfig.getStream(), partition)); - } - - /** - * Compares the sequence name from the task with one generated for the task's group ID and returns false if they do - * not match. The sequence name is generated from a hash of the dataSchema, tuningConfig, starting offsets, and the - * minimumMessageTime or maximumMessageTime if set. - */ - private boolean isTaskCurrent(int taskGroupId, String taskId) - { - Optional taskOptional = taskStorage.getTask(taskId); - if (!taskOptional.isPresent() || !(taskOptional.get() instanceof KinesisIndexTask)) { - return false; - } - - String taskSequenceName = ((KinesisIndexTask) taskOptional.get()).getIOConfig().getBaseSequenceName(); - - return generateSequenceName(taskGroupId).equals(taskSequenceName); - } - - private ListenableFuture stopTasksInGroup(TaskGroup taskGroup) - { - if (taskGroup == null) { - return Futures.immediateFuture(null); - } - - final List> futures = Lists.newArrayList(); - for (Map.Entry entry : taskGroup.tasks.entrySet()) { - if (!entry.getValue().status.isComplete()) { - futures.add(stopTask(entry.getKey(), false)); - } - } - - return Futures.successfulAsList(futures); } - private ListenableFuture stopTask(final String id, final boolean publish) - { - return Futures.transform( - taskClient.stopAsync(id, publish), new Function() - { - @Nullable - @Override - public Void apply(@Nullable Boolean result) - { - if (result == null || !result) { - log.info("Task [%s] failed to stop in a timely manner, killing task", id); - killTask(id); - } - return null; - } - } - ); - } - private void killTask(final String id) + @Override + protected void scheduleReporting(ScheduledExecutorService reportingExec) { - Optional taskQueue = taskMaster.getTaskQueue(); - if (taskQueue.isPresent()) { - taskQueue.get().shutdown(id); - } else { - log.error("Failed to get task queue because I'm not the leader!"); - } + // TODO: Implement this for Kinesis which uses approximate time from latest instead of offset lag +/* + reportingExec.scheduleAtFixedRate( + computeAndEmitLag(taskClient), + ioConfig.getStartDelay().getMillis() + 10000, // wait for tasks to start up + Math.max(monitorSchedulerConfig.getEmitterPeriod().getMillis(), 60 * 1000), + TimeUnit.MILLISECONDS + ); +*/ } - private int getTaskGroupIdForPartition(String partitionId) + @Override + protected int getTaskGroupIdForPartition(String partitionId) { if (!partitionIds.contains(partitionId)) { partitionIds.add(partitionId); } - return partitionIds.indexOf(partitionId) % ioConfig.getTaskCount(); + return partitionIds.indexOf(partitionId) % spec.getIoConfig().getTaskCount(); } - private boolean isTaskInPendingCompletionGroups(String taskId) + @Override + protected boolean checkSourceMetaDataMatch(DataSourceMetadata metadata) { - for (List taskGroups : pendingCompletionTaskGroups.values()) { - for (TaskGroup taskGroup : taskGroups) { - if (taskGroup.tasks.containsKey(taskId)) { - return true; - } - } - } - return false; + return metadata instanceof KinesisDataSourceMetadata; } - private SupervisorReport generateReport(boolean includeOffsets) + @Override + protected boolean checkTaskInstance(Task task) { - int numPartitions = 0; - for (Map partitionGroup : partitionGroups.values()) { - numPartitions += partitionGroup.size(); - } + return task instanceof KinesisIndexTask; + } - KinesisSupervisorReportPayload payload = new KinesisSupervisorReportPayload( - dataSource, + @Override + protected SeekableStreamSupervisorReportPayload createReportPayload( + int numPartitions, boolean includeOffsets + ) + { + KinesisSupervisorIOConfig ioConfig = spec.getIoConfig(); + return new KinesisSupervisorReportPayload( + spec.getDataSchema().getDataSource(), ioConfig.getStream(), numPartitions, ioConfig.getReplicas(), ioConfig.getTaskDuration().getMillis() / 1000, spec.isSuspended() ); - - SupervisorReport report = new SupervisorReport<>( - dataSource, - DateTimes.nowUtc(), - payload - ); - - List taskReports = Lists.newArrayList(); - List>> futures = Lists.newArrayList(); - - try { - for (TaskGroup taskGroup : taskGroups.values()) { - for (Map.Entry entry : taskGroup.tasks.entrySet()) { - String taskId = entry.getKey(); - DateTime startTime = entry.getValue().startTime; - Long remainingSeconds = null; - if (startTime != null) { - remainingSeconds = Math.max( - 0, ioConfig.getTaskDuration().getMillis() - (DateTime.now().getMillis() - startTime.getMillis()) - ) / 1000; - } - - taskReports.add( - new TaskReportData( - taskId, - (includeOffsets ? taskGroup.partitionOffsets : null), - null, - startTime, - remainingSeconds, - TaskReportData.TaskType.ACTIVE - ) - ); - - if (includeOffsets) { - futures.add(taskClient.getCurrentOffsetsAsync(taskId, false)); - } - } - } - - for (List taskGroups : pendingCompletionTaskGroups.values()) { - for (TaskGroup taskGroup : taskGroups) { - for (Map.Entry entry : taskGroup.tasks.entrySet()) { - String taskId = entry.getKey(); - DateTime startTime = entry.getValue().startTime; - Long remainingSeconds = null; - if (taskGroup.completionTimeout != null) { - remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - DateTime.now().getMillis()) - / 1000; - } - - taskReports.add( - new TaskReportData( - taskId, - (includeOffsets ? taskGroup.partitionOffsets : null), - null, - startTime, - remainingSeconds, - TaskReportData.TaskType.PUBLISHING - ) - ); - - if (includeOffsets) { - futures.add(taskClient.getCurrentOffsetsAsync(taskId, false)); - } - } - } - } - - List> results = Futures.successfulAsList(futures) - .get(futureTimeoutInSeconds, TimeUnit.SECONDS); - for (int i = 0; i < taskReports.size(); i++) { - TaskReportData reportData = taskReports.get(i); - if (includeOffsets) { - reportData.setCurrentSequenceNumbers(results.get(i)); - } - payload.addTask(reportData); - } - } - catch (Exception e) { - log.warn(e, "Failed to generate status report"); - } - - return report; } - private Runnable buildRunTask() + // not yet supported, will be implemented in the future + @Override + protected Map getLagPerPartition(Map currentOffsets) { - return new Runnable() - { - @Override - public void run() - { - notices.add(new RunNotice()); - } - }; + return ImmutableMap.of(); } - /** - * Collect row ingestion stats from all tasks managed by this supervisor. - * - * @return A map of groupId->taskId->task row stats - * - * @throws InterruptedException - * @throws ExecutionException - * @throws TimeoutException - */ - private Map> getCurrentTotalStats() throws InterruptedException, ExecutionException, TimeoutException + @Override + protected SeekableStreamDataSourceMetadata createDataSourceMetaData( + String stream, Map map + ) { - Map> allStats = Maps.newHashMap(); - final List> futures = new ArrayList<>(); - final List> groupAndTaskIds = new ArrayList<>(); - - for (int groupId : taskGroups.keySet()) { - TaskGroup group = taskGroups.get(groupId); - for (String taskId : group.taskIds()) { - futures.add( - Futures.transform( - taskClient.getMovingAveragesAsync(taskId), - (Function, StatsFromTaskResult>) (currentStats) -> { - return new StatsFromTaskResult( - groupId, - taskId, - currentStats - ); - } - ) - ); - groupAndTaskIds.add(new Pair<>(groupId, taskId)); - } - } - - for (int groupId : pendingCompletionTaskGroups.keySet()) { - TaskGroup group = taskGroups.get(groupId); - for (String taskId : group.taskIds()) { - futures.add( - Futures.transform( - taskClient.getMovingAveragesAsync(taskId), - (Function, StatsFromTaskResult>) (currentStats) -> { - return new StatsFromTaskResult( - groupId, - taskId, - currentStats - ); - } - ) - ); - groupAndTaskIds.add(new Pair<>(groupId, taskId)); - } - } - - List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - for (int i = 0; i < results.size(); i++) { - StatsFromTaskResult result = results.get(i); - if (result != null) { - Map groupMap = allStats.computeIfAbsent(result.getGroupId(), k -> Maps.newHashMap()); - groupMap.put(result.getTaskId(), result.getStats()); - } else { - Pair groupAndTaskId = groupAndTaskIds.get(i); - log.error("Failed to get stats for group[%d]-task[%s]", groupAndTaskId.lhs, groupAndTaskId.rhs); - } - } - - return allStats; + return new KinesisDataSourceMetadata( + new KinesisPartitions(stream, map) + ); } - private static class StatsFromTaskResult + @Override + protected Map createNewTaskEndPartitions(Set startPartitions) { - private final String groupId; - private final String taskId; - private final Map stats; - - public StatsFromTaskResult( - int groupId, - String taskId, - Map stats - ) - { - this.groupId = String.valueOf(groupId); - this.taskId = taskId; - this.stats = stats; - } - - public String getGroupId() - { - return groupId; - } - - public String getTaskId() - { - return taskId; - } - - public Map getStats() - { - return stats; + Map endPartitions = new HashMap<>(); + for (String partition : startPartitions) { + endPartitions.put(partition, KinesisPartitions.NO_END_SEQUENCE_NUMBER); } + return endPartitions; } diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java index 3937d00aca14..6060729e7aca 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ @@ -21,25 +21,14 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; import org.apache.druid.indexing.kinesis.KinesisRegion; -import org.joda.time.Duration; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.joda.time.Period; -public class KinesisSupervisorIOConfig +public class KinesisSupervisorIOConfig extends SeekableStreamSupervisorIOConfig { - private final String stream; private final String endpoint; - private final Integer replicas; - private final Integer taskCount; - private final Duration taskDuration; - private final Duration startDelay; - private final Duration period; - private final boolean useEarliestSequenceNumber; - private final Duration completionTimeout; - private final Optional lateMessageRejectionPeriod; - private final Optional earlyMessageRejectionPeriod; // In determining a suitable value for recordsPerFetch: // - Each data record can be up to 1 MB in size @@ -83,23 +72,21 @@ public KinesisSupervisorIOConfig( @JsonProperty("deaggregate") boolean deaggregate ) { - this.stream = Preconditions.checkNotNull(stream, "stream cannot be null"); + super( + Preconditions.checkNotNull(stream, "stream"), + replicas, + taskCount, + taskDuration, + startDelay, + period, + useEarliestSequenceNumber, + completionTimeout, + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod + ); this.endpoint = endpoint != null ? endpoint : (region != null ? region.getEndpoint() : KinesisRegion.US_EAST_1.getEndpoint()); - this.replicas = replicas != null ? replicas : 1; - this.taskCount = taskCount != null ? taskCount : 1; - this.taskDuration = defaultDuration(taskDuration, "PT1H"); - this.startDelay = defaultDuration(startDelay, "PT5S"); - this.period = defaultDuration(period, "PT30S"); - this.useEarliestSequenceNumber = useEarliestSequenceNumber != null ? useEarliestSequenceNumber : false; - this.completionTimeout = defaultDuration(completionTimeout, "PT6H"); - this.lateMessageRejectionPeriod = lateMessageRejectionPeriod == null - ? Optional.absent() - : Optional.of(lateMessageRejectionPeriod.toStandardDuration()); - this.earlyMessageRejectionPeriod = earlyMessageRejectionPeriod == null - ? Optional.absent() - : Optional.of(earlyMessageRejectionPeriod.toStandardDuration()); this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : 4000; this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : 0; this.awsAccessKeyId = awsAccessKeyId; @@ -112,7 +99,7 @@ public KinesisSupervisorIOConfig( @JsonProperty public String getStream() { - return stream; + return getId(); } @JsonProperty @@ -121,60 +108,6 @@ public String getEndpoint() return endpoint; } - @JsonProperty - public Integer getReplicas() - { - return replicas; - } - - @JsonProperty - public Integer getTaskCount() - { - return taskCount; - } - - @JsonProperty - public Duration getTaskDuration() - { - return taskDuration; - } - - @JsonProperty - public Duration getStartDelay() - { - return startDelay; - } - - @JsonProperty - public Duration getPeriod() - { - return period; - } - - @JsonProperty - public boolean isUseEarliestSequenceNumber() - { - return useEarliestSequenceNumber; - } - - @JsonProperty - public Duration getCompletionTimeout() - { - return completionTimeout; - } - - @JsonProperty - public Optional getLateMessageRejectionPeriod() - { - return lateMessageRejectionPeriod; - } - - @JsonProperty - public Optional getEarlyMessageRejectionPeriod() - { - return earlyMessageRejectionPeriod; - } - @JsonProperty public Integer getRecordsPerFetch() { @@ -221,17 +154,17 @@ public boolean isDeaggregate() public String toString() { return "KinesisSupervisorIOConfig{" + - "stream='" + stream + '\'' + + "stream='" + getStream() + '\'' + ", endpoint='" + endpoint + '\'' + - ", replicas=" + replicas + - ", taskCount=" + taskCount + - ", taskDuration=" + taskDuration + - ", startDelay=" + startDelay + - ", period=" + period + - ", useEarliestSequenceNumber=" + useEarliestSequenceNumber + - ", completionTimeout=" + completionTimeout + - ", lateMessageRejectionPeriod=" + lateMessageRejectionPeriod + - ", earlyMessageRejectionPeriod=" + earlyMessageRejectionPeriod + + ", replicas=" + getReplicas() + + ", taskCount=" + getTaskCount() + + ", taskDuration=" + getTaskDuration() + + ", startDelay=" + getStartDelay() + + ", period=" + getPeriod() + + ", useEarliestSequenceNumber=" + isUseEarliestSequenceNumber() + + ", completionTimeout=" + getCompletionTimeout() + + ", lateMessageRejectionPeriod=" + getLateMessageRejectionPeriod() + + ", earlyMessageRejectionPeriod=" + getEarlyMessageRejectionPeriod() + ", recordsPerFetch=" + recordsPerFetch + ", fetchDelayMillis=" + fetchDelayMillis + ", awsAccessKeyId='" + awsAccessKeyId + '\'' + @@ -242,8 +175,4 @@ public String toString() '}'; } - private static Duration defaultDuration(final Period period, final String theDefault) - { - return (period == null ? new Period(theDefault) : period).toStandardDuration(); - } } diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java index d539b460f96f..d7c8ebb716fc 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java @@ -1,22 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.indexing.kinesis.supervisor; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.Lists; -import org.apache.druid.java.util.common.IAE; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; -import java.util.List; +import java.util.Collections; -public class KinesisSupervisorReportPayload +public class KinesisSupervisorReportPayload extends SeekableStreamSupervisorReportPayload { - private final String dataSource; - private final String stream; - private final Integer partitions; - private final Integer replicas; - private final Long durationSeconds; - private final List activeTasks; - private final List publishingTasks; - private final boolean suspended; - public KinesisSupervisorReportPayload( String dataSource, String stream, @@ -26,87 +34,33 @@ public KinesisSupervisorReportPayload( boolean suspended ) { - this.dataSource = dataSource; - this.stream = stream; - this.partitions = partitions; - this.replicas = replicas; - this.durationSeconds = durationSeconds; - this.activeTasks = Lists.newArrayList(); - this.publishingTasks = Lists.newArrayList(); - this.suspended = suspended; - } - - @JsonProperty - public String getDataSource() - { - return dataSource; - } - - @JsonProperty - public String getStream() - { - return stream; - } - - @JsonProperty - public Integer getPartitions() - { - return partitions; - } - - @JsonProperty - public Integer getReplicas() - { - return replicas; - } - - @JsonProperty - public Long getDurationSeconds() - { - return durationSeconds; - } - - @JsonProperty - public List getActiveTasks() - { - return activeTasks; - } - - @JsonProperty - public List getPublishingTasks() - { - return publishingTasks; - } - - @JsonProperty - public boolean getSuspended() - { - return suspended; + super( + dataSource, + stream, + partitions, + replicas, + durationSeconds, + Collections.emptyMap(), + Collections.emptyMap(), + null, + null, + suspended + ); } @Override public String toString() { return "{" + - "dataSource='" + dataSource + '\'' + - ", stream='" + stream + '\'' + - ", partitions=" + partitions + - ", replicas=" + replicas + - ", durationSeconds=" + durationSeconds + - ", active=" + activeTasks + - ", publishing=" + publishingTasks + - ", suspended=" + suspended + + "dataSource='" + getDataSource() + '\'' + + ", stream='" + getId() + '\'' + + ", partitions=" + getPartitions() + + ", replicas=" + getReplicas() + + ", durationSeconds=" + getDurationSeconds() + + ", active=" + getActiveTasks() + + ", publishing=" + getPublishingTasks() + + ", suspended=" + getSuspended() + '}'; } - public void addTask(TaskReportData data) - { - if (data.getType().equals(TaskReportData.TaskType.ACTIVE)) { - activeTasks.add(data); - } else if (data.getType().equals(TaskReportData.TaskType.PUBLISHING)) { - publishingTasks.add(data); - } else { - throw new IAE("Unknown task type [%s]", data.getType().name()); - } - } -} \ No newline at end of file +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java index b89748ff1f0b..542e912b6924 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -32,7 +31,7 @@ import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; -import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; @@ -40,23 +39,8 @@ import java.util.List; import java.util.Map; -public class KinesisSupervisorSpec implements SupervisorSpec +public class KinesisSupervisorSpec extends SeekableStreamSupervisorSpec { - private final DataSchema dataSchema; - private final KinesisSupervisorTuningConfig tuningConfig; - private final KinesisSupervisorIOConfig ioConfig; - private final Map context; - private final boolean suspended; - - private final TaskStorage taskStorage; - private final TaskMaster taskMaster; - private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; - private final KinesisIndexTaskClientFactory kinesisIndexTaskClientFactory; - private final ObjectMapper mapper; - private final ServiceEmitter emitter; - private final DruidMonitorSchedulerConfig monitorSchedulerConfig; - private final RowIngestionMetersFactory rowIngestionMetersFactory; - @JsonCreator public KinesisSupervisorSpec( @JsonProperty("dataSchema") DataSchema dataSchema, @@ -74,97 +58,52 @@ public KinesisSupervisorSpec( @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory ) { - this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); - this.tuningConfig = tuningConfig != null - ? tuningConfig - : new KinesisSupervisorTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); - this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); - this.context = context; - this.suspended = suspended != null ? suspended : false; - - this.taskStorage = taskStorage; - this.taskMaster = taskMaster; - this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; - this.kinesisIndexTaskClientFactory = kinesisIndexTaskClientFactory; - this.mapper = mapper; - this.emitter = emitter; - this.monitorSchedulerConfig = monitorSchedulerConfig; - this.rowIngestionMetersFactory = rowIngestionMetersFactory; - } - - @JsonProperty - public DataSchema getDataSchema() - { - return dataSchema; - } - - @JsonProperty - public KinesisSupervisorTuningConfig getTuningConfig() - { - return tuningConfig; - } - - @JsonProperty - public KinesisSupervisorIOConfig getIoConfig() - { - return ioConfig; - } - - @JsonProperty - public Map getContext() - { - return context; - } - - @Override - @JsonProperty("suspended") - public boolean isSuspended() - { - return suspended; - } - - public ServiceEmitter getEmitter() - { - return emitter; - } - - @Override - public String getId() - { - return dataSchema.getDataSource(); + super( + dataSchema, + tuningConfig != null + ? tuningConfig + : new KinesisSupervisorTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ), + ioConfig, + context, + suspended, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + kinesisIndexTaskClientFactory, + mapper, + emitter, + monitorSchedulerConfig, + rowIngestionMetersFactory + ); } - public DruidMonitorSchedulerConfig getMonitorSchedulerConfig() - { - return monitorSchedulerConfig; - } @Override public Supervisor createSupervisor() @@ -173,7 +112,7 @@ public Supervisor createSupervisor() taskStorage, taskMaster, indexerMetadataStorageCoordinator, - kinesisIndexTaskClientFactory, + (KinesisIndexTaskClientFactory) indexTaskClientFactory, mapper, this, rowIngestionMetersFactory @@ -190,10 +129,10 @@ public List getDataSources() public String toString() { return "KinesisSupervisorSpec{" + - "dataSchema=" + dataSchema + - ", tuningConfig=" + tuningConfig + - ", ioConfig=" + ioConfig + - ", suspended=" + suspended + + "dataSchema=" + getDataSchema() + + ", tuningConfig=" + getTuningConfig() + + ", ioConfig=" + getIoConfig() + + ", suspended=" + isSuspended() + '}'; } @@ -209,18 +148,33 @@ public KinesisSupervisorSpec createRunningSpec() return toggleSuspend(false); } - private KinesisSupervisorSpec toggleSuspend(boolean suspend) + @Override + @JsonProperty + public KinesisSupervisorTuningConfig getTuningConfig() + { + return (KinesisSupervisorTuningConfig) super.getTuningConfig(); + } + + @Override + @JsonProperty + public KinesisSupervisorIOConfig getIoConfig() + { + return (KinesisSupervisorIOConfig) super.getIoConfig(); + } + + @Override + protected KinesisSupervisorSpec toggleSuspend(boolean suspend) { return new KinesisSupervisorSpec( - dataSchema, - tuningConfig, - ioConfig, - context, + getDataSchema(), + getTuningConfig(), + getIoConfig(), + getContext(), suspend, taskStorage, taskMaster, indexerMetadataStorageCoordinator, - kinesisIndexTaskClientFactory, + (KinesisIndexTaskClientFactory) indexTaskClientFactory, mapper, emitter, monitorSchedulerConfig, diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 54e1fd67f945..0df0d8d9e3d3 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.kinesis.KinesisTuningConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Duration; @@ -29,7 +30,7 @@ import javax.annotation.Nullable; import java.io.File; -public class KinesisSupervisorTuningConfig extends KinesisTuningConfig +public class KinesisSupervisorTuningConfig extends KinesisTuningConfig implements SeekableStreamSupervisorTuningConfig { private final Integer workerThreads; private final Integer chatThreads; @@ -93,40 +94,51 @@ public KinesisSupervisorTuningConfig( this.workerThreads = workerThreads; this.chatThreads = chatThreads; this.chatRetries = (chatRetries != null ? chatRetries : 8); - this.httpTimeout = defaultDuration(httpTimeout, "PT10S"); - this.shutdownTimeout = defaultDuration(shutdownTimeout, "PT80S"); + this.httpTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(httpTimeout, "PT10S"); + this.shutdownTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(shutdownTimeout, "PT80S"); } + @Override @JsonProperty public Integer getWorkerThreads() { return workerThreads; } + @Override @JsonProperty public Integer getChatThreads() { return chatThreads; } + @Override @JsonProperty public Long getChatRetries() { return chatRetries; } + @Override @JsonProperty public Duration getHttpTimeout() { return httpTimeout; } + @Override @JsonProperty public Duration getShutdownTimeout() { return shutdownTimeout; } + @Override + public Duration getOffsetFetchPeriod() + { + throw new UnsupportedOperationException("kinesis supervisor does not support getOffsetFetchPeriod"); + } + @Override public String toString() { @@ -156,8 +168,4 @@ public String toString() '}'; } - private static Duration defaultDuration(final Period period, final String theDefault) - { - return (period == null ? new Period(theDefault) : period).toStandardDuration(); - } } diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/TaskReportData.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/TaskReportData.java deleted file mode 100644 index d1cae2834d86..000000000000 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/TaskReportData.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.kinesis.supervisor; - -import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.DateTime; - -import java.util.Map; - -public class TaskReportData -{ - public enum TaskType - { - ACTIVE, PUBLISHING, UNKNOWN - } - - private final String id; - private final Map startingSequenceNumbers; - private final DateTime startTime; - private final Long remainingSeconds; - private final TaskType type; - private Map currentSequenceNumbers; - - public TaskReportData( - String id, - Map startingSequenceNumbers, - Map currentSequenceNumbers, - DateTime startTime, - Long remainingSeconds, - TaskType type - ) - { - this.id = id; - this.startingSequenceNumbers = startingSequenceNumbers; - this.currentSequenceNumbers = currentSequenceNumbers; - this.startTime = startTime; - this.remainingSeconds = remainingSeconds; - this.type = type; - } - - @JsonProperty - public String getId() - { - return id; - } - - @JsonProperty - public Map getStartingSequenceNumbers() - { - return startingSequenceNumbers; - } - - @JsonProperty - public Map getCurrentSequenceNumbers() - { - return currentSequenceNumbers; - } - - public void setCurrentSequenceNumbers(Map currentSequenceNumbers) - { - this.currentSequenceNumbers = currentSequenceNumbers; - } - - @JsonProperty - public DateTime getStartTime() - { - return startTime; - } - - @JsonProperty - public Long getRemainingSeconds() - { - return remainingSeconds; - } - - @JsonProperty - public TaskType getType() - { - return type; - } - - @Override - public String toString() - { - return "{" + - "id='" + id + '\'' + - (startingSequenceNumbers != null ? ", startingSequenceNumbers=" + startingSequenceNumbers : "") + - (currentSequenceNumbers != null ? ", currentSequenceNumbers=" + currentSequenceNumbers : "") + - ", startTime=" + startTime + - ", remainingSeconds=" + remainingSeconds + - '}'; - } -} diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index d5c2e3e99dd1..bde56f2832f4 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -1,4 +1,23 @@ + + 4.0.0 diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java index fd094b26375e..e85595944011 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java @@ -39,8 +39,11 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Maps; import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.java.util.common.IAE; +import java.util.Map; import java.util.Objects; public abstract class SeekableStreamDataSourceMetadata implements DataSourceMetadata @@ -70,28 +73,108 @@ public boolean isValidStart() @Override public boolean matches(DataSourceMetadata other) { - if (getClass() != other.getClass()) { + if (!getClass().equals(other.getClass())) { return false; } return plus(other).equals(other.plus(this)); } + @Override - public abstract DataSourceMetadata plus(DataSourceMetadata other); + public DataSourceMetadata plus(DataSourceMetadata other) + { + if (!(this.getClass().isInstance(other))) { + throw new IAE( + "Expected instance of %s, got %s", + this.getClass().getCanonicalName(), + other.getClass().getCanonicalName() + ); + } + + @SuppressWarnings("unchecked") + final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; + + if (that.getSeekableStreamPartitions().getId().equals(seekableStreamPartitions.getId())) { + // Same topic, merge offsets. + final Map newMap = Maps.newHashMap(); + + for (Map.Entry entry : seekableStreamPartitions.getPartitionSequenceMap().entrySet()) { + newMap.put(entry.getKey(), entry.getValue()); + } + + for (Map.Entry entry : that.getSeekableStreamPartitions().getPartitionSequenceMap().entrySet()) { + newMap.put(entry.getKey(), entry.getValue()); + } + + return createConcretDataSourceMetaData(seekableStreamPartitions.getId(), newMap); + } else { + // Different topic, prefer "other". + return other; + } + } + @Override - public abstract DataSourceMetadata minus(DataSourceMetadata other); + public DataSourceMetadata minus(DataSourceMetadata other) + { + if (!(this.getClass().isInstance(other))) { + throw new IAE( + "Expected instance of %s, got %s", + this.getClass().getCanonicalName(), + other.getClass().getCanonicalName() + ); + } + + @SuppressWarnings("unchecked") + final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; + + if (that.getSeekableStreamPartitions().getId().equals(seekableStreamPartitions.getId())) { + // Same stream, remove partitions present in "that" from "this" + final Map newMap = Maps.newHashMap(); + + for (Map.Entry entry : seekableStreamPartitions.getPartitionSequenceMap().entrySet()) { + if (!that.getSeekableStreamPartitions().getPartitionSequenceMap().containsKey(entry.getKey())) { + newMap.put(entry.getKey(), entry.getValue()); + } + } + + return createConcretDataSourceMetaData(seekableStreamPartitions.getId(), newMap); + } else { + // Different stream, prefer "this". + return this; + } + } @Override - public abstract boolean equals(Object o); + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || !getClass().equals(o.getClass())) { + return false; + } + SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) o; + return Objects.equals(getSeekableStreamPartitions(), that.getSeekableStreamPartitions()); + } @Override public int hashCode() { - return Objects.hash(seekableStreamPartitions); + return Objects.hash(getSeekableStreamPartitions()); } @Override - public abstract String toString(); + public String toString() + { + return "SeekableStreamDataSourceMetadata{" + + "SeekableStreamPartitions=" + getSeekableStreamPartitions() + + '}'; + } + + protected abstract SeekableStreamDataSourceMetadata createConcretDataSourceMetaData( + String streamId, + Map newMap + ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 176305177daa..2b5d958495ae 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -34,7 +34,6 @@ import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; import org.apache.druid.segment.indexing.DataSchema; @@ -50,14 +49,14 @@ //TODO: need more refactoring for run() public abstract class SeekableStreamIndexTask extends AbstractTask implements ChatHandler { - private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTask.class); private static final Random RANDOM = new Random(); - private static final String TYPE = "index_seekable_stream"; protected final DataSchema dataSchema; protected final InputRowParser parser; protected final SeekableStreamTuningConfig tuningConfig; protected final SeekableStreamIOConfig ioConfig; protected final Optional chatHandlerProvider; + protected final String type; + protected CircularBuffer savedParseExceptions; @JsonCreator public SeekableStreamIndexTask( @@ -69,23 +68,23 @@ public SeekableStreamIndexTask( @JsonProperty("context") Map context, @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, - @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, + String type ) { super( - id == null ? makeTaskId(dataSchema.getDataSource(), RANDOM.nextInt()) : id, - StringUtils.format("%s_%s", TYPE, dataSchema.getDataSource()), + id == null ? makeTaskId(dataSchema.getDataSource(), RANDOM.nextInt(), type) : id, + StringUtils.format("%s_%s", type, dataSchema.getDataSource()), taskResource, dataSchema.getDataSource(), context ); - + this.type = type; this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); this.parser = Preconditions.checkNotNull((InputRowParser) dataSchema.getParser(), "parser"); this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); - final CircularBuffer savedParseExceptions; if (tuningConfig.getMaxSavedParseExceptions() > 0) { savedParseExceptions = new CircularBuffer<>(tuningConfig.getMaxSavedParseExceptions()); } else { @@ -93,13 +92,13 @@ public SeekableStreamIndexTask( } } - private static String makeTaskId(String dataSource, int randomBits) + private static String makeTaskId(String dataSource, int randomBits, String type) { final StringBuilder suffix = new StringBuilder(8); for (int i = 0; i < Integer.BYTES * 2; ++i) { suffix.append((char) ('a' + ((randomBits >>> (i * 4)) & 0x0F))); } - return Joiner.on("_").join(TYPE, dataSource, suffix); + return Joiner.on("_").join(type, dataSource, suffix); } @Override @@ -111,7 +110,7 @@ public int getPriority() @Override public String getType() { - return TYPE; + return type; } @Override @@ -139,7 +138,7 @@ public SeekableStreamIOConfig getIOConfig() } @Override - public abstract TaskStatus run(TaskToolbox toolbox); + public abstract TaskStatus run(TaskToolbox toolbox) throws Exception; @Override public abstract boolean canRestore(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 3d359643f480..c058cb99df3c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -351,7 +351,5 @@ public ListenableFuture getStatusAsync(final Str protected abstract JavaType constructMapType(Class mapType); - public abstract Class getTaskClassType(); - } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index e8f9dc73fa92..55f1ba873c09 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -21,34 +21,49 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import java.util.Map; import java.util.Objects; +// TODO: may consider deleting Kinesis and KafaPartitions classes and just use this instead public abstract class SeekableStreamPartitions { private final String id; - private final Map partitionOffsetMap; + private final Map partitionSequenceMap; @JsonCreator public SeekableStreamPartitions( @JsonProperty("id") final String id, - @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap + @JsonProperty("partitionSequenceMap") final Map partitionOffsetMap ) { this.id = id; - this.partitionOffsetMap = ImmutableMap.copyOf(partitionOffsetMap); + this.partitionSequenceMap = ImmutableMap.copyOf(partitionOffsetMap); + // Validate partitionSequenceNumberMap + for (Map.Entry entry : partitionOffsetMap.entrySet()) { + Preconditions.checkArgument( + entry.getValue() != null, + String.format( + "partition id[%s] sequence/offset number[%s] invalid", + entry.getKey(), + entry.getValue() + ) + ); + } } + @JsonProperty public String getId() { return id; } + @JsonProperty public Map getPartitionSequenceMap() { - return partitionOffsetMap; + return partitionSequenceMap; } @Override @@ -62,16 +77,23 @@ public boolean equals(Object o) } SeekableStreamPartitions that = (SeekableStreamPartitions) o; return Objects.equals(id, that.id) && - Objects.equals(partitionOffsetMap, that.partitionOffsetMap); + Objects.equals(partitionSequenceMap, that.partitionSequenceMap); } @Override public int hashCode() { - return Objects.hash(id, partitionOffsetMap); + return Objects.hash(id, partitionSequenceMap); } @Override - public abstract String toString(); + public String toString() + { + return "SeekableStreamPartitions{" + + "stream/topic='" + id + '\'' + + ", partitionSequenceMap=" + partitionSequenceMap + + '}'; + } + public abstract T2 getNoEndSequenceNumber(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java index 667a8f481940..adb727bacbb6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java @@ -36,6 +36,7 @@ public abstract class SeekableStreamTuningConfig implements TuningConfig, Append { private static final int DEFAULT_MAX_ROWS_PER_SEGMENT = 5_000_000; private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false; + private static final boolean DEFAULT_SKIP_SEQUENCE_NUMBER_AVAILABILITY_CHECK = false; private final int maxRowsInMemory; private final long maxBytesInMemory; @@ -53,6 +54,7 @@ public abstract class SeekableStreamTuningConfig implements TuningConfig, Append @Nullable private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; private final Period intermediateHandoffPeriod; + private final boolean skipSequenceNumberAvailabilityCheck; private final boolean logParseExceptions; private final int maxParseExceptions; @@ -73,6 +75,7 @@ public SeekableStreamTuningConfig( @Deprecated @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, @JsonProperty("handoffConditionTimeout") @Nullable Long handoffConditionTimeout, @JsonProperty("resetOffsetAutomatically") @Nullable Boolean resetOffsetAutomatically, + @JsonProperty("skipSequenceNumberAvailabilityCheck") Boolean skipSequenceNumberAvailabilityCheck, @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod, @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, @@ -108,6 +111,9 @@ public SeekableStreamTuningConfig( this.intermediateHandoffPeriod = intermediateHandoffPeriod == null ? new Period().withDays(Integer.MAX_VALUE) : intermediateHandoffPeriod; + this.skipSequenceNumberAvailabilityCheck = skipSequenceNumberAvailabilityCheck == null + ? DEFAULT_SKIP_SEQUENCE_NUMBER_AVAILABILITY_CHECK + : skipSequenceNumberAvailabilityCheck; if (this.reportParseExceptions) { this.maxParseExceptions = 0; @@ -246,6 +252,12 @@ public int getMaxSavedParseExceptions() return maxSavedParseExceptions; } + @JsonProperty + public boolean isSkipSequenceNumberAvailabilityCheck() + { + return skipSequenceNumberAvailabilityCheck; + } + public abstract SeekableStreamTuningConfig withBasePersistDirectory(File dir); @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java index 85dee67f72f2..d102e68537d2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java @@ -23,6 +23,7 @@ public class Record { + // TODO: end of shard marker public static final String END_OF_SHARD_MARKER = "EOS"; private final String streamName; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java index 42e3b96e626f..b91b1e76bc18 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java @@ -24,8 +24,6 @@ import java.util.Set; import java.util.concurrent.TimeoutException; -// TODO: need to integrate this with Kafka - /** * The RecordSupplier interface is a wrapper for the incoming seekable data stream * (i.e. Kafka consumer) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java index 57163560430a..fbb99cde814e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java @@ -20,29 +20,31 @@ package org.apache.druid.indexing.seekablestream.common; -public class SequenceNumberPlus +public class SequenceNumberPlus { - private final String sequenceNumber; + private final T sequenceNumber; private final boolean exclusive; + private final boolean useExclusive; - private SequenceNumberPlus(String sequenceNumber, boolean exclusive) + private SequenceNumberPlus(T sequenceNumber, boolean useExclusive, boolean exclusive) { this.sequenceNumber = sequenceNumber; this.exclusive = exclusive; + this.useExclusive = useExclusive; } - public static SequenceNumberPlus of(String sequenceNumber, boolean exclusive) + public static SequenceNumberPlus of(T sequenceNumber, boolean useExclusive, boolean exclusive) { - return new SequenceNumberPlus(sequenceNumber, exclusive); + return new SequenceNumberPlus<>(sequenceNumber, useExclusive, exclusive); } - public String get() + public T get() { return sequenceNumber; } public boolean isExclusive() { - return exclusive; + return useExclusive && exclusive; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 8b2b543db11d..154209fe01f5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -63,7 +63,9 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; +import org.apache.druid.indexing.seekablestream.common.Record; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.SequenceNumberPlus; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; @@ -102,9 +104,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -//TODO: rename offset -> sequence -//TODO: prune 'kafka' and 'kinesis' -//TODO: resolve warnings + inspect code +//TODO: documentation +//TODO: compare with kinesis supervisor for subsequently discovered partitions public abstract class SeekableStreamSupervisor, T2 extends Comparable> implements Supervisor { @@ -114,9 +115,11 @@ public abstract class SeekableStreamSupervisor, T2 ext private static final long MAX_RUN_FREQUENCY_MILLIS = 1000; private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120; private static final CopyOnWriteArrayList EMPTY_LIST = Lists.newCopyOnWriteArrayList(); - protected final ConcurrentHashMap taskGroups = new ConcurrentHashMap<>(); - protected final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); - protected final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); + protected final List partitionIds = new CopyOnWriteArrayList<>(); + protected final Set subsequentlyDiscoveredPartitions = new HashSet<>(); + private final ConcurrentHashMap taskGroups = new ConcurrentHashMap<>(); + private final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); + private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); private final TaskStorage taskStorage; private final TaskMaster taskMaster; private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; @@ -139,33 +142,28 @@ public abstract class SeekableStreamSupervisor, T2 ext private final Object stopLock = new Object(); private final Object stateChangeLock = new Object(); private final Object recordSupplierLock = new Object(); - private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); - private final boolean checkpointSupported; private final T2 NOT_SET; - private final T2 MAX_SEQUENCE_NUMBER; - protected volatile Map latestOffsetsFromStream; - protected volatile DateTime offsetsLastUpdated; + private final boolean useExclusiveStartingSequence; + protected volatile Map latestSequenceFromStream; + protected volatile DateTime sequenceLastUpdated; private boolean listenerRegistered = false; private long lastRunTime; private volatile DateTime firstRunTime; - private volatile DateTime earlyPublishTime = null; private volatile RecordSupplier recordSupplier; private volatile boolean started = false; private volatile boolean stopped = false; - //---------------------------GOOD----------------------------------------- public SeekableStreamSupervisor( final String supervisorId, final TaskStorage taskStorage, final TaskMaster taskMaster, final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, - final SeekableStreamIndexTaskClientFactory taskClientFactory, + final SeekableStreamIndexTaskClientFactory> taskClientFactory, final ObjectMapper mapper, final SeekableStreamSupervisorSpec spec, final RowIngestionMetersFactory rowIngestionMetersFactory, - final T2 NOT_SET, // TODO: fix these flags and extra variables - final T2 MAX_SEQUENCE_NUMBER, - final boolean checkpointSupported + final T2 NOT_SET, + final boolean useExclusiveStartingSequence ) { this.taskStorage = taskStorage; @@ -175,8 +173,7 @@ public SeekableStreamSupervisor( this.spec = spec; this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.NOT_SET = NOT_SET; - this.MAX_SEQUENCE_NUMBER = MAX_SEQUENCE_NUMBER; //TODO: placeholder till fix - this.checkpointSupported = checkpointSupported; + this.useExclusiveStartingSequence = useExclusiveStartingSequence; this.dataSource = spec.getDataSchema().getDataSource(); this.ioConfig = spec.getIoConfig(); @@ -259,9 +256,7 @@ protected static String getRandomId() return suffix.toString(); } - //-------------------------------GOOD---------------------------------------- - //-------------------------------GOOD---------------------------------------- @Override public void start() { @@ -326,7 +321,15 @@ public void start() } } - //---------------------------GOOOD------------------------------ + protected abstract RecordSupplier setupRecordSupplier(); + + private Runnable buildRunTask() + { + return () -> notices.add(new RunNotice()); + } + + protected abstract void scheduleReporting(ScheduledExecutorService reportingExec); + @Override public void stop(boolean stopGracefully) { @@ -386,6 +389,108 @@ public void stop(boolean stopGracefully) } } + @Override + public void reset(DataSourceMetadata dataSourceMetadata) + { + log.info("Posting ResetNotice"); + notices.add(new ResetNotice(dataSourceMetadata)); + } + + @Override + public SupervisorReport getStatus() + { + return generateReport(true); + } + + private SupervisorReport> generateReport( + boolean includeOffsets + ) + { + int numPartitions = partitionGroups.values().stream().mapToInt(Map::size).sum(); + + final SeekableStreamSupervisorReportPayload payload = createReportPayload(numPartitions, includeOffsets); + + SupervisorReport> report = new SupervisorReport<>( + dataSource, + DateTimes.nowUtc(), + payload + ); + + List> taskReports = Lists.newArrayList(); + + try { + for (TaskGroup taskGroup : taskGroups.values()) { + for (Map.Entry entry : taskGroup.tasks.entrySet()) { + String taskId = entry.getKey(); + @Nullable + DateTime startTime = entry.getValue().startTime; + Map currentOffsets = entry.getValue().currentSequences; + Long remainingSeconds = null; + if (startTime != null) { + remainingSeconds = Math.max( + 0, ioConfig.getTaskDuration().getMillis() - (System.currentTimeMillis() - startTime.getMillis()) + ) / 1000; + } + + taskReports.add( + new TaskReportData<>( + taskId, + includeOffsets ? taskGroup.startingSequences : null, + includeOffsets ? currentOffsets : null, + startTime, + remainingSeconds, + TaskReportData.TaskType.ACTIVE, + includeOffsets ? getLagPerPartition(currentOffsets) : null + ) + ); + } + } + + for (List taskGroups : pendingCompletionTaskGroups.values()) { + for (TaskGroup taskGroup : taskGroups) { + for (Map.Entry entry : taskGroup.tasks.entrySet()) { + String taskId = entry.getKey(); + @Nullable + DateTime startTime = entry.getValue().startTime; + Map currentOffsets = entry.getValue().currentSequences; + Long remainingSeconds = null; + if (taskGroup.completionTimeout != null) { + remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - System.currentTimeMillis()) + / 1000; + } + + taskReports.add( + new TaskReportData<>( + taskId, + includeOffsets ? taskGroup.startingSequences : null, + includeOffsets ? currentOffsets : null, + startTime, + remainingSeconds, + TaskReportData.TaskType.PUBLISHING, + null + ) + ); + } + } + } + + taskReports.forEach(payload::addTask); + } + catch (Exception e) { + log.warn(e, "Failed to generate status report"); + } + + return report; + } + + protected abstract SeekableStreamSupervisorReportPayload createReportPayload( + int numPartitions, + boolean includeOffsets + ); + + protected abstract Map getLagPerPartition(Map currentOffsets); + + @Override public Map> getStats() { @@ -402,20 +507,72 @@ public Map> getStats() } } - @Override - public SupervisorReport getStatus() + /** + * Collect row ingestion stats from all tasks managed by this supervisor. + * + * @return A map of groupId->taskId->task row stats + * + * @throws InterruptedException + * @throws ExecutionException + * @throws TimeoutException + */ + private Map> getCurrentTotalStats() + throws InterruptedException, ExecutionException, TimeoutException { - return generateReport(true); - } + Map> allStats = Maps.newHashMap(); + final List> futures = new ArrayList<>(); + final List> groupAndTaskIds = new ArrayList<>(); - @Override - public void reset(DataSourceMetadata dataSourceMetadata) - { - log.info("Posting ResetNotice"); - notices.add(new ResetNotice(dataSourceMetadata)); + for (int groupId : taskGroups.keySet()) { + TaskGroup group = taskGroups.get(groupId); + for (String taskId : group.taskIds()) { + futures.add( + Futures.transform( + taskClient.getMovingAveragesAsync(taskId), + (Function, StatsFromTaskResult>) (currentStats) -> new StatsFromTaskResult( + groupId, + taskId, + currentStats + ) + ) + ); + groupAndTaskIds.add(new Pair<>(groupId, taskId)); + } + } + + for (int groupId : pendingCompletionTaskGroups.keySet()) { + TaskGroup group = taskGroups.get(groupId); + for (String taskId : group.taskIds()) { + futures.add( + Futures.transform( + taskClient.getMovingAveragesAsync(taskId), + (Function, StatsFromTaskResult>) (currentStats) -> new StatsFromTaskResult( + groupId, + taskId, + currentStats + ) + ) + ); + groupAndTaskIds.add(new Pair<>(groupId, taskId)); + } + } + + List results = Futures.successfulAsList(futures) + .get(futureTimeoutInSeconds, TimeUnit.SECONDS); + for (int i = 0; i < results.size(); i++) { + StatsFromTaskResult result = results.get(i); + if (result != null) { + Map groupMap = allStats.computeIfAbsent(result.getGroupId(), k -> Maps.newHashMap()); + groupMap.put(result.getTaskId(), result.getStats()); + } else { + Pair groupAndTaskId = groupAndTaskIds.get(i); + log.error("Failed to get stats for group[%d]-task[%s]", groupAndTaskId.lhs, groupAndTaskId.rhs); + } + } + + return allStats; } - //TODO: checkpoints @Override public abstract void checkpoint( @Nullable Integer taskGroupId, @@ -452,7 +609,6 @@ protected void runInternal() } } - //-------------------------------GOOD---------------------------------------- private void possiblyRegisterListener() { if (listenerRegistered) { @@ -488,7 +644,6 @@ public void statusChanged(String taskId, TaskStatus status) } } - //-------------------------------GOOD---------------------------------------- @VisibleForTesting protected void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException { @@ -506,7 +661,6 @@ protected void gracefulShutdownInternal() throws ExecutionException, Interrupted checkTaskDuration(); } - //-------------------------------GOOD---------------------------------------- @VisibleForTesting protected void resetInternal(DataSourceMetadata dataSourceMetadata) { @@ -520,18 +674,26 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) } else { if (!checkSourceMetaDataMatch(dataSourceMetadata)) { - throw new IAE("Expected KafkaDataSourceMetadata but found instance of [%s]", dataSourceMetadata.getClass()); + throw new IAE( + "Datasource metadata instance does not match required, found instance of [%s]", + dataSourceMetadata.getClass() + ); } // Reset only the partitions in dataSourceMetadata if it has not been reset yet + @SuppressWarnings("unchecked") final SeekableStreamDataSourceMetadata resetMetadata = (SeekableStreamDataSourceMetadata) dataSourceMetadata; if (resetMetadata.getSeekableStreamPartitions().getId().equals(ioConfig.getId())) { // metadata can be null final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); if (metadata != null && !checkSourceMetaDataMatch(metadata)) { - throw new IAE("Expected KafkaDataSourceMetadata but found instance of [%s]", metadata.getClass()); + throw new IAE( + "Datasource metadata instance does not match required, found instance of [%s]", + metadata.getClass() + ); } + @SuppressWarnings("unchecked") final SeekableStreamDataSourceMetadata currentMetadata = (SeekableStreamDataSourceMetadata) metadata; // defend against consecutive reset requests from replicas @@ -620,7 +782,10 @@ private void killTasksInGroup(TaskGroup taskGroup) private void killTaskGroupForPartitions(Set partitions) { for (T1 partition : partitions) { - killTasksInGroup(taskGroups.get(getTaskGroupIdForPartition(partition))); + int taskGroupId = getTaskGroupIdForPartition(partition); + killTasksInGroup(taskGroups.get(taskGroupId)); + partitionGroups.remove(taskGroupId); + taskGroups.remove(taskGroupId); } } @@ -650,6 +815,7 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti } taskCount++; + @SuppressWarnings("unchecked") final SeekableStreamIndexTask seekableStreamIndexTask = (SeekableStreamIndexTask) task; final String taskId = task.getId(); @@ -695,8 +861,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) seekableStreamIndexTask.getIOConfig() .getStartPartitions() .getPartitionSequenceMap() - ) - ); + )); // update partitionGroups with the publishing task's offsets (if they are greater than what is // existing) so that the next tasks will start reading from where this task left off @@ -770,11 +935,11 @@ public Boolean apply(SeekableStreamIndexTask.Status status) } ); taskGroupsToVerify.put(taskGroupId, taskGroup); - final TaskData prevTaskGroup = taskGroup.tasks.putIfAbsent(taskId, new TaskData()); - if (prevTaskGroup != null) { + final TaskData prevTaskData = taskGroup.tasks.putIfAbsent(taskId, new TaskData()); + if (prevTaskData != null) { throw new ISE( "WTH? a taskGroup[%s] already exists for new task[%s]", - prevTaskGroup, + prevTaskData, taskId ); } @@ -825,6 +990,8 @@ private void verifyAndMergeCheckpoints(final Collection taskGroupsToV } } + //TODO: prob wanna refactor this + /** * This method does two things - * 1. Makes sure the checkpoints information in the taskGroup is consistent with that of the tasks, if not kill @@ -875,8 +1042,18 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) throw new RuntimeException(e); } - final SeekableStreamDataSourceMetadata latestDataSourceMetadata = (SeekableStreamDataSourceMetadata) indexerMetadataStorageCoordinator - .getDataSourceMetadata(dataSource); + final DataSourceMetadata rawDataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); + + if (rawDataSourceMetadata != null && !checkSourceMetaDataMatch(rawDataSourceMetadata)) { + throw new IAE( + "Datasource metadata instance does not match required, found instance of [%s]", + rawDataSourceMetadata.getClass() + ); + } + + @SuppressWarnings("unchecked") + final SeekableStreamDataSourceMetadata latestDataSourceMetadata = (SeekableStreamDataSourceMetadata) rawDataSourceMetadata; + final boolean hasValidOffsetsFromDb = latestDataSourceMetadata != null && latestDataSourceMetadata.getSeekableStreamPartitions() != null && ioConfig.getId().equals( @@ -1037,7 +1214,10 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) return false; } - String taskSequenceName = ((SeekableStreamIndexTask) taskOptional.get()).getIOConfig().getBaseSequenceName(); + @SuppressWarnings("unchecked") + SeekableStreamIndexTask task = (SeekableStreamIndexTask) taskOptional.get(); + + String taskSequenceName = task.getIOConfig().getBaseSequenceName(); if (taskGroups.get(taskGroupId) != null) { return Preconditions .checkNotNull(taskGroups.get(taskGroupId), "null taskGroup for taskId[%s]", taskGroupId) @@ -1045,11 +1225,11 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) .equals(taskSequenceName); } else { return generateSequenceName( - ((SeekableStreamIndexTask) taskOptional.get()).getIOConfig() - .getStartPartitions() - .getPartitionSequenceMap(), - ((SeekableStreamIndexTask) taskOptional.get()).getIOConfig().getMinimumMessageTime(), - ((SeekableStreamIndexTask) taskOptional.get()).getIOConfig().getMaximumMessageTime() + task.getIOConfig() + .getStartPartitions() + .getPartitionSequenceMap(), + task.getIOConfig().getMinimumMessageTime(), + task.getIOConfig().getMaximumMessageTime() ).equals(taskSequenceName); } } @@ -1091,7 +1271,7 @@ protected String generateSequenceName( private void updatePartitionDataFromStream() { - Set partitionIds = null; + Set partitionIds; try { synchronized (recordSupplierLock) { partitionIds = recordSupplier.getPartitionIds(ioConfig.getId()); @@ -1102,12 +1282,32 @@ private void updatePartitionDataFromStream() return; } - log.debug("Found [%d] partitions for topic [%s]", partitionIds.size(), ioConfig.getId()); + if (partitionIds == null) { + log.warn("No partitions found for stream[%s]", ioConfig.getId()); + return; + } + + log.debug("Found [%d] partitions for stream [%s]", partitionIds.size(), ioConfig.getId()); + + Set closedPartitions = getOffsetsFromMetadataStorage() + .entrySet() + .stream() + .filter(x -> Record.END_OF_SHARD_MARKER.equals(x.getValue())) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + boolean initialPartitionDiscovery = this.partitionIds.isEmpty(); for (T1 partitionId : partitionIds) { + if (closedPartitions.contains(partitionId)) { + continue; + } + + if (!initialPartitionDiscovery && !this.partitionIds.contains(partitionId)) { + subsequentlyDiscoveredPartitions.add(partitionId); + //TODO: early publish time + } + int taskGroupId = getTaskGroupIdForPartition(partitionId); - // TODO: check for closed partitions (not sure if can be done in Kafka) - // TODO: earlyPublishTime ConcurrentHashMap partitionMap = partitionGroups.computeIfAbsent( taskGroupId, @@ -1142,7 +1342,6 @@ private void updateTaskStatus() throws ExecutionException, InterruptedException, Futures.transform( taskClient.getStartTimeAsync(taskId), new Function() { - @Nullable @Override public Boolean apply(@Nullable DateTime startTime) { @@ -1193,13 +1392,6 @@ public Boolean apply(@Nullable DateTime startTime) } } - //TODO: prob wanna refactor this - - private Runnable buildRunTask() - { - return () -> notices.add(new RunNotice()); - } - private void checkTaskDuration() throws ExecutionException, InterruptedException, TimeoutException { final List>> futures = Lists.newArrayList(); @@ -1217,14 +1409,14 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } } - // TODO: early publish time -// boolean doEarlyPublish = false; -// if (earlyPublishTime != null && (earlyPublishTime.isBeforeNow() || earlyPublishTime.isEqualNow())) { -// log.info("Early publish requested - signalling tasks to publish"); -// -// earlyPublishTime = null; -// doEarlyPublish = true; -// } + // earlyPublishTime is only supported in Kinesis + // for now it is not implemented + // boolean doEarlyPublish = false; + // if (earlyPublishTime != null && (earlyPublishTime.isBeforeNow() || earlyPublishTime.isEqualNow())) { + // log.info("Early publish requested - signalling tasks to publish"); + // earlyPublishTime = null; + // doEarlyPublish = true; + // } // if this task has run longer than the configured duration, signal all tasks in the group to persist if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { @@ -1379,7 +1571,10 @@ public Map apply(List> input) for (int i = 0; i < results.size(); i++) { if (results.get(i) == null || !results.get(i)) { String taskId = setEndOffsetTaskIds.get(i); - log.warn("Task [%s] failed to respond to [set end offsets] in a timely manner, killing task", taskId); + log.warn( + "Task [%s] failed to respond to [set end offsets] in a timely manner, killing task", + taskId + ); killTask(taskId); taskGroup.tasks.remove(taskId); } @@ -1582,14 +1777,32 @@ private void createNewTasks() throws JsonProcessingException ) : Optional.absent()); try { + + Map> startingOffsets = generateStartingSequencesForPartitionGroup(groupId); + + ImmutableMap simpleStartingOffsets = startingOffsets + .entrySet() + .stream() + .filter(x -> x.getValue().get() != null) + .collect(Collectors.collectingAndThen( + Collectors.toMap(Map.Entry::getKey, x -> x.getValue().get()), + ImmutableMap::copyOf + )); + + Set exclusiveStartSequenceNumberPartitions = startingOffsets + .entrySet().stream() + .filter(x -> x.getValue().get() != null && x.getValue().isExclusive()) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + taskGroups.put( groupId, new TaskGroup( groupId, - generateStartingSequencesForPartitionGroup(groupId), + simpleStartingOffsets, minimumMessageTime, maximumMessageTime, - null //TODO: exclusive sequence + exclusiveStartSequenceNumberPartitions ) ); } @@ -1608,12 +1821,11 @@ private void createNewTasks() throws JsonProcessingException TaskGroup taskGroup = entry.getValue(); Integer groupId = entry.getKey(); - //TODO: kinesis -// if (taskGroup.startingSequences == null || taskGroup.startingSequences -// .values().stream().allMatch(x -> x == null || Record.END_OF_SHARD_MARKER.equals(x))) { -// log.debug("Nothing to read in any partition for taskGroup [%d], skipping task creation", groupId); -// continue; -// } + if (taskGroup.startingSequences == null || taskGroup.startingSequences + .values().stream().allMatch(x -> x == null || Record.END_OF_SHARD_MARKER.equals(x))) { + log.debug("Nothing to read in any partition for taskGroup [%d], skipping task creation", groupId); + continue; + } if (ioConfig.getReplicas() > taskGroup.tasks.size()) { log.info( @@ -1660,89 +1872,27 @@ protected int getNoticesQueueSize() return notices.size(); } - /** - * Collect row ingestion stats from all tasks managed by this supervisor. - * - * @return A map of groupId->taskId->task row stats - * - * @throws InterruptedException - * @throws ExecutionException - * @throws TimeoutException - */ - private Map> getCurrentTotalStats() - throws InterruptedException, ExecutionException, TimeoutException - { - Map> allStats = Maps.newHashMap(); - final List> futures = new ArrayList<>(); - final List> groupAndTaskIds = new ArrayList<>(); - - for (int groupId : taskGroups.keySet()) { - TaskGroup group = taskGroups.get(groupId); - for (String taskId : group.taskIds()) { - futures.add( - Futures.transform( - taskClient.getMovingAveragesAsync(taskId), - (Function, StatsFromTaskResult>) (currentStats) -> { - return new StatsFromTaskResult( - groupId, - taskId, - currentStats - ); - } - ) - ); - groupAndTaskIds.add(new Pair<>(groupId, taskId)); - } - } - - for (int groupId : pendingCompletionTaskGroups.keySet()) { - TaskGroup group = taskGroups.get(groupId); - for (String taskId : group.taskIds()) { - futures.add( - Futures.transform( - taskClient.getMovingAveragesAsync(taskId), - (Function, StatsFromTaskResult>) (currentStats) -> { - return new StatsFromTaskResult( - groupId, - taskId, - currentStats - ); - } - ) - ); - groupAndTaskIds.add(new Pair<>(groupId, taskId)); - } - } - - List results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - for (int i = 0; i < results.size(); i++) { - StatsFromTaskResult result = results.get(i); - if (result != null) { - Map groupMap = allStats.computeIfAbsent(result.getGroupId(), k -> Maps.newHashMap()); - groupMap.put(result.getTaskId(), result.getStats()); - } else { - Pair groupAndTaskId = groupAndTaskIds.get(i); - log.error("Failed to get stats for group[%d]-task[%s]", groupAndTaskId.lhs, groupAndTaskId.rhs); - } - } - - return allStats; - } - - private ImmutableMap generateStartingSequencesForPartitionGroup(int groupId) throws TimeoutException + private ImmutableMap> generateStartingSequencesForPartitionGroup(int groupId) + throws TimeoutException { - ImmutableMap.Builder builder = ImmutableMap.builder(); + ImmutableMap.Builder> builder = ImmutableMap.builder(); for (Map.Entry entry : partitionGroups.get(groupId).entrySet()) { T1 partition = entry.getKey(); T2 offset = entry.getValue(); - if (offset != null && !offset.equals(NOT_SET)) { + if (NOT_SET.equals(offset)) { // if we are given a startingOffset (set by a previous task group which is pending completion) then use it - builder.put(partition, offset); + if (!Record.END_OF_SHARD_MARKER.equals(offset)) { + builder.put(partition, SequenceNumberPlus.of(offset, useExclusiveStartingSequence, true)); + } } else { // if we don't have a startingOffset (first run or we had some previous failures and reset the offsets) then - // get the offset from metadata storage (if available) or Kafka (otherwise) - builder.put(partition, getOffsetFromStorageForPartition(partition)); + // get the offset from metadata storage (if available) or Kafka/Kinesis (otherwise) + SequenceNumberPlus offsetFromStorage = getOffsetFromStorageForPartition(partition); + + if (!Record.END_OF_SHARD_MARKER.equals(offset)) { + builder.put(partition, offsetFromStorage); + } } } return builder.build(); @@ -1750,44 +1900,45 @@ private ImmutableMap generateStartingSequencesForPartitionGroup(int grou /** * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't - * find any data, it will retrieve the latest or earliest Kafka offset depending on the useEarliestOffset config. + * find any data, it will retrieve the latest or earliest Kafka/Kinesis offset depending on the useEarliestOffset config. */ - private T2 getOffsetFromStorageForPartition(T1 partition) throws TimeoutException + private SequenceNumberPlus getOffsetFromStorageForPartition(T1 partition) throws TimeoutException { final Map metadataOffsets = getOffsetsFromMetadataStorage(); T2 offset = metadataOffsets.get(partition); if (offset != null) { log.debug("Getting offset [%s] from metadata storage for partition [%s]", offset, partition); + if (!taskTuningConfig.isSkipSequenceNumberAvailabilityCheck()) { + try { + T2 latestSequence = getOffsetFromStreamForPartition(partition, false); - // TODO: tuningConfig.isSkipSequenceNumberAvailabilityCheck() - - try { - T2 latestSequence = getOffsetFromStreamForPartition(partition, false); - if (latestSequence == null || offset.compareTo(latestSequence) > 0) { - if (taskTuningConfig.isResetOffsetAutomatically()) { - // TODO: reset internal - throw new ISE( - "Previous sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) - automatically resetting offset", - offset, - partition, - latestSequence - ); + if (latestSequence == null || offset.compareTo(latestSequence) > 0) { + if (taskTuningConfig.isResetOffsetAutomatically()) { + resetInternal( + createDataSourceMetaData(ioConfig.getId(), ImmutableMap.of(partition, offset)) + ); + throw new ISE( + "Previous sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) - automatically resetting offset", + offset, + partition, + latestSequence + ); - } else { - throw new ISE( - "Previous sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]). You can clear the previous sequenceNumber and start reading from a valid message by using the supervisor's reset API.", - offset, - partition, - latestSequence - ); + } else { + throw new ISE( + "Previous sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]). You can clear the previous sequenceNumber and start reading from a valid message by using the supervisor's reset API.", + offset, + partition, + latestSequence + ); + } } } + catch (TimeoutException e) { + throw new ISE(e, "Timeout while fetching earliest sequence number for partition [%s]", partition); + } } - catch (TimeoutException e) { - throw new ISE(e, "Timeout while fetching earliest sequence number for partition [%s]", partition); - } - - return offset; + return SequenceNumberPlus.of(offset, useExclusiveStartingSequence, true); } else { boolean useEarliestSequenceNumber = ioConfig.isUseEarliestSequenceNumber(); if (subsequentlyDiscoveredPartitions.contains(partition)) { @@ -1800,7 +1951,7 @@ private T2 getOffsetFromStorageForPartition(T1 partition) throws TimeoutExceptio offset = getOffsetFromStreamForPartition(partition, useEarliestSequenceNumber); log.info("Getting sequence number [%s] for partition [%s]", offset, partition); - return offset; + return SequenceNumberPlus.of(offset, useExclusiveStartingSequence, false); } } @@ -1809,11 +1960,12 @@ private Map getOffsetsFromMetadataStorage() final DataSourceMetadata dataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); if (dataSourceMetadata instanceof SeekableStreamDataSourceMetadata && checkSourceMetaDataMatch(dataSourceMetadata)) { + @SuppressWarnings("unchecked") SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata).getSeekableStreamPartitions(); if (partitions != null) { if (!ioConfig.getId().equals(partitions.getId())) { log.warn( - "Topic in metadata storage [%s] doesn't match spec topic [%s], ignoring stored offsets", + "Topic/stream in metadata storage [%s] doesn't match spec topic/stream [%s], ignoring stored offsets", partitions.getId(), ioConfig.getId() ); @@ -1845,11 +1997,8 @@ private void createTasksForGroup(int groupId, int replicas) throws JsonProcessin { TaskGroup group = taskGroups.get(groupId); Map startPartitions = group.startingSequences; - Map endPartitions = new HashMap<>(); + Map endPartitions = createNewTaskEndPartitions(startPartitions.keySet()); Set exclusiveStartSequenceNumberPartitions = taskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; - for (T1 partition : startPartitions.keySet()) { - endPartitions.put(partition, MAX_SEQUENCE_NUMBER); - } DateTime minimumMessageTime = taskGroups.get(groupId).minimumMessageTime.orNull(); DateTime maximumMessageTime = taskGroups.get(groupId).maximumMessageTime.orNull(); @@ -1866,7 +2015,7 @@ private void createTasksForGroup(int groupId, int replicas) throws JsonProcessin ); - List> taskList = createIndexTask( + List> taskList = createIndexTasks( replicas, group.baseSequenceName, sortingMapper, @@ -1892,6 +2041,21 @@ private void createTasksForGroup(int groupId, int replicas) throws JsonProcessin } } + @VisibleForTesting + protected Runnable updateCurrentAndLatestOffsets() + { + return () -> { + try { + updateCurrentOffsets(); + updateLatestOffsetsFromStream(); + sequenceLastUpdated = DateTimes.nowUtc(); + } + catch (Exception e) { + log.warn(e, "Exception while getting current/latest offsets"); + } + }; + } + private void updateCurrentOffsets() throws InterruptedException, ExecutionException, TimeoutException { final List> futures = Stream.concat( @@ -1937,30 +2101,15 @@ private void updateLatestOffsetsFromStream() recordSupplier.assign(partitions); recordSupplier.seekToLatest(partitions); - latestOffsetsFromStream = partitions.stream() - .collect(Collectors.toMap( - StreamPartition::getPartitionId, - x -> recordSupplier.position(x) - )); + latestSequenceFromStream = partitions.stream() + .collect(Collectors.toMap( + StreamPartition::getPartitionId, + x -> recordSupplier.position(x) + )); } } - @VisibleForTesting - protected Runnable updateCurrentAndLatestOffsets() - { - return () -> { - try { - updateCurrentOffsets(); - updateLatestOffsetsFromStream(); - offsetsLastUpdated = DateTimes.nowUtc(); - } - catch (Exception e) { - log.warn(e, "Exception while getting current/latest offsets"); - } - }; - } - protected Map getHighestCurrentOffsets() { return taskGroups @@ -1975,87 +2124,6 @@ protected Map getHighestCurrentOffsets() )); } - private SupervisorReport> generateReport( - boolean includeOffsets - ) - { - int numPartitions = partitionGroups.values().stream().mapToInt(Map::size).sum(); - - final SeekableStreamSupervisorReportPayload payload = createReportPayload(numPartitions, includeOffsets); - - SupervisorReport> report = new SupervisorReport<>( - dataSource, - DateTimes.nowUtc(), - payload - ); - - List> taskReports = Lists.newArrayList(); - - try { - for (TaskGroup taskGroup : taskGroups.values()) { - for (Map.Entry entry : taskGroup.tasks.entrySet()) { - String taskId = entry.getKey(); - @Nullable - DateTime startTime = entry.getValue().startTime; - Map currentOffsets = entry.getValue().currentSequences; - Long remainingSeconds = null; - if (startTime != null) { - remainingSeconds = Math.max( - 0, ioConfig.getTaskDuration().getMillis() - (System.currentTimeMillis() - startTime.getMillis()) - ) / 1000; - } - - taskReports.add( - new TaskReportData( - taskId, - includeOffsets ? taskGroup.startingSequences : null, - includeOffsets ? currentOffsets : null, - startTime, - remainingSeconds, - TaskReportData.TaskType.ACTIVE, - includeOffsets ? getLagPerPartition(currentOffsets) : null - ) - ); - } - } - - for (List taskGroups : pendingCompletionTaskGroups.values()) { - for (TaskGroup taskGroup : taskGroups) { - for (Map.Entry entry : taskGroup.tasks.entrySet()) { - String taskId = entry.getKey(); - @Nullable - DateTime startTime = entry.getValue().startTime; - Map currentOffsets = entry.getValue().currentSequences; - Long remainingSeconds = null; - if (taskGroup.completionTimeout != null) { - remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - System.currentTimeMillis()) - / 1000; - } - - taskReports.add( - new TaskReportData( - taskId, - includeOffsets ? taskGroup.startingSequences : null, - includeOffsets ? currentOffsets : null, - startTime, - remainingSeconds, - TaskReportData.TaskType.PUBLISHING, - null - ) - ); - } - } - } - - taskReports.forEach(payload::addTask); - } - catch (Exception e) { - log.warn(e, "Failed to generate status report"); - } - - return report; - } - protected abstract SeekableStreamIOConfig createIoConfig( int groupId, Map startPartitions, @@ -2067,7 +2135,7 @@ protected abstract SeekableStreamIOConfig createIoConfig( SeekableStreamSupervisorIOConfig ioConfig ); - protected abstract List> createIndexTask( + protected abstract List> createIndexTasks( int replicas, String baseSequenceName, ObjectMapper sortingMapper, @@ -2077,22 +2145,18 @@ protected abstract List> createIndexTask( RowIngestionMetersFactory rowIngestionMetersFactory ) throws JsonProcessingException; - protected abstract RecordSupplier setupRecordSupplier(); - - protected abstract void scheduleReporting(ScheduledExecutorService reportingExec); - protected abstract int getTaskGroupIdForPartition(T1 partition); protected abstract boolean checkSourceMetaDataMatch(DataSourceMetadata metadata); protected abstract boolean checkTaskInstance(Task task); - protected abstract SeekableStreamSupervisorReportPayload createReportPayload( - int numPartitions, - boolean includeOffsets + protected abstract SeekableStreamDataSourceMetadata createDataSourceMetaData( + String + stream, Map map ); - protected abstract Map getLagPerPartition(Map currentOffsets); + protected abstract Map createNewTaskEndPartitions(Set startPartitions); /** * Notice is used to queue tasks that are internal to the supervisor @@ -2176,6 +2240,7 @@ public Set taskIds() { return tasks.keySet(); } + } private class TaskData diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java index 8e32fa9e73a3..ecc876321de0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java @@ -33,8 +33,9 @@ public class TaskReportData private final DateTime startTime; private final Long remainingSeconds; private final TaskType type; - private final Map currentOffsets; + private Map currentOffsets; private final Map lag; + public TaskReportData( String id, @Nullable Map startingOffsets, @@ -99,6 +100,11 @@ public Map getLag() return lag; } + public void setCurrentSequenceNumbers(Map currentOffsets) + { + this.currentOffsets = currentOffsets; + } + @Override public String toString() { From a6f94991321533bbe39df51f81417e823292223d Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 24 Sep 2018 18:12:11 -0700 Subject: [PATCH 11/87] unite tests for kinesis-indexing-service --- .../druid/indexing/kafka/KafkaIOConfig.java | 17 +- .../KafkaSupervisorTuningConfig.java | 1 - .../indexing/kafka/KafkaIndexTaskTest.java | 2 +- .../kafka/supervisor/KafkaSupervisorTest.java | 2 +- .../druid/indexing/kafka/test/TestBroker.java | 9 +- .../kinesis-indexing-service/pom.xml | 4 + .../indexing/kinesis/KinesisTuningConfig.java | 2 +- .../KinesisSupervisorTuningConfig.java | 6 - .../KinesisDataSourceMetadataTest.java | 261 +-- .../indexing/kinesis/KinesisIOConfigTest.java | 521 ++--- .../kinesis/KinesisIndexTaskClientTest.java | 1945 ++++++++-------- .../kinesis/KinesisIndexTaskTest.java | 1962 +++++++++++------ .../kinesis/KinesisTuningConfigTest.java | 327 ++- .../KinesisSupervisorIOConfigTest.java | 306 ++- .../supervisor/KinesisSupervisorTest.java | 36 +- .../KinesisSupervisorTuningConfigTest.java | 240 +- .../SeekableStreamDataSourceMetadata.java | 18 - .../SeekableStreamIOConfig.java | 26 +- .../SeekableStreamIndexTask.java | 3 +- .../SeekableStreamSupervisorTuningConfig.java | 3 - 20 files changed, 3244 insertions(+), 2447 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java index 30673e2ac7b8..a4f1f195c56e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java @@ -63,20 +63,11 @@ public KafkaIOConfig( this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); this.skipOffsetGaps = skipOffsetGaps != null ? skipOffsetGaps : DEFAULT_SKIP_OFFSET_GAPS; - Preconditions.checkArgument( - startPartitions.getTopic().equals(endPartitions.getTopic()), - "start topic and end topic must match" - ); - - Preconditions.checkArgument( - startPartitions.getPartitionOffsetMap().keySet().equals(endPartitions.getPartitionOffsetMap().keySet()), - "start partition set and end partition set must match" - ); - - for (int partition : endPartitions.getPartitionOffsetMap().keySet()) { + for (int partition : endPartitions.getPartitionSequenceMap().keySet()) { Preconditions.checkArgument( - endPartitions.getPartitionOffsetMap().get(partition) >= startPartitions.getPartitionOffsetMap() - .get(partition), + endPartitions.getPartitionSequenceMap() + .get(partition) + .compareTo(startPartitions.getPartitionSequenceMap().get(partition)) >= 0, "end offset must be >= start offset for partition[%s]", partition ); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index 5f624e886e5b..651380382d5c 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -130,7 +130,6 @@ public Duration getShutdownTimeout() return shutdownTimeout; } - @Override @JsonProperty public Duration getOffsetFetchPeriod() { diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 0ba4c33b0c8f..c231980b4bb6 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -72,6 +72,7 @@ import org.apache.druid.indexing.common.task.IndexTaskTest; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor; +import org.apache.druid.indexing.kafka.test.TestBroker; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.MetadataTaskStorage; @@ -79,7 +80,6 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; -import org.apache.druid.indexing.seekablestream.test.TestBroker; import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; import org.apache.druid.indexing.test.TestDataSegmentKiller; import org.apache.druid.java.util.common.CompressionUtils; diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 6a625e6b4387..259201fd17e9 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -50,6 +50,7 @@ import org.apache.druid.indexing.kafka.KafkaIndexTaskClient; import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; import org.apache.druid.indexing.kafka.KafkaPartitions; +import org.apache.druid.indexing.kafka.test.TestBroker; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; @@ -62,7 +63,6 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; -import org.apache.druid.indexing.seekablestream.test.TestBroker; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java index 16ae12a43fa6..05c164201d18 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.seekablestream.test; +package org.apache.druid.indexing.kafka.test; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; @@ -81,7 +81,12 @@ public void start() final KafkaConfig config = new KafkaConfig(props); - server = new KafkaServer(config, SystemTime.SYSTEM, Some.apply(StringUtils.format("TestingBroker[%d]-", id)), List$.MODULE$.empty()); + server = new KafkaServer( + config, + SystemTime.SYSTEM, + Some.apply(StringUtils.format("TestingBroker[%d]-", id)), + List$.MODULE$.empty() + ); server.startup(); } diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml index eb2eb5af9f10..25707bd0f16d 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml @@ -38,5 +38,9 @@ druid-imply-kinesis-lib ${project.parent.version} + + org.easymock + easymock + diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java index 255122bb62cf..2b406ba4acf9 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java @@ -106,7 +106,7 @@ public KinesisTuningConfig( this.fetchThreads = fetchThreads; // we handle this being null later Preconditions.checkArgument( - !resetOffsetAutomatically || !skipSequenceNumberAvailabilityCheck, + !(super.isResetOffsetAutomatically() && super.isSkipSequenceNumberAvailabilityCheck()), "resetOffsetAutomatically cannot be used if skipSequenceNumberAvailabilityCheck=true" ); } diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 0df0d8d9e3d3..e56237db2e36 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -133,12 +133,6 @@ public Duration getShutdownTimeout() return shutdownTimeout; } - @Override - public Duration getOffsetFetchPeriod() - { - throw new UnsupportedOperationException("kinesis supervisor does not support getOffsetFetchPeriod"); - } - @Override public String toString() { diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java index 52d40185a118..506889b59428 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java @@ -1,130 +1,131 @@ -///* -// * Licensed to Metamarkets Group Inc. (Metamarkets) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. Metamarkets licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, -// * software distributed under the License is distributed on an -// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// * KIND, either express or implied. See the License for the -// * specific language governing permissions and limitations -// * under the License. -// */ -// -//package org.apache.druid.indexing.kinesis; -// -//import com.google.common.collect.ImmutableMap; -//import org.junit.Assert; -//import org.junit.Test; -// -//import java.util.Map; -// -//public class KinesisDataSourceMetadataTest -//{ -// private static final KinesisDataSourceMetadata KM0 = KM("foo", ImmutableMap.of()); -// private static final KinesisDataSourceMetadata KM1 = KM("foo", ImmutableMap.of(0, 2L, 1, 3L)); -// private static final KinesisDataSourceMetadata KM2 = KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)); -// private static final KinesisDataSourceMetadata KM3 = KM("foo", ImmutableMap.of(0, 2L, 2, 5L)); -// -// @Test -// public void testMatches() -// { -// Assert.assertTrue(KM0.matches(KM0)); -// Assert.assertTrue(KM0.matches(KM1)); -// Assert.assertTrue(KM0.matches(KM2)); -// Assert.assertTrue(KM0.matches(KM3)); -// -// Assert.assertTrue(KM1.matches(KM0)); -// Assert.assertTrue(KM1.matches(KM1)); -// Assert.assertFalse(KM1.matches(KM2)); -// Assert.assertTrue(KM1.matches(KM3)); -// -// Assert.assertTrue(KM2.matches(KM0)); -// Assert.assertFalse(KM2.matches(KM1)); -// Assert.assertTrue(KM2.matches(KM2)); -// Assert.assertTrue(KM2.matches(KM3)); -// -// Assert.assertTrue(KM3.matches(KM0)); -// Assert.assertTrue(KM3.matches(KM1)); -// Assert.assertTrue(KM3.matches(KM2)); -// Assert.assertTrue(KM3.matches(KM3)); -// } -// -// @Test -// public void testIsValidStart() -// { -// Assert.assertTrue(KM0.isValidStart()); -// Assert.assertTrue(KM1.isValidStart()); -// Assert.assertTrue(KM2.isValidStart()); -// Assert.assertTrue(KM3.isValidStart()); -// } -// -// @Test -// public void testPlus() -// { -// Assert.assertEquals( -// KM("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)), -// KM1.plus(KM3) -// ); -// -// Assert.assertEquals( -// KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)), -// KM0.plus(KM2) -// ); -// -// Assert.assertEquals( -// KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)), -// KM1.plus(KM2) -// ); -// -// Assert.assertEquals( -// KM("foo", ImmutableMap.of(0, 2L, 1, 3L, 2, 5L)), -// KM2.plus(KM1) -// ); -// -// Assert.assertEquals( -// KM("foo", ImmutableMap.of(0, 2L, 1, 4L, 2, 5L)), -// KM2.plus(KM2) -// ); -// } -// -// @Test -// public void testMinus() -// { -// Assert.assertEquals( -// KM("foo", ImmutableMap.of(1, 3L)), -// KM1.minus(KM3) -// ); -// -// Assert.assertEquals( -// KM("foo", ImmutableMap.of()), -// KM0.minus(KM2) -// ); -// -// Assert.assertEquals( -// KM("foo", ImmutableMap.of()), -// KM1.minus(KM2) -// ); -// -// Assert.assertEquals( -// KM("foo", ImmutableMap.of(2, 5L)), -// KM2.minus(KM1) -// ); -// -// Assert.assertEquals( -// KM("foo", ImmutableMap.of()), -// KM2.minus(KM2) -// ); -// } -// -// private static KinesisDataSourceMetadata KM(String topic, Map offsets) -// { -// return new KinesisDataSourceMetadata(new KinesisPartitions(topic, offsets)); -// } -//} +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + + +package org.apache.druid.indexing.kinesis; + +import com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + +public class KinesisDataSourceMetadataTest +{ + private static final KinesisDataSourceMetadata KM0 = KM("foo", ImmutableMap.of()); + private static final KinesisDataSourceMetadata KM1 = KM("foo", ImmutableMap.of("0", "2L", "1", "3L")); + private static final KinesisDataSourceMetadata KM2 = KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")); + private static final KinesisDataSourceMetadata KM3 = KM("foo", ImmutableMap.of("0", "2L", "2", "5L")); + + @Test + public void testMatches() + { + Assert.assertTrue(KM0.matches(KM0)); + Assert.assertTrue(KM0.matches(KM1)); + Assert.assertTrue(KM0.matches(KM2)); + Assert.assertTrue(KM0.matches(KM3)); + + Assert.assertTrue(KM1.matches(KM0)); + Assert.assertTrue(KM1.matches(KM1)); + Assert.assertFalse(KM1.matches(KM2)); + Assert.assertTrue(KM1.matches(KM3)); + + Assert.assertTrue(KM2.matches(KM0)); + Assert.assertFalse(KM2.matches(KM1)); + Assert.assertTrue(KM2.matches(KM2)); + Assert.assertTrue(KM2.matches(KM3)); + + Assert.assertTrue(KM3.matches(KM0)); + Assert.assertTrue(KM3.matches(KM1)); + Assert.assertTrue(KM3.matches(KM2)); + Assert.assertTrue(KM3.matches(KM3)); + } + + @Test + public void testIsValidStart() + { + Assert.assertTrue(KM0.isValidStart()); + Assert.assertTrue(KM1.isValidStart()); + Assert.assertTrue(KM2.isValidStart()); + Assert.assertTrue(KM3.isValidStart()); + } + + @Test + public void testPlus() + { + Assert.assertEquals( + KM("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")), + KM1.plus(KM3) + ); + + Assert.assertEquals( + KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")), + KM0.plus(KM2) + ); + + Assert.assertEquals( + KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")), + KM1.plus(KM2) + ); + + Assert.assertEquals( + KM("foo", ImmutableMap.of("0", "2L", "1", "3L", "2", "5L")), + KM2.plus(KM1) + ); + + Assert.assertEquals( + KM("foo", ImmutableMap.of("0", "2L", "1", "4L", "2", "5L")), + KM2.plus(KM2) + ); + } + + @Test + public void testMinus() + { + Assert.assertEquals( + KM("foo", ImmutableMap.of("1", "3L")), + KM1.minus(KM3) + ); + + Assert.assertEquals( + KM("foo", ImmutableMap.of()), + KM0.minus(KM2) + ); + + Assert.assertEquals( + KM("foo", ImmutableMap.of()), + KM1.minus(KM2) + ); + + Assert.assertEquals( + KM("foo", ImmutableMap.of("2", "5L")), + KM2.minus(KM1) + ); + + Assert.assertEquals( + KM("foo", ImmutableMap.of()), + KM2.minus(KM2) + ); + } + + private static KinesisDataSourceMetadata KM(String stream, Map offsets) + { + return new KinesisDataSourceMetadata(new KinesisPartitions(stream, offsets)); + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index 25435f590095..2c26506f0118 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -1,251 +1,270 @@ -///* -// * Licensed to Metamarkets Group Inc. (Metamarkets) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. Metamarkets licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, -// * software distributed under the License is distributed on an -// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// * KIND, either express or implied. See the License for the -// * specific language governing permissions and limitations -// * under the License. -// */ -// -//package org.apache.druid.indexing.kinesis; -// -//import com.fasterxml.jackson.databind.JsonMappingException; -//import com.fasterxml.jackson.databind.Module; -//import com.fasterxml.jackson.databind.ObjectMapper; -//import com.google.common.collect.ImmutableMap; -//import org.apache.druid.jackson.DefaultObjectMapper; -//import org.apache.druid.segment.indexing.IOConfig; -//import org.hamcrest.CoreMatchers; -//import org.joda.time.DateTime; -//import org.junit.Assert; -//import org.junit.Rule; -//import org.junit.Test; -//import org.junit.rules.ExpectedException; -// -//public class KinesisIOConfigTest -//{ -// private final ObjectMapper mapper; -// -// public KinesisIOConfigTest() -// { -// mapper = new DefaultObjectMapper(); -// mapper.registerModules((Iterable) new KinesisIndexingServiceModule().getJacksonModules()); -// } -// -// @Rule -// public final ExpectedException exception = ExpectedException.none(); -// -// @Test -// public void testSerdeWithDefaults() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"baseSequenceName\": \"my-sequence-name\",\n" -// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" -// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n" -// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"}\n" -// + "}"; -// -// KinesisIOConfig config = (KinesisIOConfig) mapper.readValue( -// mapper.writeValueAsString( -// mapper.readValue( -// jsonStr, -// IOConfig.class -// ) -// ), IOConfig.class -// ); -// -// Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); -// Assert.assertEquals("mytopic", config.getStartPartitions().getStream()); -// Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartPartitions().getPartitionSequenceNumberMap()); -// Assert.assertEquals("mytopic", config.getEndPartitions().getStream()); -// Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndPartitions().getPartitionSequenceNumberMap()); -// Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties()); -// Assert.assertEquals(true, config.isUseTransaction()); -// Assert.assertEquals(false, config.isPauseAfterRead()); -// Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent()); -// Assert.assertFalse("skipOffsetGaps", config.isSkipOffsetGaps()); -// } -// -// @Test -// public void testSerdeWithNonDefaults() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"baseSequenceName\": \"my-sequence-name\",\n" -// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" -// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n" -// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" -// + " \"useTransaction\": false,\n" -// + " \"pauseAfterRead\": true,\n" -// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\",\n" -// + " \"skipOffsetGaps\": true\n" -// + "}"; -// -// KinesisIOConfig config = (KinesisIOConfig) mapper.readValue( -// mapper.writeValueAsString( -// mapper.readValue( -// jsonStr, -// IOConfig.class -// ) -// ), IOConfig.class -// ); -// -// Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); -// Assert.assertEquals("mytopic", config.getStartPartitions().getStream()); -// Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartPartitions().getPartitionSequenceNumberMap()); -// Assert.assertEquals("mytopic", config.getEndPartitions().getStream()); -// Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndPartitions().getPartitionSequenceNumberMap()); -// Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties()); -// Assert.assertEquals(false, config.isUseTransaction()); -// Assert.assertEquals(true, config.isPauseAfterRead()); -// Assert.assertEquals(new DateTime("2016-05-31T12:00Z"), config.getMinimumMessageTime().get()); -// Assert.assertTrue("skipOffsetGaps", config.isSkipOffsetGaps()); -// } -// -// @Test -// public void testBaseSequenceNameRequired() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" -// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n" -// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" -// + " \"useTransaction\": false,\n" -// + " \"pauseAfterRead\": true,\n" -// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" -// + "}"; -// -// exception.expect(JsonMappingException.class); -// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); -// exception.expectMessage(CoreMatchers.containsString("baseSequenceName")); -// mapper.readValue(jsonStr, IOConfig.class); -// } -// -// @Test -// public void testStartPartitionsRequired() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"baseSequenceName\": \"my-sequence-name\",\n" -// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n" -// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" -// + " \"useTransaction\": false,\n" -// + " \"pauseAfterRead\": true,\n" -// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" -// + "}"; -// -// exception.expect(JsonMappingException.class); -// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); -// exception.expectMessage(CoreMatchers.containsString("startPartitions")); -// mapper.readValue(jsonStr, IOConfig.class); -// } -// -// @Test -// public void testEndPartitionsRequired() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"baseSequenceName\": \"my-sequence-name\",\n" -// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" -// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" -// + " \"useTransaction\": false,\n" -// + " \"pauseAfterRead\": true,\n" -// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" -// + "}"; -// -// exception.expect(JsonMappingException.class); -// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); -// exception.expectMessage(CoreMatchers.containsString("endPartitions")); -// mapper.readValue(jsonStr, IOConfig.class); -// } -// -// @Test -// public void testConsumerPropertiesRequired() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"baseSequenceName\": \"my-sequence-name\",\n" -// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" -// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n" -// + " \"useTransaction\": false,\n" -// + " \"pauseAfterRead\": true,\n" -// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" -// + "}"; -// -// exception.expect(JsonMappingException.class); -// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); -// exception.expectMessage(CoreMatchers.containsString("consumerProperties")); -// mapper.readValue(jsonStr, IOConfig.class); -// } -// -// @Test -// public void testStartAndEndTopicMatch() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"baseSequenceName\": \"my-sequence-name\",\n" -// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" -// + " \"endPartitions\": {\"topic\":\"other\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n" -// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" -// + " \"useTransaction\": false,\n" -// + " \"pauseAfterRead\": true,\n" -// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" -// + "}"; -// -// exception.expect(JsonMappingException.class); -// exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class)); -// exception.expectMessage(CoreMatchers.containsString("start topic and end topic must match")); -// mapper.readValue(jsonStr, IOConfig.class); -// } -// -// @Test -// public void testStartAndEndPartitionSetMatch() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"baseSequenceName\": \"my-sequence-name\",\n" -// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" -// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15}},\n" -// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" -// + " \"useTransaction\": false,\n" -// + " \"pauseAfterRead\": true,\n" -// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" -// + "}"; -// -// exception.expect(JsonMappingException.class); -// exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class)); -// exception.expectMessage(CoreMatchers.containsString("start partition set and end partition set must match")); -// mapper.readValue(jsonStr, IOConfig.class); -// } -// -// @Test -// public void testEndOffsetGreaterThanStart() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"baseSequenceName\": \"my-sequence-name\",\n" -// + " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n" -// + " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":2}},\n" -// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" -// + " \"useTransaction\": false,\n" -// + " \"pauseAfterRead\": true,\n" -// + " \"minimumMessageTime\": \"2016-05-31T12:00Z\"\n" -// + "}"; -// -// exception.expect(JsonMappingException.class); -// exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class)); -// exception.expectMessage(CoreMatchers.containsString("end offset must be >= start offset")); -// mapper.readValue(jsonStr, IOConfig.class); -// } -//} +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.segment.indexing.IOConfig; +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class KinesisIOConfigTest +{ + private final ObjectMapper mapper; + + public KinesisIOConfigTest() + { + mapper = new DefaultObjectMapper(); + mapper.registerModules((Iterable) new KinesisIndexingServiceModule().getJacksonModules()); + } + + @Rule + public final ExpectedException exception = ExpectedException.none(); + + @Test + public void testSerdeWithDefaults() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"baseSequenceName\": \"my-sequence-name\",\n" + + " \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n" + + " \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n" + + " \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n" + + "}"; + + KinesisIOConfig config = (KinesisIOConfig) mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + IOConfig.class + ) + ), IOConfig.class + ); + + Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); + Assert.assertEquals("mystream", config.getStartPartitions().getStream()); + Assert.assertEquals( + ImmutableMap.of("0", "1", "1", "10"), + config.getStartPartitions().getPartitionSequenceNumberMap() + ); + Assert.assertEquals("mystream", config.getEndPartitions().getStream()); + Assert.assertEquals( + ImmutableMap.of("0", "15", "1", "200"), + config.getEndPartitions().getPartitionSequenceNumberMap() + ); + Assert.assertTrue(config.isUseTransaction()); + Assert.assertTrue(config.isPauseAfterRead()); + Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent()); + Assert.assertEquals(config.getEndpoint(), "kinesis.us-east-1.amazonaws.com"); + Assert.assertEquals(config.getRecordsPerFetch(), 4000); + Assert.assertEquals(config.getFetchDelayMillis(), 0); + Assert.assertNull(config.getAwsAccessKeyId()); + Assert.assertNull(config.getAwsSecretAccessKey()); + Assert.assertNull(config.getExclusiveStartSequenceNumberPartitions()); + Assert.assertNull(config.getAwsAssumedRoleArn()); + Assert.assertNull(config.getAwsExternalId()); + Assert.assertFalse(config.isDeaggregate()); + } + + @Test + public void testSerdeWithNonDefaults() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"baseSequenceName\": \"my-sequence-name\",\n" + + " \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n" + + " \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}},\n" + + " \"useTransaction\": false,\n" + + " \"pauseAfterRead\": false,\n" + + " \"minimumMessageTime\": \"2016-05-31T12:00Z\",\n" + + " \"maximumMessageTime\": \"2016-05-31T14:00Z\",\n" + + " \"endpoint\": \"kinesis.us-east-2.amazonaws.com\",\n" + + " \"recordsPerFetch\": 1000,\n" + + " \"fetchDelayMillis\": 1000,\n" + + " \"awsAccessKeyId\": \"awsid\",\n" + + " \"awsSecretAccessKey\": \"awskey\",\n" + + " \"exclusiveStartSequenceNumberPartitions\": [\"0\"],\n" + + " \"awsAssumedRoleArn\": \"role\",\n" + + " \"awsExternalId\": \"awsexternalid\",\n" + + " \"deaggregate\": true\n" + + "}"; + + KinesisIOConfig config = (KinesisIOConfig) mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + IOConfig.class + ) + ), IOConfig.class + ); + + Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); + Assert.assertEquals("mystream", config.getStartPartitions().getStream()); + Assert.assertEquals( + ImmutableMap.of("0", "1", "1", "10"), + config.getStartPartitions().getPartitionSequenceNumberMap() + ); + Assert.assertEquals("mystream", config.getEndPartitions().getStream()); + Assert.assertEquals( + ImmutableMap.of("0", "15", "1", "200"), + config.getEndPartitions().getPartitionSequenceNumberMap() + ); + Assert.assertFalse(config.isUseTransaction()); + Assert.assertFalse(config.isPauseAfterRead()); + Assert.assertTrue("maximumMessageTime", config.getMaximumMessageTime().isPresent()); + Assert.assertTrue("minimumMessageTime", config.getMinimumMessageTime().isPresent()); + Assert.assertEquals(DateTimes.of("2016-05-31T12:00Z"), config.getMinimumMessageTime().get()); + Assert.assertEquals(DateTimes.of("2016-05-31T14:00Z"), config.getMaximumMessageTime().get()); + Assert.assertEquals(config.getEndpoint(), "kinesis.us-east-2.amazonaws.com"); + Assert.assertEquals(config.getExclusiveStartSequenceNumberPartitions(), ImmutableSet.of("0")); + Assert.assertEquals(4000, config.getRecordsPerFetch()); + Assert.assertEquals(1000, config.getFetchDelayMillis()); + Assert.assertEquals("awsid", config.getAwsAccessKeyId()); + Assert.assertEquals("awskey", config.getAwsSecretAccessKey()); + Assert.assertEquals("role", config.getAwsAssumedRoleArn()); + Assert.assertEquals("awsexternalid", config.getAwsExternalId()); + Assert.assertTrue(config.isDeaggregate()); + } + + @Test + public void testBaseSequenceNameRequired() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n" + + " \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n" + + " \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n" + + "}"; + + exception.expect(JsonMappingException.class); + exception.expectCause(CoreMatchers.isA(NullPointerException.class)); + exception.expectMessage(CoreMatchers.containsString("baseSequenceName")); + mapper.readValue(jsonStr, IOConfig.class); + } + + @Test + public void testStartPartitionsRequired() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"baseSequenceName\": \"my-sequence-name\",\n" + + " \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n" + + " \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n" + + "}"; + + exception.expect(JsonMappingException.class); + exception.expectCause(CoreMatchers.isA(NullPointerException.class)); + exception.expectMessage(CoreMatchers.containsString("startPartitions")); + mapper.readValue(jsonStr, IOConfig.class); + } + + @Test + public void testEndPartitionsRequired() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"baseSequenceName\": \"my-sequence-name\",\n" + + " \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n" + + " \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}}\n" + + "}"; + + exception.expect(JsonMappingException.class); + exception.expectCause(CoreMatchers.isA(NullPointerException.class)); + exception.expectMessage(CoreMatchers.containsString("endPartitions")); + mapper.readValue(jsonStr, IOConfig.class); + } + + @Test + public void testStartAndEndstreamMatch() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"baseSequenceName\": \"my-sequence-name\",\n" + + " \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n" + + " \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n" + + " \"endPartitions\": {\"stream\":\"notmystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n" + + "}"; + + exception.expect(JsonMappingException.class); + exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class)); + exception.expectMessage(CoreMatchers.containsString("must match")); + mapper.readValue(jsonStr, IOConfig.class); + } + + @Test + public void testStartAndEndPartitionSetMatch() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"baseSequenceName\": \"my-sequence-name\",\n" + + " \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n" + + " \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n" + + " \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"2\":\"200\"}}\n" + + "}"; + + exception.expect(JsonMappingException.class); + exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class)); + exception.expectMessage(CoreMatchers.containsString("start partition set and end partition set must match")); + mapper.readValue(jsonStr, IOConfig.class); + } + + /* + @Test + public void testEndOffsetGreaterThanStart() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"baseSequenceName\": \"my-sequence-name\",\n" + + " \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n" + + " \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n" + + " \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"9\"}}\n" + + "}"; + + exception.expect(JsonMappingException.class); + exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class)); + exception.expectMessage(CoreMatchers.containsString("end offset must be >= start offset")); + mapper.readValue(jsonStr, IOConfig.class); + } + */ + + @Test + public void testEndPointRequired() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"baseSequenceName\": \"my-sequence-name\",\n" + + " \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n" + + " \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n" + + "}"; + + exception.expect(JsonMappingException.class); + exception.expectCause(CoreMatchers.isA(NullPointerException.class)); + exception.expectMessage(CoreMatchers.containsString("endpoint")); + mapper.readValue(jsonStr, IOConfig.class); + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java index b7d267d9371c..ee8dde9b4b63 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java @@ -1,985 +1,960 @@ -///* -// * Licensed to Metamarkets Group Inc. (Metamarkets) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. Metamarkets licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, -// * software distributed under the License is distributed on an -// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// * KIND, either express or implied. See the License for the -// * specific language governing permissions and limitations -// * under the License. -// */ -// -//package org.apache.druid.indexing.kinesis; -// -//import com.fasterxml.jackson.databind.ObjectMapper; -//import com.google.common.base.Optional; -//import com.google.common.collect.ImmutableList; -//import com.google.common.collect.ImmutableMap; -//import com.google.common.collect.Lists; -//import com.google.common.collect.Maps; -//import com.google.common.util.concurrent.Futures; -//import com.google.common.util.concurrent.ListenableFuture; -//import com.metamx.http.client.HttpClient; -//import com.metamx.http.client.Request; -//import com.metamx.http.client.response.FullResponseHandler; -//import com.metamx.http.client.response.FullResponseHolder; -//import org.apache.druid.indexing.common.TaskInfoProvider; -//import org.apache.druid.indexing.common.TaskLocation; -//import org.apache.druid.indexing.common.TaskStatus; -//import org.apache.druid.jackson.DefaultObjectMapper; -//import org.apache.druid.java.util.common.IAE; -//import org.easymock.Capture; -//import org.easymock.CaptureType; -//import org.easymock.EasyMockSupport; -//import org.jboss.netty.handler.codec.http.HttpHeaders; -//import org.jboss.netty.handler.codec.http.HttpMethod; -//import org.jboss.netty.handler.codec.http.HttpResponse; -//import org.jboss.netty.handler.codec.http.HttpResponseStatus; -//import org.joda.time.DateTime; -//import org.joda.time.Duration; -//import org.junit.After; -//import org.junit.Assert; -//import org.junit.Before; -//import org.junit.Test; -//import org.junit.runner.RunWith; -//import org.junit.runners.Parameterized; -// -//import java.io.IOException; -//import java.net.URL; -//import java.util.List; -//import java.util.Map; -// -//import static org.easymock.EasyMock.anyObject; -//import static org.easymock.EasyMock.capture; -//import static org.easymock.EasyMock.eq; -//import static org.easymock.EasyMock.expect; -//import static org.easymock.EasyMock.reset; -// -//@RunWith(Parameterized.class) -//public class KinesisIndexTaskClientTest extends EasyMockSupport -//{ -// private static final ObjectMapper objectMapper = new DefaultObjectMapper(); -// private static final String TEST_ID = "test-id"; -// private static final List TEST_IDS = Lists.newArrayList("test-id1", "test-id2", "test-id3", "test-id4"); -// private static final String TEST_HOST = "test-host"; -// private static final int TEST_PORT = 1234; -// private static final String TEST_DATASOURCE = "test-datasource"; -// private static final Duration TEST_HTTP_TIMEOUT = new Duration(5000); -// private static final long TEST_NUM_RETRIES = 0; -// private static final String URL_FORMATTER = "http://%s:%d/druid/worker/v1/chat/%s/%s"; -// -// private int numThreads; -// private HttpClient httpClient; -// private TaskInfoProvider taskInfoProvider; -// private FullResponseHolder responseHolder; -// private HttpResponse response; -// private HttpHeaders headers; -// private KinesisIndexTaskClient client; -// -// @Parameterized.Parameters(name = "numThreads = {0}") -// public static Iterable constructorFeeder() -// { -// return ImmutableList.of(new Object[]{1}, new Object[]{8}); -// } -// -// public KinesisIndexTaskClientTest(int numThreads) -// { -// this.numThreads = numThreads; -// } -// -// @Before -// public void setUp() throws Exception -// { -// httpClient = createMock(HttpClient.class); -// taskInfoProvider = createMock(TaskInfoProvider.class); -// responseHolder = createMock(FullResponseHolder.class); -// response = createMock(HttpResponse.class); -// headers = createMock(HttpHeaders.class); -// -// client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider); -// expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT)).anyTimes(); -// expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); -// -// for (int i = 0; i < TEST_IDS.size(); i++) { -// expect(taskInfoProvider.getTaskLocation(TEST_IDS.get(i))).andReturn(new TaskLocation(TEST_HOST, TEST_PORT)) -// .anyTimes(); -// expect(taskInfoProvider.getTaskStatus(TEST_IDS.get(i))).andReturn(Optional.of(TaskStatus.running(TEST_IDS.get(i)))) -// .anyTimes(); -// } -// } -// -// @After -// public void tearDown() throws Exception -// { -// client.close(); -// } -// -// @Test -// public void testNoTaskLocation() throws Exception -// { -// reset(taskInfoProvider); -// expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); -// expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); -// replayAll(); -// -// Assert.assertEquals(false, client.stop(TEST_ID, true)); -// Assert.assertEquals(false, client.resume(TEST_ID)); -// Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID)); -// Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID, 10)); -// Assert.assertEquals(KinesisIndexTask.Status.NOT_STARTED, client.getStatus(TEST_ID)); -// Assert.assertEquals(null, client.getStartTime(TEST_ID)); -// Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsets(TEST_ID, true)); -// Assert.assertEquals(ImmutableMap.of(), client.getEndOffsets(TEST_ID)); -// Assert.assertEquals(false, client.setEndOffsets(TEST_ID, ImmutableMap.of())); -// Assert.assertEquals(false, client.setEndOffsets(TEST_ID, ImmutableMap.of(), true)); -// -// verifyAll(); -// } -// -// @Test(expected = KinesisIndexTaskClient.TaskNotRunnableException.class) -// public void testTaskNotRunnableException() throws Exception -// { -// reset(taskInfoProvider); -// expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT)).anyTimes(); -// expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.failure(TEST_ID))).anyTimes(); -// replayAll(); -// -// client.getCurrentOffsets(TEST_ID, true); -// verifyAll(); -// } -// -// @Test(expected = RuntimeException.class) -// public void testInternalServerError() throws Exception -// { -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2); -// expect( -// httpClient.go( -// anyObject(Request.class), -// anyObject(FullResponseHandler.class), -// eq(TEST_HTTP_TIMEOUT) -// ) -// ).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// replayAll(); -// -// client.getCurrentOffsets(TEST_ID, true); -// verifyAll(); -// } -// -// @Test(expected = IAE.class) -// public void testBadRequest() throws Exception -// { -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2); -// expect(responseHolder.getContent()).andReturn(""); -// expect( -// httpClient.go( -// anyObject(Request.class), -// anyObject(FullResponseHandler.class), -// eq(TEST_HTTP_TIMEOUT) -// ) -// ).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// replayAll(); -// -// client.getCurrentOffsets(TEST_ID, true); -// verifyAll(); -// } -// -// @Test -// public void testTaskLocationMismatch() throws Exception -// { -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) -// .andReturn(HttpResponseStatus.OK); -// expect(responseHolder.getResponse()).andReturn(response); -// expect(responseHolder.getContent()).andReturn("") -// .andReturn("{}"); -// expect(response.headers()).andReturn(headers); -// expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id"); -// expect( -// httpClient.go( -// anyObject(Request.class), -// anyObject(FullResponseHandler.class), -// eq(TEST_HTTP_TIMEOUT) -// ) -// ).andReturn( -// Futures.immediateFuture(responseHolder) -// ).times(2); -// replayAll(); -// -// Map results = client.getCurrentOffsets(TEST_ID, true); -// verifyAll(); -// -// Assert.assertEquals(0, results.size()); -// } -// -// @Test -// public void testGetCurrentOffsets() throws Exception -// { -// Capture captured = Capture.newInstance(); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); -// expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// replayAll(); -// -// Map results = client.getCurrentOffsets(TEST_ID, true); -// verifyAll(); -// -// Request request = captured.getValue(); -// Assert.assertEquals(HttpMethod.GET, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"), -// request.getUrl() -// ); -// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); -// -// Assert.assertEquals(2, results.size()); -// Assert.assertEquals(1, (long) results.get(0)); -// Assert.assertEquals(10, (long) results.get(1)); -// } -// -// @Test -// public void testGetCurrentOffsetsWithRetry() throws Exception -// { -// client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 3); -// -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) -// .andReturn(HttpResponseStatus.OK).times(1); -// expect(responseHolder.getContent()).andReturn("").times(2) -// .andReturn("{\"0\":1, \"1\":10}"); -// expect(responseHolder.getResponse()).andReturn(response).times(2); -// expect(response.headers()).andReturn(headers).times(2); -// expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2); -// -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ).times(3); -// -// replayAll(); -// -// Map results = client.getCurrentOffsets(TEST_ID, true); -// verifyAll(); -// -// Assert.assertEquals(3, captured.getValues().size()); -// for (Request request : captured.getValues()) { -// Assert.assertEquals(HttpMethod.GET, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"), -// request.getUrl() -// ); -// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); -// } -// -// Assert.assertEquals(2, results.size()); -// Assert.assertEquals(1, (long) results.get(0)); -// Assert.assertEquals(10, (long) results.get(1)); -// } -// -// @Test(expected = RuntimeException.class) -// public void testGetCurrentOffsetsWithExhaustedRetries() throws Exception -// { -// client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); -// -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); -// expect(responseHolder.getContent()).andReturn("").anyTimes(); -// expect(responseHolder.getResponse()).andReturn(response).anyTimes(); -// expect(response.headers()).andReturn(headers).anyTimes(); -// expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).anyTimes(); -// -// expect( -// httpClient.go( -// anyObject(Request.class), -// anyObject(FullResponseHandler.class), -// eq(TEST_HTTP_TIMEOUT) -// ) -// ).andReturn(Futures.immediateFuture(responseHolder)).anyTimes(); -// replayAll(); -// -// client.getCurrentOffsets(TEST_ID, true); -// verifyAll(); -// } -// -// @Test -// public void testGetEndOffsets() throws Exception -// { -// Capture captured = Capture.newInstance(); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); -// expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// replayAll(); -// -// Map results = client.getEndOffsets(TEST_ID); -// verifyAll(); -// -// Request request = captured.getValue(); -// Assert.assertEquals(HttpMethod.GET, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end"), -// request.getUrl() -// ); -// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); -// -// Assert.assertEquals(2, results.size()); -// Assert.assertEquals(1, (long) results.get(0)); -// Assert.assertEquals(10, (long) results.get(1)); -// } -// -// @Test -// public void testGetStartTime() throws Exception -// { -// client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); -// DateTime now = DateTime.now(); -// -// Capture captured = Capture.newInstance(); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) -// .andReturn(HttpResponseStatus.OK); -// expect(responseHolder.getResponse()).andReturn(response); -// expect(response.headers()).andReturn(headers); -// expect(headers.get("X-Druid-Task-Id")).andReturn(null); -// expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ).times(2); -// replayAll(); -// -// DateTime results = client.getStartTime(TEST_ID); -// verifyAll(); -// -// Request request = captured.getValue(); -// Assert.assertEquals(HttpMethod.GET, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/time/start"), -// request.getUrl() -// ); -// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); -// -// Assert.assertEquals(now, results); -// } -// -// @Test -// public void testGetStatus() throws Exception -// { -// KinesisIndexTask.Status status = KinesisIndexTask.Status.READING; -// -// Capture captured = Capture.newInstance(); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); -// expect(responseHolder.getContent()).andReturn(String.format("\"%s\"", status.toString())).anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// replayAll(); -// -// KinesisIndexTask.Status results = client.getStatus(TEST_ID); -// verifyAll(); -// -// Request request = captured.getValue(); -// Assert.assertEquals(HttpMethod.GET, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/status"), -// request.getUrl() -// ); -// Assert.assertTrue(null, request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); -// -// Assert.assertEquals(status, results); -// } -// -// @Test -// public void testPause() throws Exception -// { -// Capture captured = Capture.newInstance(); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); -// expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// replayAll(); -// -// Map results = client.pause(TEST_ID); -// verifyAll(); -// -// Request request = captured.getValue(); -// Assert.assertEquals(HttpMethod.POST, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/pause"), -// request.getUrl() -// ); -// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); -// -// Assert.assertEquals(2, results.size()); -// Assert.assertEquals(1, (long) results.get(0)); -// Assert.assertEquals(10, (long) results.get(1)); -// } -// -// @Test -// public void testPauseWithTimeout() throws Exception -// { -// Capture captured = Capture.newInstance(); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); -// expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// replayAll(); -// -// Map results = client.pause(TEST_ID, 101); -// verifyAll(); -// -// Request request = captured.getValue(); -// Assert.assertEquals(HttpMethod.POST, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/pause?timeout=101"), -// request.getUrl() -// ); -// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); -// -// Assert.assertEquals(2, results.size()); -// Assert.assertEquals(1, (long) results.get(0)); -// Assert.assertEquals(10, (long) results.get(1)); -// } -// -// @Test -// public void testPauseWithSubsequentGetOffsets() throws Exception -// { -// Capture captured = Capture.newInstance(); -// Capture captured2 = Capture.newInstance(); -// Capture captured3 = Capture.newInstance(); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.ACCEPTED).times(2) -// .andReturn(HttpResponseStatus.OK).times(2); -// expect(responseHolder.getContent()).andReturn("\"PAUSED\"") -// .andReturn("{\"0\":1, \"1\":10}").anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// expect(httpClient.go(capture(captured2), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// expect(httpClient.go(capture(captured3), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// -// replayAll(); -// -// Map results = client.pause(TEST_ID); -// verifyAll(); -// -// Request request = captured.getValue(); -// Assert.assertEquals(HttpMethod.POST, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/pause"), -// request.getUrl() -// ); -// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); -// -// request = captured2.getValue(); -// Assert.assertEquals(HttpMethod.GET, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/status"), -// request.getUrl() -// ); -// -// request = captured3.getValue(); -// Assert.assertEquals(HttpMethod.GET, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"), -// request.getUrl() -// ); -// -// Assert.assertEquals(2, results.size()); -// Assert.assertEquals(1, (long) results.get(0)); -// Assert.assertEquals(10, (long) results.get(1)); -// } -// -// @Test -// public void testResume() throws Exception -// { -// Capture captured = Capture.newInstance(); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// replayAll(); -// -// client.resume(TEST_ID); -// verifyAll(); -// -// Request request = captured.getValue(); -// Assert.assertEquals(HttpMethod.POST, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/resume"), -// request.getUrl() -// ); -// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); -// } -// -// @Test -// public void testSetEndOffsets() throws Exception -// { -// Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); -// -// Capture captured = Capture.newInstance(); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// replayAll(); -// -// client.setEndOffsets(TEST_ID, endOffsets); -// verifyAll(); -// -// Request request = captured.getValue(); -// Assert.assertEquals(HttpMethod.POST, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end"), -// request.getUrl() -// ); -// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); -// Assert.assertEquals("{\"0\":15,\"1\":120}", new String(request.getContent().array())); -// } -// -// @Test -// public void testSetEndOffsetsAndResume() throws Exception -// { -// Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); -// -// Capture captured = Capture.newInstance(); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// replayAll(); -// -// client.setEndOffsets(TEST_ID, endOffsets, true); -// verifyAll(); -// -// Request request = captured.getValue(); -// Assert.assertEquals(HttpMethod.POST, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end?resume=true"), -// request.getUrl() -// ); -// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); -// Assert.assertEquals("{\"0\":15,\"1\":120}", new String(request.getContent().array())); -// } -// -// @Test -// public void testStop() throws Exception -// { -// Capture captured = Capture.newInstance(); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// replayAll(); -// -// client.stop(TEST_ID, false); -// verifyAll(); -// -// Request request = captured.getValue(); -// Assert.assertEquals(HttpMethod.POST, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/stop"), -// request.getUrl() -// ); -// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); -// } -// -// @Test -// public void testStopAndPublish() throws Exception -// { -// Capture captured = Capture.newInstance(); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ); -// replayAll(); -// -// client.stop(TEST_ID, true); -// verifyAll(); -// -// Request request = captured.getValue(); -// Assert.assertEquals(HttpMethod.POST, request.getMethod()); -// Assert.assertEquals( -// new URL("http://test-host:1234/druid/worker/v1/chat/test-id/stop?publish=true"), -// request.getUrl() -// ); -// Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); -// } -// -// @Test -// public void testStopAsync() throws Exception -// { -// final int numRequests = TEST_IDS.size(); -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ).times(numRequests); -// replayAll(); -// -// List expectedUrls = Lists.newArrayList(); -// List> futures = Lists.newArrayList(); -// for (int i = 0; i < numRequests; i++) { -// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "stop"))); -// futures.add(client.stopAsync(TEST_IDS.get(i), false)); -// } -// -// List responses = Futures.allAsList(futures).get(); -// -// verifyAll(); -// List requests = captured.getValues(); -// -// Assert.assertEquals(numRequests, requests.size()); -// Assert.assertEquals(numRequests, responses.size()); -// for (int i = 0; i < numRequests; i++) { -// Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); -// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); -// Assert.assertTrue(responses.get(i)); -// } -// } -// -// @Test -// public void testResumeAsync() throws Exception -// { -// final int numRequests = TEST_IDS.size(); -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ).times(numRequests); -// replayAll(); -// -// List expectedUrls = Lists.newArrayList(); -// List> futures = Lists.newArrayList(); -// for (int i = 0; i < numRequests; i++) { -// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "resume"))); -// futures.add(client.resumeAsync(TEST_IDS.get(i))); -// } -// -// List responses = Futures.allAsList(futures).get(); -// -// verifyAll(); -// List requests = captured.getValues(); -// -// Assert.assertEquals(numRequests, requests.size()); -// Assert.assertEquals(numRequests, responses.size()); -// for (int i = 0; i < numRequests; i++) { -// Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); -// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); -// Assert.assertTrue(responses.get(i)); -// } -// } -// -// @Test -// public void testPauseAsync() throws Exception -// { -// final int numRequests = TEST_IDS.size(); -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ).times(numRequests); -// replayAll(); -// -// List expectedUrls = Lists.newArrayList(); -// List>> futures = Lists.newArrayList(); -// for (int i = 0; i < numRequests; i++) { -// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "pause"))); -// futures.add(client.pauseAsync(TEST_IDS.get(i))); -// } -// -// List> responses = Futures.allAsList(futures).get(); -// -// verifyAll(); -// List requests = captured.getValues(); -// -// Assert.assertEquals(numRequests, requests.size()); -// Assert.assertEquals(numRequests, responses.size()); -// for (int i = 0; i < numRequests; i++) { -// Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); -// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); -// Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of(0, 1L)), responses.get(i)); -// } -// } -// -// @Test -// public void testPauseAsyncWithTimeout() throws Exception -// { -// final int numRequests = TEST_IDS.size(); -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ).times(numRequests); -// replayAll(); -// -// List expectedUrls = Lists.newArrayList(); -// List>> futures = Lists.newArrayList(); -// for (int i = 0; i < numRequests; i++) { -// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "pause?timeout=9"))); -// futures.add(client.pauseAsync(TEST_IDS.get(i), 9)); -// } -// -// List> responses = Futures.allAsList(futures).get(); -// -// verifyAll(); -// List requests = captured.getValues(); -// -// Assert.assertEquals(numRequests, requests.size()); -// Assert.assertEquals(numRequests, responses.size()); -// for (int i = 0; i < numRequests; i++) { -// Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); -// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); -// Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of(0, 1L)), responses.get(i)); -// } -// } -// -// @Test -// public void testGetStatusAsync() throws Exception -// { -// final int numRequests = TEST_IDS.size(); -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ).times(numRequests); -// replayAll(); -// -// List expectedUrls = Lists.newArrayList(); -// List> futures = Lists.newArrayList(); -// for (int i = 0; i < numRequests; i++) { -// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "status"))); -// futures.add(client.getStatusAsync(TEST_IDS.get(i))); -// } -// -// List responses = Futures.allAsList(futures).get(); -// -// verifyAll(); -// List requests = captured.getValues(); -// -// Assert.assertEquals(numRequests, requests.size()); -// Assert.assertEquals(numRequests, responses.size()); -// for (int i = 0; i < numRequests; i++) { -// Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); -// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); -// Assert.assertEquals(KinesisIndexTask.Status.READING, responses.get(i)); -// } -// } -// -// @Test -// public void testGetStartTimeAsync() throws Exception -// { -// final DateTime now = DateTime.now(); -// final int numRequests = TEST_IDS.size(); -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ).times(numRequests); -// replayAll(); -// -// List expectedUrls = Lists.newArrayList(); -// List> futures = Lists.newArrayList(); -// for (int i = 0; i < numRequests; i++) { -// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "time/start"))); -// futures.add(client.getStartTimeAsync(TEST_IDS.get(i))); -// } -// -// List responses = Futures.allAsList(futures).get(); -// -// verifyAll(); -// List requests = captured.getValues(); -// -// Assert.assertEquals(numRequests, requests.size()); -// Assert.assertEquals(numRequests, responses.size()); -// for (int i = 0; i < numRequests; i++) { -// Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); -// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); -// Assert.assertEquals(now, responses.get(i)); -// } -// } -// -// @Test -// public void testGetCurrentOffsetsAsync() throws Exception -// { -// final int numRequests = TEST_IDS.size(); -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ).times(numRequests); -// replayAll(); -// -// List expectedUrls = Lists.newArrayList(); -// List>> futures = Lists.newArrayList(); -// for (int i = 0; i < numRequests; i++) { -// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/current"))); -// futures.add(client.getCurrentOffsetsAsync(TEST_IDS.get(i), false)); -// } -// -// List> responses = Futures.allAsList(futures).get(); -// -// verifyAll(); -// List requests = captured.getValues(); -// -// Assert.assertEquals(numRequests, requests.size()); -// Assert.assertEquals(numRequests, responses.size()); -// for (int i = 0; i < numRequests; i++) { -// Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); -// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); -// Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of(0, 1L)), responses.get(i)); -// } -// } -// -// @Test -// public void testGetEndOffsetsAsync() throws Exception -// { -// final int numRequests = TEST_IDS.size(); -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ).times(numRequests); -// replayAll(); -// -// List expectedUrls = Lists.newArrayList(); -// List>> futures = Lists.newArrayList(); -// for (int i = 0; i < numRequests; i++) { -// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/end"))); -// futures.add(client.getEndOffsetsAsync(TEST_IDS.get(i))); -// } -// -// List> responses = Futures.allAsList(futures).get(); -// -// verifyAll(); -// List requests = captured.getValues(); -// -// Assert.assertEquals(numRequests, requests.size()); -// Assert.assertEquals(numRequests, responses.size()); -// for (int i = 0; i < numRequests; i++) { -// Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); -// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); -// Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of(0, 1L)), responses.get(i)); -// } -// } -// -// @Test -// public void testSetEndOffsetsAsync() throws Exception -// { -// final Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); -// final int numRequests = TEST_IDS.size(); -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ).times(numRequests); -// replayAll(); -// -// List expectedUrls = Lists.newArrayList(); -// List> futures = Lists.newArrayList(); -// for (int i = 0; i < numRequests; i++) { -// expectedUrls.add(new URL(String.format(URL_FORMATTER, TEST_HOST, TEST_PORT, TEST_IDS.get(i), "offsets/end"))); -// futures.add(client.setEndOffsetsAsync(TEST_IDS.get(i), endOffsets)); -// } -// -// List responses = Futures.allAsList(futures).get(); -// -// verifyAll(); -// List requests = captured.getValues(); -// -// Assert.assertEquals(numRequests, requests.size()); -// Assert.assertEquals(numRequests, responses.size()); -// for (int i = 0; i < numRequests; i++) { -// Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); -// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); -// Assert.assertTrue(responses.get(i)); -// } -// } -// -// @Test -// public void testSetEndOffsetsAsyncWithResume() throws Exception -// { -// final Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); -// final int numRequests = TEST_IDS.size(); -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); -// expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( -// Futures.immediateFuture(responseHolder) -// ).times(numRequests); -// replayAll(); -// -// List expectedUrls = Lists.newArrayList(); -// List> futures = Lists.newArrayList(); -// for (int i = 0; i < numRequests; i++) { -// expectedUrls.add( -// new URL( -// String.format( -// URL_FORMATTER, -// TEST_HOST, -// TEST_PORT, -// TEST_IDS.get(i), -// "offsets/end?resume=true" -// ) -// ) -// ); -// futures.add(client.setEndOffsetsAsync(TEST_IDS.get(i), endOffsets, true)); -// } -// -// List responses = Futures.allAsList(futures).get(); -// -// verifyAll(); -// List requests = captured.getValues(); -// -// Assert.assertEquals(numRequests, requests.size()); -// Assert.assertEquals(numRequests, responses.size()); -// for (int i = 0; i < numRequests; i++) { -// Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); -// Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); -// Assert.assertTrue(responses.get(i)); -// } -// } -// -// private class TestableKinesisIndexTaskClient extends KinesisIndexTaskClient -// { -// public TestableKinesisIndexTaskClient( -// HttpClient httpClient, -// ObjectMapper jsonMapper, -// TaskInfoProvider taskInfoProvider -// ) -// { -// this(httpClient, jsonMapper, taskInfoProvider, TEST_NUM_RETRIES); -// } -// -// public TestableKinesisIndexTaskClient( -// HttpClient httpClient, -// ObjectMapper jsonMapper, -// TaskInfoProvider taskInfoProvider, -// long numRetries -// ) -// { -// super(httpClient, jsonMapper, taskInfoProvider, TEST_DATASOURCE, numThreads, TEST_HTTP_TIMEOUT, numRetries); -// } -// -// @Override -// void checkConnection(String host, int port) throws IOException { } -// } -//} +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.IndexTaskClient; +import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.FullResponseHandler; +import org.apache.druid.java.util.http.client.response.FullResponseHolder; +import org.easymock.Capture; +import org.easymock.CaptureType; +import org.easymock.EasyMockSupport; +import org.jboss.netty.handler.codec.http.HttpHeaders; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponse; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.net.URL; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.capture; +import static org.easymock.EasyMock.eq; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.reset; + +@RunWith(Parameterized.class) +public class KinesisIndexTaskClientTest extends EasyMockSupport +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private static final ObjectMapper objectMapper = new DefaultObjectMapper(); + private static final String TEST_ID = "test-id"; + private static final List TEST_IDS = Lists.newArrayList("test-id1", "test-id2", "test-id3", "test-id4"); + private static final String TEST_HOST = "test-host"; + private static final int TEST_PORT = 1234; + private static final int TEST_TLS_PORT = -1; + private static final String TEST_DATASOURCE = "test-datasource"; + private static final Duration TEST_HTTP_TIMEOUT = new Duration(5000); + private static final long TEST_NUM_RETRIES = 0; + private static final String URL_FORMATTER = "http://%s:%d/druid/worker/v1/chat/%s/%s"; + + private int numThreads; + private HttpClient httpClient; + private TaskInfoProvider taskInfoProvider; + private FullResponseHolder responseHolder; + private HttpResponse response; + private HttpHeaders headers; + private KinesisIndexTaskClient client; + + @Parameterized.Parameters(name = "numThreads = {0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of(new Object[]{1}, new Object[]{8}); + } + + public KinesisIndexTaskClientTest(int numThreads) + { + this.numThreads = numThreads; + } + + @Before + public void setUp() + { + httpClient = createMock(HttpClient.class); + taskInfoProvider = createMock(TaskInfoProvider.class); + responseHolder = createMock(FullResponseHolder.class); + response = createMock(HttpResponse.class); + headers = createMock(HttpHeaders.class); + + client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider); + expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); + + for (String testId : TEST_IDS) { + expect(taskInfoProvider.getTaskLocation(testId)) + .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + expect(taskInfoProvider.getTaskStatus(testId)) + .andReturn(Optional.of(TaskStatus.running(testId))) + .anyTimes(); + } + } + + @After + public void tearDown() + { + client.close(); + } + + @Test + public void testNoTaskLocation() throws IOException + { + reset(taskInfoProvider); + expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); + expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); + replayAll(); + + Assert.assertFalse(client.stop(TEST_ID, true)); + Assert.assertFalse(client.resume(TEST_ID)); + Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID)); + Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID)); + Assert.assertEquals(SeekableStreamIndexTask.Status.NOT_STARTED, client.getStatus(TEST_ID)); + Assert.assertNull(client.getStartTime(TEST_ID)); + Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsets(TEST_ID, true)); + Assert.assertEquals(ImmutableMap.of(), client.getEndOffsets(TEST_ID)); + Assert.assertFalse(client.setEndOffsets(TEST_ID, Collections.emptyMap(), true)); + Assert.assertFalse(client.setEndOffsets(TEST_ID, Collections.emptyMap(), true)); + + verifyAll(); + } + + @Test + public void testTaskNotRunnableException() + { + expectedException.expect(IndexTaskClient.TaskNotRunnableException.class); + expectedException.expectMessage("Aborting request because task [test-id] is not runnable"); + + reset(taskInfoProvider); + expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.failure(TEST_ID))).anyTimes(); + replayAll(); + + client.getCurrentOffsets(TEST_ID, true); + verifyAll(); + } + + @Test + public void testInternalServerError() + { + expectedException.expect(RuntimeException.class); + expectedException.expectMessage("org.apache.druid.java.util.common.IOE: Received status [500]"); + + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2); + expect( + httpClient.go( + anyObject(Request.class), + anyObject(FullResponseHandler.class), + eq(TEST_HTTP_TIMEOUT) + ) + ).andReturn( + Futures.immediateFuture(responseHolder) + ); + replayAll(); + + client.getCurrentOffsets(TEST_ID, true); + verifyAll(); + } + + @Test + public void testBadRequest() + { + expectedException.expect(IAE.class); + expectedException.expectMessage("Received 400 Bad Request with body:"); + + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2); + expect(responseHolder.getContent()).andReturn(""); + expect( + httpClient.go( + anyObject(Request.class), + anyObject(FullResponseHandler.class), + eq(TEST_HTTP_TIMEOUT) + ) + ).andReturn( + Futures.immediateFuture(responseHolder) + ); + replayAll(); + + client.getCurrentOffsets(TEST_ID, true); + verifyAll(); + } + + @Test + public void testTaskLocationMismatch() + { + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) + .andReturn(HttpResponseStatus.OK); + expect(responseHolder.getResponse()).andReturn(response); + expect(responseHolder.getContent()).andReturn("") + .andReturn("{}"); + expect(response.headers()).andReturn(headers); + expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id"); + expect( + httpClient.go( + anyObject(Request.class), + anyObject(FullResponseHandler.class), + eq(TEST_HTTP_TIMEOUT) + ) + ).andReturn( + Futures.immediateFuture(responseHolder) + ).times(2); + replayAll(); + + Map results = client.getCurrentOffsets(TEST_ID, true); + verifyAll(); + + Assert.assertEquals(0, results.size()); + } + + @Test + public void testGetCurrentOffsets() throws Exception + { + Capture captured = Capture.newInstance(); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ); + replayAll(); + + Map results = client.getCurrentOffsets(TEST_ID, true); + verifyAll(); + + Request request = captured.getValue(); + Assert.assertEquals(HttpMethod.GET, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"), + request.getUrl() + ); + Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + + Assert.assertEquals(2, results.size()); + Assert.assertEquals("1", results.get("0")); + Assert.assertEquals("10", results.get("1")); + } + + @Test + public void testGetCurrentOffsetsWithRetry() throws Exception + { + client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 3); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) + .andReturn(HttpResponseStatus.OK).times(1); + expect(responseHolder.getContent()).andReturn("").times(2) + .andReturn("{\"0\":1, \"1\":10}"); + expect(responseHolder.getResponse()).andReturn(response).times(2); + expect(response.headers()).andReturn(headers).times(2); + expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2); + + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(3); + + replayAll(); + + Map results = client.getCurrentOffsets(TEST_ID, true); + verifyAll(); + + Assert.assertEquals(3, captured.getValues().size()); + for (Request request : captured.getValues()) { + Assert.assertEquals(HttpMethod.GET, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"), + request.getUrl() + ); + Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + } + + Assert.assertEquals(2, results.size()); + Assert.assertEquals("1", results.get("0")); + Assert.assertEquals("10", results.get("1")); + } + + @Test + public void testGetCurrentOffsetsWithExhaustedRetries() + { + expectedException.expect(RuntimeException.class); + expectedException.expectMessage("org.apache.druid.java.util.common.IOE: Received status [404]"); + + client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); + + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); + expect(responseHolder.getContent()).andReturn("").anyTimes(); + expect(responseHolder.getResponse()).andReturn(response).anyTimes(); + expect(response.headers()).andReturn(headers).anyTimes(); + expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).anyTimes(); + + expect( + httpClient.go( + anyObject(Request.class), + anyObject(FullResponseHandler.class), + eq(TEST_HTTP_TIMEOUT) + ) + ).andReturn(Futures.immediateFuture(responseHolder)).anyTimes(); + replayAll(); + + client.getCurrentOffsets(TEST_ID, true); + verifyAll(); + } + + @Test + public void testGetEndOffsets() throws Exception + { + Capture captured = Capture.newInstance(); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ); + replayAll(); + + Map results = client.getEndOffsets(TEST_ID); + verifyAll(); + + Request request = captured.getValue(); + Assert.assertEquals(HttpMethod.GET, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end"), + request.getUrl() + ); + Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + + Assert.assertEquals(2, results.size()); + Assert.assertEquals("1", results.get("0")); + Assert.assertEquals("10", results.get("1")); + } + + @Test + public void testGetStartTime() throws Exception + { + client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); + DateTime now = DateTimes.nowUtc(); + + Capture captured = Capture.newInstance(); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) + .andReturn(HttpResponseStatus.OK); + expect(responseHolder.getResponse()).andReturn(response); + expect(response.headers()).andReturn(headers); + expect(headers.get("X-Druid-Task-Id")).andReturn(null); + expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(2); + replayAll(); + + DateTime results = client.getStartTime(TEST_ID); + verifyAll(); + + Request request = captured.getValue(); + Assert.assertEquals(HttpMethod.GET, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/time/start"), + request.getUrl() + ); + Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + + Assert.assertEquals(now, results); + } + + @Test + public void testGetStatus() throws Exception + { + SeekableStreamIndexTask.Status status = SeekableStreamIndexTask.Status.READING; + + Capture captured = Capture.newInstance(); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + expect(responseHolder.getContent()).andReturn(StringUtils.format("\"%s\"", status.toString())).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ); + replayAll(); + + SeekableStreamIndexTask.Status results = client.getStatus(TEST_ID); + verifyAll(); + + Request request = captured.getValue(); + Assert.assertEquals(HttpMethod.GET, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/status"), + request.getUrl() + ); + Assert.assertTrue(null, request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + + Assert.assertEquals(status, results); + } + + @Test + public void testPause() throws Exception + { + Capture captured = Capture.newInstance(); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); + expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ); + replayAll(); + + Map results = client.pause(TEST_ID); + verifyAll(); + + Request request = captured.getValue(); + Assert.assertEquals(HttpMethod.POST, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/pause"), + request.getUrl() + ); + Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + + Assert.assertEquals(2, results.size()); + Assert.assertEquals("1", results.get("0")); + Assert.assertEquals("10", results.get("1")); + } + + @Test + public void testPauseWithSubsequentGetOffsets() throws Exception + { + Capture captured = Capture.newInstance(); + Capture captured2 = Capture.newInstance(); + Capture captured3 = Capture.newInstance(); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.ACCEPTED).times(2) + .andReturn(HttpResponseStatus.OK).times(2); + expect(responseHolder.getContent()).andReturn("\"PAUSED\"") + .andReturn("{\"0\":1, \"1\":10}").anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ); + expect(httpClient.go(capture(captured2), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ); + expect(httpClient.go(capture(captured3), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ); + + replayAll(); + + Map results = client.pause(TEST_ID); + verifyAll(); + + Request request = captured.getValue(); + Assert.assertEquals(HttpMethod.POST, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/pause"), + request.getUrl() + ); + Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + + request = captured2.getValue(); + Assert.assertEquals(HttpMethod.GET, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/status"), + request.getUrl() + ); + + request = captured3.getValue(); + Assert.assertEquals(HttpMethod.GET, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/current"), + request.getUrl() + ); + + Assert.assertEquals(2, results.size()); + Assert.assertEquals("1", results.get("0")); + Assert.assertEquals("10", results.get("1")); + } + + @Test + public void testResume() throws Exception + { + Capture captured = Capture.newInstance(); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ); + replayAll(); + + client.resume(TEST_ID); + verifyAll(); + + Request request = captured.getValue(); + Assert.assertEquals(HttpMethod.POST, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/resume"), + request.getUrl() + ); + Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + } + + @Test + public void testSetEndOffsets() throws Exception + { + Map endOffsets = ImmutableMap.of("0", "15", "1", "120"); + + Capture captured = Capture.newInstance(); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ); + replayAll(); + + client.setEndOffsets(TEST_ID, endOffsets, true); + verifyAll(); + + Request request = captured.getValue(); + Assert.assertEquals(HttpMethod.POST, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end?finish=true"), + request.getUrl() + ); + Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + Assert.assertEquals("{\"0\":\"15\",\"1\":\"120\"}", StringUtils.fromUtf8(request.getContent().array())); + } + + @Test + public void testSetEndOffsetsAndResume() throws Exception + { + Map endOffsets = ImmutableMap.of("0", "15", "1", "120"); + + Capture captured = Capture.newInstance(); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ); + replayAll(); + + client.setEndOffsets(TEST_ID, endOffsets, true); + verifyAll(); + + Request request = captured.getValue(); + Assert.assertEquals(HttpMethod.POST, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/offsets/end?finish=true"), + request.getUrl() + ); + Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + Assert.assertEquals("{\"0\":\"15\",\"1\":\"120\"}", StringUtils.fromUtf8(request.getContent().array())); + } + + @Test + public void testStop() throws Exception + { + Capture captured = Capture.newInstance(); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ); + replayAll(); + + client.stop(TEST_ID, false); + verifyAll(); + + Request request = captured.getValue(); + Assert.assertEquals(HttpMethod.POST, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/stop"), + request.getUrl() + ); + Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + } + + @Test + public void testStopAndPublish() throws Exception + { + Capture captured = Capture.newInstance(); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ); + replayAll(); + + client.stop(TEST_ID, true); + verifyAll(); + + Request request = captured.getValue(); + Assert.assertEquals(HttpMethod.POST, request.getMethod()); + Assert.assertEquals( + new URL("http://test-host:1234/druid/worker/v1/chat/test-id/stop?publish=true"), + request.getUrl() + ); + Assert.assertTrue(request.getHeaders().get("X-Druid-Task-Id").contains("test-id")); + } + + @Test + public void testStopAsync() throws Exception + { + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + for (String testId : TEST_IDS) { + expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "stop"))); + futures.add(client.stopAsync(testId, false)); + } + + List responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertTrue(responses.get(i)); + } + } + + @Test + public void testResumeAsync() throws Exception + { + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + for (String testId : TEST_IDS) { + expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "resume"))); + futures.add(client.resumeAsync(testId)); + } + + List responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertTrue(responses.get(i)); + } + } + + @Test + public void testPauseAsync() throws Exception + { + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List>> futures = Lists.newArrayList(); + for (String testId : TEST_IDS) { + expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "pause"))); + futures.add(client.pauseAsync(testId)); + } + + List> responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of("0", "1")), responses.get(i)); + } + } + + @Test + public void testGetStatusAsync() throws Exception + { + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + for (String testId : TEST_IDS) { + expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "status"))); + futures.add(client.getStatusAsync(testId)); + } + + List responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertEquals(SeekableStreamIndexTask.Status.READING, responses.get(i)); + } + } + + @Test + public void testGetStartTimeAsync() throws Exception + { + final DateTime now = DateTimes.nowUtc(); + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + for (String testId : TEST_IDS) { + expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "time/start"))); + futures.add(client.getStartTimeAsync(testId)); + } + + List responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertEquals(now, responses.get(i)); + } + } + + @Test + public void testGetCurrentOffsetsAsync() throws Exception + { + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List>> futures = Lists.newArrayList(); + for (String testId : TEST_IDS) { + expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "offsets/current"))); + futures.add(client.getCurrentOffsetsAsync(testId, false)); + } + + List> responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of("0", "1")), responses.get(i)); + } + } + + @Test + public void testGetEndOffsetsAsync() throws Exception + { + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List>> futures = Lists.newArrayList(); + for (String testId : TEST_IDS) { + expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "offsets/end"))); + futures.add(client.getEndOffsetsAsync(testId)); + } + + List> responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertEquals(Maps.newLinkedHashMap(ImmutableMap.of("0", "1")), responses.get(i)); + } + } + + @Test + public void testSetEndOffsetsAsync() throws Exception + { + Map endOffsets = ImmutableMap.of("0", "15L", "1", "120L"); + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + for (String testId : TEST_IDS) { + expectedUrls.add(new URL(StringUtils.format( + URL_FORMATTER, + TEST_HOST, + TEST_PORT, + testId, + StringUtils.format("offsets/end?finish=%s", true) + ))); + futures.add(client.setEndOffsetsAsync(testId, endOffsets, true)); + } + + List responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertTrue(responses.get(i)); + } + } + + @Test + public void testSetEndOffsetsAsyncWithResume() throws Exception + { + Map endOffsets = ImmutableMap.of("0", "15L", "1", "120L"); + final int numRequests = TEST_IDS.size(); + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + Futures.immediateFuture(responseHolder) + ).times(numRequests); + replayAll(); + + List expectedUrls = Lists.newArrayList(); + List> futures = Lists.newArrayList(); + for (String testId : TEST_IDS) { + expectedUrls.add( + new URL( + StringUtils.format( + URL_FORMATTER, + TEST_HOST, + TEST_PORT, + testId, + "offsets/end?finish=true" + ) + ) + ); + futures.add(client.setEndOffsetsAsync(testId, endOffsets, true)); + } + + List responses = Futures.allAsList(futures).get(); + + verifyAll(); + List requests = captured.getValues(); + + Assert.assertEquals(numRequests, requests.size()); + Assert.assertEquals(numRequests, responses.size()); + for (int i = 0; i < numRequests; i++) { + Assert.assertEquals(HttpMethod.POST, requests.get(i).getMethod()); + Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); + Assert.assertTrue(responses.get(i)); + } + } + + private class TestableKafkaIndexTaskClient extends KinesisIndexTaskClient + { + public TestableKafkaIndexTaskClient( + HttpClient httpClient, + ObjectMapper jsonMapper, + TaskInfoProvider taskInfoProvider + ) + { + this(httpClient, jsonMapper, taskInfoProvider, TEST_NUM_RETRIES); + } + + public TestableKafkaIndexTaskClient( + HttpClient httpClient, + ObjectMapper jsonMapper, + TaskInfoProvider taskInfoProvider, + long numRetries + ) + { + super(httpClient, jsonMapper, taskInfoProvider, TEST_DATASOURCE, numThreads, TEST_HTTP_TIMEOUT, numRetries); + } + + @Override + protected void checkConnection(String host, int port) + { + } + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 065a64998cf1..74f708c40007 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -1,18 +1,18 @@ ///* -// * Licensed to Metamarkets Group Inc. (Metamarkets) under one -// * or more contributor license agreements. See the NOTICE file +// * 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. Metamarkets licenses this file +// * 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 +// * with the License. You may obtain a copy of the License at // * -// * http://www.apache.org/licenses/LICENSE-2.0 +// * 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 +// * KIND, either express or implied. See the License for the // * specific language governing permissions and limitations // * under the License. // */ @@ -22,8 +22,6 @@ //import com.fasterxml.jackson.core.type.TypeReference; //import com.fasterxml.jackson.databind.Module; //import com.fasterxml.jackson.databind.ObjectMapper; -//import com.google.common.base.Charsets; -//import com.google.common.base.Function; //import com.google.common.base.Predicate; //import com.google.common.base.Predicates; //import com.google.common.base.Throwables; @@ -33,27 +31,33 @@ //import com.google.common.collect.ImmutableSet; //import com.google.common.collect.Iterables; //import com.google.common.collect.Lists; +//import com.google.common.collect.Sets; //import com.google.common.io.Files; //import com.google.common.util.concurrent.ListenableFuture; //import com.google.common.util.concurrent.ListeningExecutorService; //import com.google.common.util.concurrent.MoreExecutors; -//import com.metamx.common.logger.Logger; -//import com.metamx.emitter.EmittingLogger; -//import com.metamx.emitter.core.LoggingEmitter; -//import com.metamx.emitter.service.ServiceEmitter; -//import com.metamx.metrics.MonitorScheduler; +//import org.apache.curator.test.TestingCluster; //import org.apache.druid.client.cache.CacheConfig; +//import org.apache.druid.client.cache.CachePopulatorStats; //import org.apache.druid.client.cache.MapCache; -//import org.apache.druid.concurrent.Execs; //import org.apache.druid.data.input.impl.DimensionsSpec; +//import org.apache.druid.data.input.impl.FloatDimensionSchema; //import org.apache.druid.data.input.impl.JSONParseSpec; -//import org.apache.druid.data.input.impl.JSONPathFieldSpec; -//import org.apache.druid.data.input.impl.JSONPathSpec; +//import org.apache.druid.data.input.impl.LongDimensionSchema; +//import org.apache.druid.data.input.impl.StringDimensionSchema; //import org.apache.druid.data.input.impl.StringInputRowParser; //import org.apache.druid.data.input.impl.TimestampSpec; +//import org.apache.druid.discovery.DataNodeService; +//import org.apache.druid.discovery.DruidNodeAnnouncer; +//import org.apache.druid.discovery.LookupNodeService; +//import org.apache.druid.indexer.TaskState; +//import org.apache.druid.indexer.TaskStatus; +//import org.apache.druid.indexing.common.Counters; +//import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; //import org.apache.druid.indexing.common.SegmentLoaderFactory; //import org.apache.druid.indexing.common.TaskLock; -//import org.apache.druid.indexing.common.TaskStatus; +//import org.apache.druid.indexing.common.TaskReport; +//import org.apache.druid.indexing.common.TaskReportFileWriter; //import org.apache.druid.indexing.common.TaskToolbox; //import org.apache.druid.indexing.common.TaskToolboxFactory; //import org.apache.druid.indexing.common.TestUtils; @@ -62,45 +66,61 @@ //import org.apache.druid.indexing.common.actions.TaskActionToolbox; //import org.apache.druid.indexing.common.config.TaskConfig; //import org.apache.druid.indexing.common.config.TaskStorageConfig; +//import org.apache.druid.indexing.common.stats.RowIngestionMeters; +//import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +//import org.apache.druid.indexing.common.task.IndexTaskTest; //import org.apache.druid.indexing.common.task.Task; -//import org.apache.druid.indexing.kinesis.test.TestBroker; +//import org.apache.druid.indexing.overlord.DataSourceMetadata; //import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; //import org.apache.druid.indexing.overlord.MetadataTaskStorage; //import org.apache.druid.indexing.overlord.TaskLockbox; //import org.apache.druid.indexing.overlord.TaskStorage; //import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +//import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +//import org.apache.druid.indexing.seekablestream.test.TestBroker; //import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; //import org.apache.druid.indexing.test.TestDataSegmentKiller; -//import org.apache.druid.jackson.DefaultObjectMapper; //import org.apache.druid.java.util.common.CompressionUtils; +//import org.apache.druid.java.util.common.DateTimes; //import org.apache.druid.java.util.common.ISE; +//import org.apache.druid.java.util.common.Intervals; +//import org.apache.druid.java.util.common.StringUtils; +//import org.apache.druid.java.util.common.concurrent.Execs; //import org.apache.druid.java.util.common.granularity.Granularities; -//import org.apache.druid.java.util.common.guava.Sequences; +//import org.apache.druid.java.util.common.logger.Logger; +//import org.apache.druid.java.util.common.parsers.JSONPathSpec; +//import org.apache.druid.java.util.emitter.EmittingLogger; +//import org.apache.druid.java.util.emitter.core.NoopEmitter; +//import org.apache.druid.java.util.emitter.service.ServiceEmitter; +//import org.apache.druid.java.util.metrics.MonitorScheduler; +//import org.apache.druid.math.expr.ExprMacroTable; +//import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; //import org.apache.druid.metadata.EntryExistsException; //import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; -//import org.apache.druid.metadata.SQLMetadataStorageActionHandlerFactory; //import org.apache.druid.metadata.TestDerbyConnector; //import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; //import org.apache.druid.query.Druids; //import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; //import org.apache.druid.query.Query; //import org.apache.druid.query.QueryRunner; -//import org.apache.druid.query.QueryRunnerFactory; //import org.apache.druid.query.QueryRunnerFactoryConglomerate; //import org.apache.druid.query.QueryToolChest; -//import org.apache.druid.query.QueryWatcher; //import org.apache.druid.query.Result; //import org.apache.druid.query.SegmentDescriptor; //import org.apache.druid.query.aggregation.AggregatorFactory; //import org.apache.druid.query.aggregation.CountAggregatorFactory; +//import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; //import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +//import org.apache.druid.query.filter.SelectorDimFilter; //import org.apache.druid.query.timeseries.TimeseriesQuery; //import org.apache.druid.query.timeseries.TimeseriesQueryEngine; //import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; //import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; //import org.apache.druid.query.timeseries.TimeseriesResultValue; +//import org.apache.druid.segment.DimensionHandlerUtils; //import org.apache.druid.segment.IndexIO; //import org.apache.druid.segment.QueryableIndex; +//import org.apache.druid.segment.TestHelper; //import org.apache.druid.segment.column.DictionaryEncodedColumn; //import org.apache.druid.segment.indexing.DataSchema; //import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -110,110 +130,161 @@ //import org.apache.druid.segment.loading.SegmentLoaderConfig; //import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager; //import org.apache.druid.segment.loading.StorageLocationConfig; +//import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; //import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; //import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +//import org.apache.druid.segment.transform.ExpressionTransform; +//import org.apache.druid.segment.transform.TransformSpec; +//import org.apache.druid.server.DruidNode; //import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; +//import org.apache.druid.server.coordination.ServerType; //import org.apache.druid.timeline.DataSegment; -//import org.apache.curator.test.TestingCluster; //import org.apache.kafka.clients.producer.KafkaProducer; //import org.apache.kafka.clients.producer.ProducerRecord; //import org.easymock.EasyMock; -//import org.joda.time.DateTime; //import org.joda.time.Interval; //import org.joda.time.Period; //import org.junit.After; +//import org.junit.AfterClass; //import org.junit.Assert; //import org.junit.Before; +//import org.junit.BeforeClass; //import org.junit.Rule; //import org.junit.Test; //import org.junit.rules.TemporaryFolder; //import org.junit.runner.RunWith; //import org.junit.runners.Parameterized; // +//import javax.annotation.Nullable; //import java.io.File; //import java.io.IOException; -//import java.util.ArrayList; +//import java.lang.reflect.InvocationTargetException; +//import java.lang.reflect.Method; +//import java.nio.charset.StandardCharsets; +//import java.util.Arrays; +//import java.util.HashMap; //import java.util.List; //import java.util.Map; +//import java.util.Objects; //import java.util.Set; -//import java.util.concurrent.Callable; +//import java.util.TreeMap; //import java.util.concurrent.Executor; //import java.util.concurrent.Executors; //import java.util.concurrent.TimeUnit; //import java.util.concurrent.TimeoutException; // +//import static org.apache.druid.query.QueryPlus.wrap; +// //@RunWith(Parameterized.class) //public class KinesisIndexTaskTest //{ -// private final boolean buildV9Directly; +// private static final Logger log = new Logger(KinesisIndexTask.class); +// private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); +// private static final long POLL_RETRY_MS = 100; +// +// private static TestingCluster zkServer; +// private static TestBroker kafkaServer; +// private static ServiceEmitter emitter; +// private static ListeningExecutorService taskExec; +// private static int topicPostfix; +// +// private final List runningTasks = Lists.newArrayList(); +// // private long handoffConditionTimeout = 0; // private boolean reportParseExceptions = false; +// private boolean logParseExceptions = true; +// private Integer maxParseExceptions = null; +// private Integer maxSavedParseExceptions = null; +// private boolean resetOffsetAutomatically = false; // private boolean doHandoff = true; +// private Integer maxRowsPerSegment = null; +// private Long maxTotalRows = null; +// private Period intermediateHandoffPeriod = null; // -// private TestingCluster zkServer; -// private TestBroker kafkaServer; -// private ServiceEmitter emitter; -// private ListeningExecutorService taskExec; // private TaskToolboxFactory toolboxFactory; // private IndexerMetadataStorageCoordinator metadataStorageCoordinator; // private TaskStorage taskStorage; // private TaskLockbox taskLockbox; // private File directory; +// private String topic; +// private List> records; +// private final boolean isIncrementalHandoffSupported; +// private final Set checkpointRequestsHash = Sets.newHashSet(); +// private File reportsFile; +// private RowIngestionMetersFactory rowIngestionMetersFactory; +// +// private int handoffCount = 0; +// +// // This should be removed in versions greater that 0.12.x +// // isIncrementalHandoffSupported should always be set to true in those later versions +// @Parameterized.Parameters(name = "isIncrementalHandoffSupported = {0}") +// public static Iterable constructorFeeder() +// { +// return ImmutableList.of(new Object[]{true}, new Object[]{false}); +// } // -// private final List runningTasks = Lists.newArrayList(); +// public KafkaIndexTaskTest(boolean isIncrementalHandoffSupported) +// { +// this.isIncrementalHandoffSupported = isIncrementalHandoffSupported; +// } // -// private static final Logger log = new Logger(KinesisIndexTaskTest.class); -// private static final ObjectMapper objectMapper = new DefaultObjectMapper(); -// -// private static final DataSchema DATA_SCHEMA; -// -// private static final List> RECORDS = ImmutableList.of( -// new ProducerRecord("topic0", 0, null, JB("2008", "a", "y", 1.0f)), -// new ProducerRecord("topic0", 0, null, JB("2009", "b", "y", 1.0f)), -// new ProducerRecord("topic0", 0, null, JB("2010", "c", "y", 1.0f)), -// new ProducerRecord("topic0", 0, null, JB("2011", "d", "y", 1.0f)), -// new ProducerRecord("topic0", 0, null, JB("2011", "e", "y", 1.0f)), -// new ProducerRecord("topic0", 0, null, "unparseable".getBytes()), -// new ProducerRecord("topic0", 0, null, null), -// new ProducerRecord("topic0", 0, null, JB("2013", "f", "y", 1.0f)), -// new ProducerRecord("topic0", 1, null, JB("2012", "g", "y", 1.0f)), -// new ProducerRecord("topic0", 1, null, JB("2011", "h", "y", 1.0f)) +// private static final DataSchema DATA_SCHEMA = new DataSchema( +// "test_ds", +// objectMapper.convertValue( +// new StringInputRowParser( +// new JSONParseSpec( +// new TimestampSpec("timestamp", "iso", null), +// new DimensionsSpec( +// Arrays.asList( +// new StringDimensionSchema("dim1"), +// new StringDimensionSchema("dim1t"), +// new StringDimensionSchema("dim2"), +// new LongDimensionSchema("dimLong"), +// new FloatDimensionSchema("dimFloat") +// ), +// null, +// null +// ), +// new JSONPathSpec(true, ImmutableList.of()), +// ImmutableMap.of() +// ), +// StandardCharsets.UTF_8.name() +// ), +// Map.class +// ), +// new AggregatorFactory[]{ +// new DoubleSumAggregatorFactory("met1sum", "met1"), +// new CountAggregatorFactory("rows") +// }, +// new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), +// null, +// objectMapper // ); // -// static { -// DATA_SCHEMA = new DataSchema( -// "test_ds", -// objectMapper.convertValue( -// new StringInputRowParser( -// new JSONParseSpec( -// new TimestampSpec("timestamp", "iso", null), -// new DimensionsSpec( -// DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2")), -// null, -// null -// ), -// new JSONPathSpec(true, ImmutableList.of()), -// ImmutableMap.of() -// ), -// Charsets.UTF_8.name() -// ), -// Map.class -// ), -// new AggregatorFactory[]{new CountAggregatorFactory("rows")}, -// new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), -// objectMapper -// ); -// } -// -// @Parameterized.Parameters(name = "buildV9Directly = {0}") -// public static Iterable constructorFeeder() +// private static List> generateRecords(String topic) // { -// return ImmutableList.of(new Object[]{true}, new Object[]{false}); +// return ImmutableList.of( +// new ProducerRecord<>(topic, 0, null, JB("2008", "a", "y", "10", "20.0", "1.0")), +// new ProducerRecord<>(topic, 0, null, JB("2009", "b", "y", "10", "20.0", "1.0")), +// new ProducerRecord<>(topic, 0, null, JB("2010", "c", "y", "10", "20.0", "1.0")), +// new ProducerRecord<>(topic, 0, null, JB("2011", "d", "y", "10", "20.0", "1.0")), +// new ProducerRecord<>(topic, 0, null, JB("2011", "e", "y", "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<>(topic, 0, null, StringUtils.toUtf8("unparseable")), +// new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")), +// new ProducerRecord<>(topic, 0, null, null), +// new ProducerRecord<>(topic, 0, null, JB("2013", "f", "y", "10", "20.0", "1.0")), +// new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "notanumber", "20.0", "1.0")), +// new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "10", "notanumber", "1.0")), +// new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "10", "20.0", "notanumber")), +// new ProducerRecord<>(topic, 1, null, JB("2012", "g", "y", "10", "20.0", "1.0")), +// new ProducerRecord<>(topic, 1, null, JB("2011", "h", "y", "10", "20.0", "1.0")) +// ); // } // -// public KinesisIndexTaskTest(boolean buildV9Directly) +// private static String getTopicName() // { -// this.buildV9Directly = buildV9Directly; +// return "topic" + topicPostfix++; // } // // @Rule @@ -222,29 +293,23 @@ // @Rule // public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); // -// @Before -// public void setUp() throws Exception +// @BeforeClass +// public static void setupClass() throws Exception // { // emitter = new ServiceEmitter( // "service", // "host", -// new LoggingEmitter( -// log, -// LoggingEmitter.Level.ERROR, -// new DefaultObjectMapper() -// ) +// new NoopEmitter() // ); // emitter.start(); // EmittingLogger.registerEmitter(emitter); // -// makeToolboxFactory(); -// // zkServer = new TestingCluster(1); // zkServer.start(); // // kafkaServer = new TestBroker( // zkServer.getConnectString(), -// tempFolder.newFolder(), +// null, // 1, // ImmutableMap.of("num.partitions", "2") // ); @@ -252,20 +317,29 @@ // // taskExec = MoreExecutors.listeningDecorator( // Executors.newCachedThreadPool( -// Execs.makeThreadFactory("kinesis-task-test-%d") +// Execs.makeThreadFactory("kafka-task-test-%d") // ) // ); +// } // +// @Before +// public void setupTest() throws IOException +// { // handoffConditionTimeout = 0; // reportParseExceptions = false; +// logParseExceptions = true; +// maxParseExceptions = null; +// maxSavedParseExceptions = null; // doHandoff = true; +// topic = getTopicName(); +// records = generateRecords(topic); +// reportsFile = File.createTempFile("KafkaIndexTaskTestReports-" + System.currentTimeMillis(), "json"); +// makeToolboxFactory(); // } // // @After -// public void tearDown() throws Exception +// public void tearDownTest() // { -// emitter.close(); -// // synchronized (runningTasks) { // for (Task task : runningTasks) { // task.stopGracefully(); @@ -273,7 +347,13 @@ // // runningTasks.clear(); // } +// reportsFile.delete(); +// destroyToolboxFactory(); +// } // +// @AfterClass +// public static void tearDownClass() throws Exception +// { // taskExec.shutdown(); // taskExec.awaitTermination(9999, TimeUnit.DAYS); // @@ -283,7 +363,7 @@ // zkServer.stop(); // zkServer = null; // -// destroyToolboxFactory(); +// emitter.close(); // } // // @Test(timeout = 60_000L) @@ -291,196 +371,600 @@ // { // // Insert data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { +// for (ProducerRecord record : records) { // kafkaProducer.send(record).get(); // } // } // -// final KinesisIndexTask task = createTask( +// final KafkaIndexTask task = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); // // final ListenableFuture future = runTask(task); // // // Wait for task to exit -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // // // Check metrics -// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published metadata // SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // // // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); // } // // @Test(timeout = 60_000L) // public void testRunBeforeDataInserted() throws Exception // { -// final KinesisIndexTask task = createTask( +// final KafkaIndexTask task = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); // // final ListenableFuture future = runTask(task); // // // Wait for the task to start reading -// while (task.getStatus() != KinesisIndexTask.Status.READING) { +// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { // Thread.sleep(10); // } // // // Insert data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { +// for (ProducerRecord record : records) { // kafkaProducer.send(record).get(); // } // } // // // Wait for task to exit -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // // // Check metrics -// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published metadata // SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // // // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); +// } +// +// @Test(timeout = 60_000L) +// public void testIncrementalHandOff() throws Exception +// { +// if (!isIncrementalHandoffSupported) { +// return; +// } +// final String baseSequenceName = "sequence0"; +// // as soon as any segment has more than one record, incremental publishing should happen +// maxRowsPerSegment = 2; +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : records) { +// kafkaProducer.send(record).get(); +// } +// } +// Map consumerProps = kafkaServer.consumerProperties(); +// consumerProps.put("max.poll.records", "1"); +// +// final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L)); +// // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering +// // of events fetched across two partitions from Kafka +// final KafkaPartitions checkpoint1 = new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 0L)); +// final KafkaPartitions checkpoint2 = new KafkaPartitions(topic, ImmutableMap.of(0, 4L, 1, 2L)); +// final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L)); +// final KafkaIndexTask task = createTask( +// null, +// new KafkaIOConfig( +// 0, +// baseSequenceName, +// startPartitions, +// endPartitions, +// consumerProps, +// true, +// null, +// null, +// false +// ) +// ); +// final ListenableFuture future = runTask(task); +// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { +// Thread.sleep(10); +// } +// final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); +// Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets) || checkpoint2.getPartitionOffsetMap() +// .equals(currentOffsets)); +// task.getRunner().setEndOffsets(currentOffsets, false); +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); +// +// Assert.assertEquals(1, checkpointRequestsHash.size()); +// Assert.assertTrue( +// checkpointRequestsHash.contains( +// Objects.hash( +// DATA_SCHEMA.getDataSource(), +// 0, +// new KafkaDataSourceMetadata(startPartitions), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets)) +// ) +// ) +// ); +// +// // Check metrics +// Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); +// SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); +// SegmentDescriptor desc4 = SD(task, "2011/P1D", 0); +// SegmentDescriptor desc5 = SD(task, "2011/P1D", 1); +// SegmentDescriptor desc6 = SD(task, "2012/P1D", 0); +// SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); +// Assert.assertEquals( +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); +// Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4)) +// && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) || +// (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4)) +// && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5)))); +// Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6)); +// Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); +// } +// +// @Test(timeout = 60_000L) +// public void testIncrementalHandOffMaxTotalRows() throws Exception +// { +// if (!isIncrementalHandoffSupported) { +// return; +// } +// final String baseSequenceName = "sequence0"; +// // incremental publish should happen every 3 records +// maxRowsPerSegment = Integer.MAX_VALUE; +// maxTotalRows = 3L; +// +// // Insert data +// int numToAdd = records.size() - 2; +// +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (int i = 0; i < numToAdd; i++) { +// kafkaProducer.send(records.get(i)).get(); +// } +// +// Map consumerProps = kafkaServer.consumerProperties(); +// consumerProps.put("max.poll.records", "1"); +// +// final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L)); +// final KafkaPartitions checkpoint1 = new KafkaPartitions(topic, ImmutableMap.of(0, 3L, 1, 0L)); +// final KafkaPartitions checkpoint2 = new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 0L)); +// +// final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L)); +// final KafkaIndexTask task = createTask( +// null, +// new KafkaIOConfig( +// 0, +// baseSequenceName, +// startPartitions, +// endPartitions, +// consumerProps, +// true, +// null, +// null, +// false +// ) +// ); +// final ListenableFuture future = runTask(task); +// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { +// Thread.sleep(10); +// } +// final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); +// +// Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets)); +// task.getRunner().setEndOffsets(currentOffsets, false); +// +// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { +// Thread.sleep(10); +// } +// +// // add remaining records +// for (int i = numToAdd; i < records.size(); i++) { +// kafkaProducer.send(records.get(i)).get(); +// } +// final Map nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); +// +// Assert.assertTrue(checkpoint2.getPartitionOffsetMap().equals(nextOffsets)); +// task.getRunner().setEndOffsets(nextOffsets, false); +// +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); +// +// Assert.assertEquals(2, checkpointRequestsHash.size()); +// Assert.assertTrue( +// checkpointRequestsHash.contains( +// Objects.hash( +// DATA_SCHEMA.getDataSource(), +// 0, +// new KafkaDataSourceMetadata(startPartitions), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets)) +// ) +// ) +// ); +// Assert.assertTrue( +// checkpointRequestsHash.contains( +// Objects.hash( +// DATA_SCHEMA.getDataSource(), +// 0, +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets)), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, nextOffsets)) +// ) +// ) +// ); +// +// // Check metrics +// Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); +// SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); +// SegmentDescriptor desc4 = SD(task, "2011/P1D", 0); +// SegmentDescriptor desc5 = SD(task, "2011/P1D", 1); +// SegmentDescriptor desc6 = SD(task, "2012/P1D", 0); +// SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); +// Assert.assertEquals( +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); +// Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4)) +// && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) || +// (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4)) +// && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5)))); +// Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6)); +// Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); +// } +// } +// +// @Test(timeout = 60_000L) +// public void testTimeBasedIncrementalHandOff() throws Exception +// { +// if (!isIncrementalHandoffSupported) { +// return; +// } +// final String baseSequenceName = "sequence0"; +// // as soon as any segment hits maxRowsPerSegment or intermediateHandoffPeriod, incremental publishing should happen +// maxRowsPerSegment = Integer.MAX_VALUE; +// intermediateHandoffPeriod = new Period().withSeconds(0); +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : records.subList(0, 2)) { +// kafkaProducer.send(record).get(); +// } +// } +// Map consumerProps = kafkaServer.consumerProperties(); +// consumerProps.put("max.poll.records", "1"); +// +// final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L)); +// // Checkpointing will happen at checkpoint +// final KafkaPartitions checkpoint = new KafkaPartitions(topic, ImmutableMap.of(0, 1L, 1, 0L)); +// final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L)); +// final KafkaIndexTask task = createTask( +// null, +// new KafkaIOConfig( +// 0, +// baseSequenceName, +// startPartitions, +// endPartitions, +// consumerProps, +// true, +// null, +// null, +// false +// ) +// ); +// final ListenableFuture future = runTask(task); +// +// // task will pause for checkpointing +// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { +// Thread.sleep(10); +// } +// final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); +// Assert.assertTrue(checkpoint.getPartitionOffsetMap().equals(currentOffsets)); +// task.getRunner().setEndOffsets(currentOffsets, false); +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); +// +// Assert.assertEquals(1, checkpointRequestsHash.size()); +// Assert.assertTrue( +// checkpointRequestsHash.contains( +// Objects.hash( +// DATA_SCHEMA.getDataSource(), +// 0, +// new KafkaDataSourceMetadata(startPartitions), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, checkpoint.getPartitionOffsetMap())) +// ) +// ) +// ); +// +// // Check metrics +// Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); +// Assert.assertEquals( +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); // } // // @Test(timeout = 60_000L) // public void testRunWithMinimumMessageTime() throws Exception // { -// final KinesisIndexTask task = createTask( +// final KafkaIndexTask task = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 0L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, -// false, -// new DateTime("2010"), +// DateTimes.of("2010"), +// null, // false -// ), -// null, -// null +// ) // ); // // final ListenableFuture future = runTask(task); // // // Wait for the task to start reading -// while (task.getStatus() != KinesisIndexTask.Status.READING) { +// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { // Thread.sleep(10); // } // // // Insert data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { +// for (ProducerRecord record : records) { // kafkaProducer.send(record).get(); // } // } // // // Wait for task to exit -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // // // Check metrics -// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(2, task.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published metadata // SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // // // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); // } // // @Test(timeout = 60_000L) -// public void testRunOnNothing() throws Exception +// public void testRunWithMaximumMessageTime() throws Exception // { +// final KafkaIndexTask task = createTask( +// null, +// new KafkaIOConfig( +// 0, +// "sequence0", +// new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// null, +// DateTimes.of("2010"), +// false +// ) +// ); +// +// final ListenableFuture future = runTask(task); +// +// // Wait for the task to start reading +// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { +// Thread.sleep(10); +// } +// // // Insert data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { +// for (ProducerRecord record : records) { // kafkaProducer.send(record).get(); // } // } // -// final KinesisIndexTask task = createTask( +// // Wait for task to exit +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); +// SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); +// Assert.assertEquals( +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); +// } +// +// @Test(timeout = 60_000L) +// public void testRunWithTransformSpec() throws Exception +// { +// final KafkaIndexTask task = createTask( // null, -// new KinesisIOConfig( +// DATA_SCHEMA.withTransformSpec( +// new TransformSpec( +// new SelectorDimFilter("dim1", "b", null), +// ImmutableList.of( +// new ExpressionTransform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) +// ) +// ) +// ), +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), +// ) +// ); +// +// final ListenableFuture future = runTask(task); +// +// // Wait for the task to start reading +// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { +// Thread.sleep(10); +// } +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : records) { +// kafkaProducer.send(record).get(); +// } +// } +// +// // Wait for task to exit +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); +// +// // Check metrics +// Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2009/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors()); +// Assert.assertEquals( +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", desc1)); +// } +// +// @Test(timeout = 60_000L) +// public void testRunOnNothing() throws Exception +// { +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : records) { +// kafkaProducer.send(record).get(); +// } +// } +// +// final KafkaIndexTask task = createTask( // null, -// null +// new KafkaIOConfig( +// 0, +// "sequence0", +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// kafkaServer.consumerProperties(), +// true, +// null, +// null, +// false +// ) // ); // // final ListenableFuture future = runTask(task); // // // Wait for task to exit -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // // // Check metrics -// Assert.assertEquals(0, task.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published metadata // Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); @@ -493,49 +977,48 @@ // // // Insert data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { +// for (ProducerRecord record : records) { // kafkaProducer.send(record).get(); // } // } // -// final KinesisIndexTask task = createTask( +// final KafkaIndexTask task = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); // // final ListenableFuture future = runTask(task); // // // Wait for task to exit -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // // // Check metrics -// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published metadata // SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // // // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); // } // // @Test(timeout = 60_000L) @@ -546,49 +1029,48 @@ // // // Insert data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { +// for (ProducerRecord record : records) { // kafkaProducer.send(record).get(); // } // } // -// final KinesisIndexTask task = createTask( +// final KafkaIndexTask task = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); // // final ListenableFuture future = runTask(task); // // // Wait for task to exit -// Assert.assertEquals(TaskStatus.Status.FAILED, future.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // // // Check metrics -// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published metadata // SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // // // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); // } // // @Test(timeout = 60_000L) @@ -596,38 +1078,41 @@ // { // reportParseExceptions = true; // +// // these will be ignored because reportParseExceptions is true +// maxParseExceptions = 1000; +// maxSavedParseExceptions = 2; +// // // Insert data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { +// for (ProducerRecord record : records) { // kafkaProducer.send(record).get(); // } // } // -// final KinesisIndexTask task = createTask( +// final KafkaIndexTask task = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 7L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 7L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); // // final ListenableFuture future = runTask(task); // // // Wait for task to exit -// Assert.assertEquals(TaskStatus.Status.FAILED, future.get().getStatusCode()); +// Assert.assertEquals(TaskState.FAILED, future.get().getStatusCode()); // // // Check metrics -// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published metadata // Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); @@ -635,37 +1120,188 @@ // } // // @Test(timeout = 60_000L) -// public void testRunReplicas() throws Exception +// public void testMultipleParseExceptionsSuccess() throws Exception // { -// final KinesisIndexTask task1 = createTask( +// reportParseExceptions = false; +// maxParseExceptions = 6; +// maxSavedParseExceptions = 6; +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : records) { +// kafkaProducer.send(record).get(); +// } +// } +// +// final KafkaIndexTask task = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 13L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), +// ) +// ); +// +// final ListenableFuture future = runTask(task); +// +// TaskStatus status = future.get(); +// +// // Wait for task to exit +// Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); +// Assert.assertEquals(null, status.getErrorMsg()); +// +// // Check metrics +// Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessedWithError()); +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); +// SegmentDescriptor desc3 = SD(task, "2013/P1D", 0); +// SegmentDescriptor desc4 = SD(task, "2049/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); +// Assert.assertEquals( +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 13L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); +// +// Map expectedMetrics = ImmutableMap.of( +// RowIngestionMeters.BUILD_SEGMENTS, +// ImmutableMap.of( +// RowIngestionMeters.PROCESSED, 4, +// RowIngestionMeters.PROCESSED_WITH_ERROR, 3, +// RowIngestionMeters.UNPARSEABLE, 3, +// RowIngestionMeters.THROWN_AWAY, 1 +// ) +// ); +// Assert.assertEquals(expectedMetrics, reportData.getRowStats()); +// +// Map unparseableEvents = ImmutableMap.of( +// RowIngestionMeters.BUILD_SEGMENTS, +// Arrays.asList( +// "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=20.0, met1=notanumber}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [Unable to parse value[notanumber] for field[met1],]", +// "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=notanumber, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to float,]", +// "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=notanumber, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to long,]", +// "Unable to parse row [unparseable2]", +// "Unable to parse row [unparseable]", +// "Encountered row with timestamp that cannot be represented as a long: [MapBasedInputRow{timestamp=246140482-04-24T15:36:27.903Z, event={timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}]" +// ) +// ); +// +// Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); +// } +// +// @Test(timeout = 60_000L) +// public void testMultipleParseExceptionsFailure() throws Exception +// { +// reportParseExceptions = false; +// maxParseExceptions = 2; +// maxSavedParseExceptions = 2; +// +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : records) { +// kafkaProducer.send(record).get(); +// } +// } +// +// final KafkaIndexTask task = createTask( // null, -// null +// new KafkaIOConfig( +// 0, +// "sequence0", +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 10L)), +// kafkaServer.consumerProperties(), +// true, +// null, +// null, +// false +// ) +// ); +// +// final ListenableFuture future = runTask(task); +// +// TaskStatus status = future.get(); +// +// // Wait for task to exit +// Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); +// IndexTaskTest.checkTaskStatusErrorMsgForParseExceptionsExceeded(status); +// +// // Check metrics +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessedWithError()); +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); +// +// // Check published metadata +// Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); +// Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); +// +// IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); +// +// Map expectedMetrics = ImmutableMap.of( +// RowIngestionMeters.BUILD_SEGMENTS, +// ImmutableMap.of( +// RowIngestionMeters.PROCESSED, 3, +// RowIngestionMeters.PROCESSED_WITH_ERROR, 0, +// RowIngestionMeters.UNPARSEABLE, 3, +// RowIngestionMeters.THROWN_AWAY, 0 +// ) +// ); +// Assert.assertEquals(expectedMetrics, reportData.getRowStats()); +// +// Map unparseableEvents = ImmutableMap.of( +// RowIngestionMeters.BUILD_SEGMENTS, +// Arrays.asList( +// "Unable to parse row [unparseable2]", +// "Unable to parse row [unparseable]" +// ) // ); -// final KinesisIndexTask task2 = createTask( +// +// Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); +// } +// +// @Test(timeout = 60_000L) +// public void testRunReplicas() throws Exception +// { +// final KafkaIndexTask task1 = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), +// ) +// ); +// final KafkaIndexTask task2 = createTask( // null, -// null +// new KafkaIOConfig( +// 0, +// "sequence0", +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// kafkaServer.consumerProperties(), +// true, +// null, +// null, +// false +// ) // ); // // final ListenableFuture future1 = runTask(task1); @@ -673,152 +1309,148 @@ // // // Insert data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { +// for (ProducerRecord record : records) { // kafkaProducer.send(record).get(); // } // } // // // Wait for tasks to exit -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); // // // Check metrics -// Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); -// Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task2.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); +// Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published segments & metadata // SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); // SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // // // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); // } // // @Test(timeout = 60_000L) // public void testRunConflicting() throws Exception // { -// final KinesisIndexTask task1 = createTask( +// final KafkaIndexTask task1 = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); -// final KinesisIndexTask task2 = createTask( +// final KafkaIndexTask task2 = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 1, // "sequence1", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 3L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 8L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 3L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 10L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); // // // Insert data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { +// for (ProducerRecord record : records) { // kafkaProducer.send(record).get(); // } // } // // // Run first task // final ListenableFuture future1 = runTask(task1); -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); // // // Run second task // final ListenableFuture future2 = runTask(task2); -// Assert.assertEquals(TaskStatus.Status.FAILED, future2.get().getStatusCode()); +// Assert.assertEquals(TaskState.FAILED, future2.get().getStatusCode()); // // // Check metrics -// Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); -// Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(2, task2.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); +// Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published segments & metadata, should all be from the first task // SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); // SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // // // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); // } // // @Test(timeout = 60_000L) // public void testRunConflictingWithoutTransactions() throws Exception // { -// final KinesisIndexTask task1 = createTask( +// final KafkaIndexTask task1 = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // false, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); -// final KinesisIndexTask task2 = createTask( +// final KafkaIndexTask task2 = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 1, // "sequence1", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 3L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 8L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 3L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 10L)), // kafkaServer.consumerProperties(), // false, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); // // // Insert data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { +// for (ProducerRecord record : records) { // kafkaProducer.send(record).get(); // } // } // // // Run first task // final ListenableFuture future1 = runTask(task1); -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); // // // Check published segments & metadata // SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); @@ -828,15 +1460,15 @@ // // // Run second task // final ListenableFuture future2 = runTask(task2); -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); // // // Check metrics -// Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); -// Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(2, task2.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); +// Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published segments & metadata // SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1); @@ -845,102 +1477,107 @@ // Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); // // // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc3)); -// Assert.assertEquals(ImmutableList.of("f"), readSegmentDim1(desc4)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc3)); +// Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4)); // } // // @Test(timeout = 60_000L) // public void testRunOneTaskTwoPartitions() throws Exception // { -// final KinesisIndexTask task = createTask( +// final KafkaIndexTask task = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L, 1, 0L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L, 1, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 2L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); // // final ListenableFuture future = runTask(task); // // // Insert data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { +// for (ProducerRecord record : records) { // kafkaProducer.send(record).get(); // } +// kafkaProducer.flush(); // } // // // Wait for tasks to exit -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // // // Check metrics -// Assert.assertEquals(5, task.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(5, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published segments & metadata // SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); +// // desc3 will not be created in KafkaIndexTask (0.12.x) as it does not create per Kafka partition Druid segments // SegmentDescriptor desc3 = SD(task, "2011/P1D", 1); // SegmentDescriptor desc4 = SD(task, "2012/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); +// Assert.assertEquals(isIncrementalHandoffSupported +// ? ImmutableSet.of(desc1, desc2, desc4) +// : ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); // Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L, 1, 2L))), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 2L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // // // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); -// Assert.assertEquals(ImmutableList.of("g"), readSegmentDim1(desc4)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4)); // // // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically // Assert.assertEquals( -// ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), -// ImmutableSet.of(readSegmentDim1(desc2), readSegmentDim1(desc3)) +// isIncrementalHandoffSupported +// ? ImmutableSet.of(ImmutableList.of("d", "e", "h")) +// : ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), +// isIncrementalHandoffSupported +// ? ImmutableSet.of(readSegmentColumn("dim1", desc2)) +// : ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3)) // ); // } // // @Test(timeout = 60_000L) // public void testRunTwoTasksTwoPartitions() throws Exception // { -// final KinesisIndexTask task1 = createTask( +// final KafkaIndexTask task1 = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); -// final KinesisIndexTask task2 = createTask( +// final KafkaIndexTask task2 = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 1, // "sequence1", -// new KinesisPartitions("topic0", ImmutableMap.of(1, 0L)), -// new KinesisPartitions("topic0", ImmutableMap.of(1, 1L)), +// new KafkaPartitions(topic, ImmutableMap.of(1, 0L)), +// new KafkaPartitions(topic, ImmutableMap.of(1, 1L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); // // final ListenableFuture future1 = runTask(task1); @@ -948,22 +1585,22 @@ // // // Insert data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { +// for (ProducerRecord record : records) { // kafkaProducer.send(record).get(); // } // } // // // Wait for tasks to exit -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); // // // Check metrics -// Assert.assertEquals(3, task1.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); -// Assert.assertEquals(1, task2.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task2.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); +// Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published segments & metadata // SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); @@ -971,40 +1608,39 @@ // SegmentDescriptor desc3 = SD(task2, "2012/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); // Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L, 1, 1L))), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 1L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // // // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); -// Assert.assertEquals(ImmutableList.of("g"), readSegmentDim1(desc3)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); +// Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc3)); // } // // @Test(timeout = 60_000L) // public void testRestore() throws Exception // { -// final KinesisIndexTask task1 = createTask( +// final KafkaIndexTask task1 = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); // // final ListenableFuture future1 = runTask(task1); // // // Insert some data, but not enough for the task to finish // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : Iterables.limit(RECORDS, 4)) { +// for (ProducerRecord record : Iterables.limit(records, 4)) { // kafkaProducer.send(record).get(); // } // } @@ -1017,85 +1653,86 @@ // // // Stop without publishing segment // task1.stopGracefully(); -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future1.get().getStatusCode()); +// unlockAppenderatorBasePersistDirForTask(task1); +// +// Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); // // // Start a new task -// final KinesisIndexTask task2 = createTask( +// final KafkaIndexTask task2 = createTask( // task1.getId(), -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); // // final ListenableFuture future2 = runTask(task2); // // // Insert remaining data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : Iterables.skip(RECORDS, 4)) { +// for (ProducerRecord record : Iterables.skip(records, 4)) { // kafkaProducer.send(record).get(); // } // } // // // Wait for task to exit -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future2.get().getStatusCode()); +// Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); // // // Check metrics -// Assert.assertEquals(2, task1.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); -// Assert.assertEquals(1, task2.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task2.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(2, task1.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); +// Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published segments & metadata // SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); // SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // // // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); // } // // @Test(timeout = 60_000L) // public void testRunWithPauseAndResume() throws Exception // { -// final KinesisIndexTask task = createTask( +// final KafkaIndexTask task = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); // // final ListenableFuture future = runTask(task); // // // Insert some data, but not enough for the task to finish // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : Iterables.limit(RECORDS, 4)) { +// for (ProducerRecord record : Iterables.limit(records, 4)) { // kafkaProducer.send(record).get(); // } +// kafkaProducer.flush(); // } // // while (countEvents(task) != 2) { @@ -1103,19 +1740,19 @@ // } // // Assert.assertEquals(2, countEvents(task)); -// Assert.assertEquals(KinesisIndexTask.Status.READING, task.getStatus()); +// Assert.assertEquals(SeekableStreamIndexTask.Status.READING, task.getRunner().getStatus()); // // Map currentOffsets = objectMapper.readValue( -// task.pause(0).getEntity().toString(), +// task.getRunner().pause().getEntity().toString(), // new TypeReference>() // { // } // ); -// Assert.assertEquals(KinesisIndexTask.Status.PAUSED, task.getStatus()); +// Assert.assertEquals(SeekableStreamIndexTask.Status.PAUSED, task.getRunner().getStatus()); // // // Insert remaining data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : Iterables.skip(RECORDS, 4)) { +// for (ProducerRecord record : Iterables.skip(records, 4)) { // kafkaProducer.send(record).get(); // } // } @@ -1128,191 +1765,166 @@ // // carry on.. // } // -// Assert.assertEquals(currentOffsets, task.getCurrentOffsets()); +// Assert.assertEquals(currentOffsets, task.getRunner().getCurrentOffsets()); // -// task.resume(); +// task.getRunner().resume(); // -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); -// Assert.assertEquals(task.getEndOffsets(), task.getCurrentOffsets()); +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); +// Assert.assertEquals(task.getRunner().getEndOffsets(), task.getRunner().getCurrentOffsets()); // // // Check metrics -// Assert.assertEquals(3, task.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // // // Check published metadata // SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 5L))), +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // // // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); // } // // @Test(timeout = 60_000L) -// public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception +// public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception // { -// final KinesisIndexTask task = createTask( +// final KafkaIndexTask task = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 1L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 3L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, -// true, +// null, // null, // false -// ), -// null, -// null +// ) // ); // -// final ListenableFuture future = runTask(task); -// -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { -// kafkaProducer.send(record).get(); -// } -// } -// -// while (task.getStatus() != KinesisIndexTask.Status.PAUSED) { -// Thread.sleep(25); -// } -// -// // reached the end of the assigned offsets and paused instead of publishing -// Assert.assertEquals(task.getEndOffsets(), task.getCurrentOffsets()); -// Assert.assertEquals(KinesisIndexTask.Status.PAUSED, task.getStatus()); -// -// Assert.assertEquals(ImmutableMap.of(0, 3L), task.getEndOffsets()); -// Map newEndOffsets = ImmutableMap.of(0, 4L); -// task.setEndOffsets(newEndOffsets, false); -// Assert.assertEquals(newEndOffsets, task.getEndOffsets()); -// Assert.assertEquals(KinesisIndexTask.Status.PAUSED, task.getStatus()); -// task.resume(); +// runTask(task); // -// while (task.getStatus() != KinesisIndexTask.Status.PAUSED) { -// Thread.sleep(25); +// while (!task.getRunner().getStatus().equals(SeekableStreamIndexTask.Status.READING)) { +// Thread.sleep(2000); // } // -// // reached the end of the updated offsets and paused -// Assert.assertEquals(newEndOffsets, task.getCurrentOffsets()); -// Assert.assertEquals(KinesisIndexTask.Status.PAUSED, task.getStatus()); -// -// // try again but with resume flag == true -// newEndOffsets = ImmutableMap.of(0, 6L); -// task.setEndOffsets(newEndOffsets, true); -// Assert.assertEquals(newEndOffsets, task.getEndOffsets()); -// Assert.assertNotEquals(KinesisIndexTask.Status.PAUSED, task.getStatus()); +// task.getRunner().pause(); // -// while (task.getStatus() != KinesisIndexTask.Status.PAUSED) { +// while (!task.getRunner().getStatus().equals(SeekableStreamIndexTask.Status.PAUSED)) { // Thread.sleep(25); // } -// -// Assert.assertEquals(newEndOffsets, task.getCurrentOffsets()); -// Assert.assertEquals(KinesisIndexTask.Status.PAUSED, task.getStatus()); -// -// task.resume(); -// -// Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(4, task.getFireDepartmentMetrics().processed()); -// Assert.assertEquals(1, task.getFireDepartmentMetrics().unparseable()); -// Assert.assertEquals(0, task.getFireDepartmentMetrics().thrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2009/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2010/P1D", 0); -// SegmentDescriptor desc3 = SD(task, "2011/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions("topic0", ImmutableMap.of(0, 6L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("b"), readSegmentDim1(desc1)); -// Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc2)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc3)); // } // -// @Test(timeout = 30_000L) -// public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception +// @Test(timeout = 60_000L) +// public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAvailable() throws Exception // { -// final KinesisIndexTask task = createTask( +// resetOffsetAutomatically = true; +// // Insert data +// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { +// for (ProducerRecord record : records) { +// kafkaProducer.send(record).get(); +// } +// } +// +// final KafkaIndexTask task = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 2L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 5L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 200L)), +// new KafkaPartitions(topic, ImmutableMap.of(0, 500L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false -// ), -// null, -// null +// ) // ); // // runTask(task); // -// while (!task.getStatus().equals(KinesisIndexTask.Status.READING)) { -// Thread.sleep(2000); +// while (!task.getRunner().getStatus().equals(SeekableStreamIndexTask.Status.READING)) { +// Thread.sleep(20); // } // -// task.pause(0); -// -// while (!task.getStatus().equals(KinesisIndexTask.Status.PAUSED)) { -// Thread.sleep(25); +// for (int i = 0; i < 5; i++) { +// Assert.assertEquals(task.getRunner().getStatus(), SeekableStreamIndexTask.Status.READING); +// // Offset should not be reset +// Assert.assertTrue(task.getRunner().getCurrentOffsets().get(0) == 200L); // } // } // -// @Test(timeout = 30_000L) -// public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAvailable() throws Exception +// @Test(timeout = 60_000L) +// public void testRunContextSequenceAheadOfStartingOffsets() throws Exception // { +// // This tests the case when a replacement task is created in place of a failed test +// // which has done some incremental handoffs, thus the context will contain starting +// // sequence offsets from which the task should start reading and ignore the start offsets +// if (!isIncrementalHandoffSupported) { +// return; +// } // // Insert data // try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : RECORDS) { +// for (ProducerRecord record : records) { // kafkaProducer.send(record).get(); // } // } // -// final KinesisIndexTask task = createTask( +// final TreeMap> sequences = new TreeMap<>(); +// // Here the sequence number is 1 meaning that one incremental handoff was done by the failed task +// // and this task should start reading from offset 2 for partition 0 +// sequences.put(1, ImmutableMap.of(0, 2L)); +// final Map context = new HashMap<>(); +// context.put("checkpoints", objectMapper.writerWithType(new TypeReference>>() +// { +// }).writeValueAsString(sequences)); +// +// final KafkaIndexTask task = createTask( // null, -// new KinesisIOConfig( +// new KafkaIOConfig( +// 0, // "sequence0", -// new KinesisPartitions("topic0", ImmutableMap.of(0, 200L)), -// new KinesisPartitions("topic0", ImmutableMap.of(0, 500L)), +// // 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, 5L)), // kafkaServer.consumerProperties(), // true, -// false, +// null, // null, // false // ), -// null, -// true +// context // ); // -// runTask(task); +// final ListenableFuture future = runTask(task); // -// while (!task.getStatus().equals(KinesisIndexTask.Status.READING)) { -// Thread.sleep(2000); -// } +// // Wait for task to exit +// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // -// int i = 0; -// while(i++ < 5) { -// Assert.assertEquals(task.getStatus(), KinesisIndexTask.Status.READING); -// // Offset should not be reset -// Assert.assertTrue(task.getCurrentOffsets().get(0) == 200L); -// Thread.sleep(2000); -// } +// // Check metrics +// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); +// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); +// +// // Check published metadata +// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); +// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); +// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); +// Assert.assertEquals( +// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) +// ); +// +// // Check segments in deep storage +// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); +// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); // } // // private ListenableFuture runTask(final Task task) @@ -1329,23 +1941,18 @@ // runningTasks.add(task); // } // return taskExec.submit( -// new Callable() -// { -// @Override -// public TaskStatus call() throws Exception -// { -// try { -// if (task.isReady(toolbox.getTaskActionClient())) { -// return task.run(toolbox); -// } else { -// throw new ISE("Task is not ready"); -// } -// } -// catch (Exception e) { -// log.warn(e, "Task failed"); -// return TaskStatus.failure(task.getId()); +// () -> { +// try { +// if (task.isReady(toolbox.getTaskActionClient())) { +// return task.run(toolbox); +// } else { +// throw new ISE("Task is not ready"); // } // } +// catch (Exception e) { +// log.warn(e, "Task failed"); +// return TaskStatus.failure(task.getId()); +// } // } // ); // } @@ -1365,62 +1972,148 @@ // ); // } // -// private KinesisIndexTask createTask( +// private KafkaIndexTask createTask( +// final String taskId, +// final KafkaIOConfig ioConfig +// ) +// { +// return createTask(taskId, DATA_SCHEMA, ioConfig); +// } +// +// private KafkaIndexTask createTask( +// final String taskId, +// final KafkaIOConfig ioConfig, +// final Map context +// ) +// { +// return createTask(taskId, DATA_SCHEMA, ioConfig, context); +// } +// +// private KafkaIndexTask createTask( +// final String taskId, +// final DataSchema dataSchema, +// final KafkaIOConfig ioConfig +// ) +// { +// final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( +// 1000, +// null, +// maxRowsPerSegment, +// maxTotalRows, +// new Period("P1Y"), +// null, +// null, +// null, +// true, +// reportParseExceptions, +// handoffConditionTimeout, +// resetOffsetAutomatically, +// null, +// intermediateHandoffPeriod, +// logParseExceptions, +// maxParseExceptions, +// maxSavedParseExceptions +// ); +// final Map context = isIncrementalHandoffSupported +// ? ImmutableMap.of(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true) +// : null; +// final KafkaIndexTask task = new KafkaIndexTask( +// taskId, +// null, +// cloneDataSchema(dataSchema), +// tuningConfig, +// ioConfig, +// context, +// null, +// null, +// rowIngestionMetersFactory +// ); +// task.setPollRetryMs(POLL_RETRY_MS); +// return task; +// } +// +// +// private KafkaIndexTask createTask( // final String taskId, -// final KinesisIOConfig ioConfig, -// final Integer maxRowsPerSegment, -// final Boolean resetOffsetAutomatically +// final DataSchema dataSchema, +// final KafkaIOConfig ioConfig, +// final Map context // ) // { -// final KinesisTuningConfig tuningConfig = new KinesisTuningConfig( +// final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( // 1000, +// null, // maxRowsPerSegment, +// null, // new Period("P1Y"), // null, // null, // null, -// buildV9Directly, +// true, // reportParseExceptions, // handoffConditionTimeout, -// resetOffsetAutomatically +// resetOffsetAutomatically, +// null, +// null, +// logParseExceptions, +// maxParseExceptions, +// maxSavedParseExceptions // ); -// return new KinesisIndexTask( +// if (isIncrementalHandoffSupported) { +// context.put(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true); +// } +// +// final KafkaIndexTask task = new KafkaIndexTask( // taskId, // null, -// DATA_SCHEMA, +// cloneDataSchema(dataSchema), // tuningConfig, // ioConfig, +// context, // null, -// null +// null, +// rowIngestionMetersFactory +// ); +// task.setPollRetryMs(POLL_RETRY_MS); +// return task; +// } +// +// private static DataSchema cloneDataSchema(final DataSchema dataSchema) +// { +// return new DataSchema( +// dataSchema.getDataSource(), +// dataSchema.getParserMap(), +// dataSchema.getAggregators(), +// dataSchema.getGranularitySpec(), +// dataSchema.getTransformSpec(), +// objectMapper // ); // } // // private QueryRunnerFactoryConglomerate makeTimeseriesOnlyConglomerate() // { +// IntervalChunkingQueryRunnerDecorator queryRunnerDecorator = new IntervalChunkingQueryRunnerDecorator( +// null, +// null, +// null +// ) +// { +// @Override +// public QueryRunner decorate( +// QueryRunner delegate, QueryToolChest> toolChest +// ) +// { +// return delegate; +// } +// }; // return new DefaultQueryRunnerFactoryConglomerate( -// ImmutableMap., QueryRunnerFactory>of( +// ImmutableMap.of( // TimeseriesQuery.class, // new TimeseriesQueryRunnerFactory( -// new TimeseriesQueryQueryToolChest( -// new IntervalChunkingQueryRunnerDecorator(null, null, null) -// { -// @Override -// public QueryRunner decorate( -// QueryRunner delegate, QueryToolChest> toolChest -// ) -// { -// return delegate; -// } -// } -// ), +// new TimeseriesQueryQueryToolChest(queryRunnerDecorator), // new TimeseriesQueryEngine(), -// new QueryWatcher() -// { -// @Override -// public void registerQuery(Query query, ListenableFuture future) -// { -// // do nothing -// } +// (query, future) -> { +// // do nothing // } // ) // ) @@ -1431,8 +2124,9 @@ // { // directory = tempFolder.newFolder(); // final TestUtils testUtils = new TestUtils(); +// rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory(); // final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); -// for (Module module : new KinesisIndexingServiceModule().getJacksonModules()) { +// for (Module module : new KafkaIndexTaskModule().getJacksonModules()) { // objectMapper.registerModule(module); // } // final TaskConfig taskConfig = new TaskConfig( @@ -1456,7 +2150,7 @@ // taskStorage = new MetadataTaskStorage( // derbyConnector, // new TaskStorageConfig(null), -// new SQLMetadataStorageActionHandlerFactory( +// new DerbyMetadataStorageActionHandlerFactory( // derbyConnector, // derby.metadataTablesConfigSupplier().get(), // objectMapper @@ -1470,50 +2164,75 @@ // taskLockbox = new TaskLockbox(taskStorage); // final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( // taskLockbox, +// taskStorage, // metadataStorageCoordinator, // emitter, // new SupervisorManager(null) +// { +// @Override +// public boolean checkPointDataSourceMetadata( +// String supervisorId, +// @Nullable Integer taskGroupId, +// String baseSequenceName, +// @Nullable DataSourceMetadata previousDataSourceMetadata, +// @Nullable DataSourceMetadata currentDataSourceMetadata +// ) +// { +// log.info("Adding checkpoint hash to the set"); +// checkpointRequestsHash.add( +// Objects.hash( +// supervisorId, +// taskGroupId, +// previousDataSourceMetadata, +// currentDataSourceMetadata +// ) +// ); +// return true; +// } +// }, +// new Counters() // ); // final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( // taskStorage, // taskActionToolbox // ); -// final SegmentHandoffNotifierFactory handoffNotifierFactory = new SegmentHandoffNotifierFactory() +// final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier() // { // @Override -// public SegmentHandoffNotifier createSegmentHandoffNotifier(String dataSource) +// public boolean registerSegmentHandoffCallback( +// SegmentDescriptor descriptor, Executor exec, Runnable handOffRunnable +// ) // { -// return new SegmentHandoffNotifier() -// { -// @Override -// public boolean registerSegmentHandoffCallback( -// SegmentDescriptor descriptor, Executor exec, Runnable handOffRunnable -// ) -// { -// if (doHandoff) { -// // Simulate immediate handoff -// exec.execute(handOffRunnable); -// } -// return true; -// } +// if (doHandoff) { +// // Simulate immediate handoff +// exec.execute(handOffRunnable); +// } +// return true; +// } // -// @Override -// public void start() -// { -// //Noop -// } +// @Override +// public void start() +// { +// //Noop +// } // -// @Override -// public void close() -// { -// //Noop -// } -// }; +// @Override +// public void close() +// { +// //Noop // } // }; // final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig(); // dataSegmentPusherConfig.storageDirectory = getSegmentDirectory(); // final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig, objectMapper); +// SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig() +// { +// @Override +// public List getLocations() +// { +// return Lists.newArrayList(); +// } +// }; // toolboxFactory = new TaskToolboxFactory( // taskConfig, // taskActionClientFactory, @@ -1525,28 +2244,23 @@ // new TestDataSegmentAnnouncer(), // EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), // handoffNotifierFactory, -// makeTimeseriesOnlyConglomerate(), +// this::makeTimeseriesOnlyConglomerate, // MoreExecutors.sameThreadExecutor(), // queryExecutorService // EasyMock.createMock(MonitorScheduler.class), // new SegmentLoaderFactory( -// new SegmentLoaderLocalCacheManager( -// null, -// new SegmentLoaderConfig() -// { -// @Override -// public List getLocations() -// { -// return Lists.newArrayList(); -// } -// }, testUtils.getTestObjectMapper() -// ) +// new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper()) // ), // testUtils.getTestObjectMapper(), -// testUtils.getTestIndexMerger(), // testUtils.getTestIndexIO(), // MapCache.create(1024), // new CacheConfig(), -// testUtils.getTestIndexMergerV9() +// new CachePopulatorStats(), +// testUtils.getTestIndexMergerV9(), +// EasyMock.createNiceMock(DruidNodeAnnouncer.class), +// EasyMock.createNiceMock(DruidNode.class), +// new LookupNodeService("tier"), +// new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), +// new TaskReportFileWriter(reportsFile) // ); // } // @@ -1558,23 +2272,24 @@ // metadataStorageCoordinator = null; // } // -// private Set publishedDescriptors() throws IOException +// private Set publishedDescriptors() // { // return FluentIterable.from( // metadataStorageCoordinator.getUsedSegmentsForInterval( // DATA_SCHEMA.getDataSource(), -// new Interval("0000/3000") +// Intervals.of("0000/3000") // ) -// ).transform( -// new Function() -// { -// @Override -// public SegmentDescriptor apply(DataSegment input) -// { -// return input.toDescriptor(); -// } -// } -// ).toSet(); +// ).transform(DataSegment::toDescriptor).toSet(); +// } +// +// private void unlockAppenderatorBasePersistDirForTask(KafkaIndexTask task) +// throws NoSuchMethodException, InvocationTargetException, IllegalAccessException +// { +// Method unlockBasePersistDir = ((AppenderatorImpl) task.getAppenderator()).getClass() +// .getDeclaredMethod( +// "unlockBasePersistDirectory"); +// unlockBasePersistDir.setAccessible(true); +// unlockBasePersistDir.invoke(task.getAppenderator()); // } // // private File getSegmentDirectory() @@ -1582,11 +2297,11 @@ // return new File(directory, "segments"); // } // -// private List readSegmentDim1(final SegmentDescriptor descriptor) throws IOException +// private List readSegmentColumn(final String column, final SegmentDescriptor descriptor) throws IOException // { -// File indexZip = new File( -// String.format( -// "%s/%s/%s_%s/%s/%d/index.zip", +// File indexBasePath = new File( +// StringUtils.format( +// "%s/%s/%s_%s/%s/%d", // getSegmentDirectory(), // DATA_SCHEMA.getDataSource(), // descriptor.getInterval().getStart(), @@ -1595,9 +2310,10 @@ // descriptor.getPartitionNumber() // ) // ); +// // File outputLocation = new File( // directory, -// String.format( +// StringUtils.format( // "%s_%s_%s_%s", // descriptor.getInterval().getStart(), // descriptor.getInterval().getEnd(), @@ -1607,49 +2323,54 @@ // ); // outputLocation.mkdir(); // CompressionUtils.unzip( -// Files.asByteSource(indexZip), +// Files.asByteSource(new File(indexBasePath.listFiles()[0], "index.zip")), // outputLocation, -// Predicates.alwaysFalse(), +// Predicates.alwaysFalse(), // false // ); // IndexIO indexIO = new TestUtils().getTestIndexIO(); // QueryableIndex index = indexIO.loadIndex(outputLocation); -// DictionaryEncodedColumn dim1 = index.getColumn("dim1").getDictionaryEncoding(); +// DictionaryEncodedColumn theColumn = index.getColumn(column).getDictionaryEncoding(); // List values = Lists.newArrayList(); -// for (int i = 0; i < dim1.length(); i++) { -// int id = dim1.getSingleValueRow(i); -// String value = dim1.lookupName(id); +// for (int i = 0; i < theColumn.length(); i++) { +// int id = theColumn.getSingleValueRow(i); +// String value = theColumn.lookupName(id); // values.add(value); // } // return values; // } // -// public long countEvents(final Task task) throws Exception +// public long countEvents(final Task task) // { // // Do a query. // TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() // .dataSource(DATA_SCHEMA.getDataSource()) // .aggregators( -// ImmutableList.of( +// ImmutableList.of( // new LongSumAggregatorFactory("rows", "rows") // ) // ).granularity(Granularities.ALL) // .intervals("0000/3000") // .build(); // -// ArrayList> results = Sequences.toList( -// task.getQueryRunner(query).run(query, ImmutableMap.of()), -// Lists.>newArrayList() -// ); +// List> results = +// task.getQueryRunner(query).run(wrap(query), ImmutableMap.of()).toList(); // -// return results.isEmpty() ? 0 : results.get(0).getValue().getLongMetric("rows"); +// return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); // } // -// private static byte[] JB(String timestamp, String dim1, String dim2, double met1) +// private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) // { // try { // return new ObjectMapper().writeValueAsBytes( -// ImmutableMap.of("timestamp", timestamp, "dim1", dim1, "dim2", dim2, "met1", met1) +// ImmutableMap.builder() +// .put("timestamp", timestamp) +// .put("dim1", dim1) +// .put("dim2", dim2) +// .put("dimLong", dimLong) +// .put("dimFloat", dimFloat) +// .put("met1", met1) +// .build() // ); // } // catch (Exception e) { @@ -1659,7 +2380,20 @@ // // private SegmentDescriptor SD(final Task task, final String intervalString, final int partitionNum) // { -// final Interval interval = new Interval(intervalString); +// final Interval interval = Intervals.of(intervalString); // return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum); // } +// +// private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException +// { +// Map taskReports = objectMapper.readValue( +// reportsFile, +// new TypeReference>() +// { +// } +// ); +// return IngestionStatsAndErrorsTaskReportData.getPayloadFromTaskReports( +// taskReports +// ); +// } //} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java index 16d3def45953..7b22fefc771d 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java @@ -1,120 +1,207 @@ -///* -// * Licensed to Metamarkets Group Inc. (Metamarkets) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. Metamarkets licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, -// * software distributed under the License is distributed on an -// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// * KIND, either express or implied. See the License for the -// * specific language governing permissions and limitations -// * under the License. -// */ -// -//package org.apache.druid.indexing.kinesis; -// -//import com.fasterxml.jackson.databind.Module; -//import com.fasterxml.jackson.databind.ObjectMapper; -//import org.apache.druid.jackson.DefaultObjectMapper; -//import org.apache.druid.segment.IndexSpec; -//import org.apache.druid.segment.indexing.TuningConfig; -//import org.joda.time.Period; -//import org.junit.Assert; -//import org.junit.Test; -// -//import java.io.File; -// -//public class KinesisTuningConfigTest -//{ -// private final ObjectMapper mapper; -// -// public KinesisTuningConfigTest() -// { -// mapper = new DefaultObjectMapper(); -// mapper.registerModules((Iterable) new KinesisIndexingServiceModule().getJacksonModules()); -// } -// -// @Test -// public void testSerdeWithDefaults() throws Exception -// { -// String jsonStr = "{\"type\": \"kinesis\"}"; -// -// KinesisTuningConfig config = (KinesisTuningConfig) mapper.readValue( -// mapper.writeValueAsString( -// mapper.readValue( -// jsonStr, -// TuningConfig.class -// ) -// ), -// TuningConfig.class -// ); -// -// Assert.assertNotNull(config.getBasePersistDirectory()); -// Assert.assertEquals(75000, config.getMaxRowsInMemory()); -// Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment()); -// Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); -// Assert.assertEquals(0, config.getMaxPendingPersists()); -// Assert.assertEquals(new IndexSpec(), config.getIndexSpec()); -// Assert.assertEquals(true, config.getBuildV9Directly()); -// Assert.assertEquals(false, config.isReportParseExceptions()); -// Assert.assertEquals(0, config.getHandoffConditionTimeout()); -// } -// -// @Test -// public void testSerdeWithNonDefaults() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"basePersistDirectory\": \"/tmp/xxx\",\n" -// + " \"maxRowsInMemory\": 100,\n" -// + " \"maxRowsPerSegment\": 100,\n" -// + " \"intermediatePersistPeriod\": \"PT1H\",\n" -// + " \"maxPendingPersists\": 100,\n" -// + " \"buildV9Directly\": true,\n" -// + " \"reportParseExceptions\": true,\n" -// + " \"handoffConditionTimeout\": 100\n" -// + "}"; -// -// KinesisTuningConfig config = (KinesisTuningConfig) mapper.readValue( -// mapper.writeValueAsString( -// mapper.readValue( -// jsonStr, -// TuningConfig.class -// ) -// ), -// TuningConfig.class -// ); -// -// Assert.assertEquals(new File("/tmp/xxx"), config.getBasePersistDirectory()); -// Assert.assertEquals(100, config.getMaxRowsInMemory()); -// Assert.assertEquals(100, config.getMaxRowsPerSegment()); -// Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); -// Assert.assertEquals(100, config.getMaxPendingPersists()); -// Assert.assertEquals(true, config.getBuildV9Directly()); -// Assert.assertEquals(true, config.isReportParseExceptions()); -// Assert.assertEquals(100, config.getHandoffConditionTimeout()); -// } -// -// @Test -// public void testCopyOf() throws Exception -// { -// KinesisTuningConfig original = new KinesisTuningConfig(1, 2, new Period("PT3S"), new File("/tmp/xxx"), 4, new IndexSpec(), true, true, 5L, null); -// KinesisTuningConfig copy = KinesisTuningConfig.copyOf(original); -// -// Assert.assertEquals(1, copy.getMaxRowsInMemory()); -// Assert.assertEquals(2, copy.getMaxRowsPerSegment()); -// Assert.assertEquals(new Period("PT3S"), copy.getIntermediatePersistPeriod()); -// Assert.assertEquals(new File("/tmp/xxx"), copy.getBasePersistDirectory()); -// Assert.assertEquals(4, copy.getMaxPendingPersists()); -// Assert.assertEquals(new IndexSpec(), copy.getIndexSpec()); -// Assert.assertEquals(true, copy.getBuildV9Directly()); -// Assert.assertEquals(true, copy.isReportParseExceptions()); -// Assert.assertEquals(5L, copy.getHandoffConditionTimeout()); -// } -//} +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.indexing.TuningConfig; +import org.hamcrest.CoreMatchers; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.File; + +public class KinesisTuningConfigTest +{ + private final ObjectMapper mapper; + + public KinesisTuningConfigTest() + { + mapper = new DefaultObjectMapper(); + mapper.registerModules((Iterable) new KinesisIndexingServiceModule().getJacksonModules()); + } + + @Rule + public final ExpectedException exception = ExpectedException.none(); + + @Test + public void testSerdeWithDefaults() throws Exception + { + String jsonStr = "{\"type\": \"kinesis\"}"; + + KinesisTuningConfig config = (KinesisTuningConfig) mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + TuningConfig.class + ) + ), + TuningConfig.class + ); + + Assert.assertNotNull(config.getBasePersistDirectory()); + Assert.assertEquals(1000000, config.getMaxRowsInMemory()); + Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment()); + Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); + Assert.assertEquals(0, config.getMaxPendingPersists()); + Assert.assertEquals(new IndexSpec(), config.getIndexSpec()); + Assert.assertTrue(config.getBuildV9Directly()); + Assert.assertFalse(config.isReportParseExceptions()); + Assert.assertEquals(0, config.getHandoffConditionTimeout()); + Assert.assertEquals(10000, config.getRecordBufferSize()); + Assert.assertEquals(5000, config.getRecordBufferOfferTimeout()); + Assert.assertEquals(5000, config.getRecordBufferFullWait()); + Assert.assertEquals(60000, config.getFetchSequenceNumberTimeout()); + Assert.assertNull(config.getFetchThreads()); + Assert.assertFalse(config.isSkipSequenceNumberAvailabilityCheck()); + Assert.assertFalse(config.isResetOffsetAutomatically()); + } + + @Test + public void testSerdeWithNonDefaults() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"basePersistDirectory\": \"/tmp/xxx\",\n" + + " \"maxRowsInMemory\": 100,\n" + + " \"maxRowsPerSegment\": 100,\n" + + " \"intermediatePersistPeriod\": \"PT1H\",\n" + + " \"maxPendingPersists\": 100,\n" + + " \"buildV9Directly\": true,\n" + + " \"reportParseExceptions\": true,\n" + + " \"handoffConditionTimeout\": 100,\n" + + " \"recordBufferSize\": 1000,\n" + + " \"recordBufferOfferTimeout\": 500,\n" + + " \"recordBufferFullWait\": 500,\n" + + " \"fetchSequenceNumberTimeout\": 6000,\n" + + " \"resetOffsetAutomatically\": false,\n" + + " \"skipSequenceNumberAvailabilityCheck\": true,\n" + + " \"fetchThreads\": 2\n" + + "}"; + + KinesisTuningConfig config = (KinesisTuningConfig) mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + TuningConfig.class + ) + ), + TuningConfig.class + ); + + Assert.assertEquals(new File("/tmp/xxx"), config.getBasePersistDirectory()); + Assert.assertEquals(100, config.getMaxRowsInMemory()); + Assert.assertEquals(100, config.getMaxRowsPerSegment()); + Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); + Assert.assertEquals(0, config.getMaxPendingPersists()); + Assert.assertTrue(config.getBuildV9Directly()); + Assert.assertTrue(config.isReportParseExceptions()); + Assert.assertEquals(100, config.getHandoffConditionTimeout()); + Assert.assertEquals(1000, config.getRecordBufferSize()); + Assert.assertEquals(500, config.getRecordBufferOfferTimeout()); + Assert.assertEquals(500, config.getRecordBufferFullWait()); + Assert.assertEquals(6000, config.getFetchSequenceNumberTimeout()); + Assert.assertEquals(2, (int) config.getFetchThreads()); + Assert.assertTrue(config.isSkipSequenceNumberAvailabilityCheck()); + Assert.assertFalse(config.isResetOffsetAutomatically()); + } + + @Test + public void testResetOffsetAndSkipSequenceNotBothTrue() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"basePersistDirectory\": \"/tmp/xxx\",\n" + + " \"maxRowsInMemory\": 100,\n" + + " \"maxRowsPerSegment\": 100,\n" + + " \"intermediatePersistPeriod\": \"PT1H\",\n" + + " \"maxPendingPersists\": 100,\n" + + " \"buildV9Directly\": true,\n" + + " \"reportParseExceptions\": true,\n" + + " \"handoffConditionTimeout\": 100,\n" + + " \"recordBufferSize\": 1000,\n" + + " \"recordBufferOfferTimeout\": 500,\n" + + " \"recordBufferFullWait\": 500,\n" + + " \"fetchSequenceNumberTimeout\": 6000,\n" + + " \"resetOffsetAutomatically\": true,\n" + + " \"skipSequenceNumberAvailabilityCheck\": true,\n" + + " \"fetchThreads\": 2\n" + + "}"; + + exception.expect(JsonMappingException.class); + exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class)); + exception.expectMessage(CoreMatchers.containsString( + "resetOffsetAutomatically cannot be used if skipSequenceNumberAvailabilityCheck=true")); + mapper.readValue(jsonStr, TuningConfig.class); + } + + @Test + public void testCopyOf() + { + KinesisTuningConfig original = new KinesisTuningConfig( + 1, + (long) 3, + 2, + new Period("PT3S"), + new File("/tmp/xxx"), + 4, + new IndexSpec(), + true, + true, + 5L, + true, + false, + 1000, + 500, + 500, + 6000, + 2, + null, + null, + null, + null + ); + KinesisTuningConfig copy = original.copyOf(); + + Assert.assertEquals(1, copy.getMaxRowsInMemory()); + Assert.assertEquals(3, copy.getMaxBytesInMemory()); + Assert.assertEquals(2, copy.getMaxRowsPerSegment()); + Assert.assertEquals(new Period("PT3S"), copy.getIntermediatePersistPeriod()); + Assert.assertEquals(new File("/tmp/xxx"), copy.getBasePersistDirectory()); + Assert.assertEquals(0, copy.getMaxPendingPersists()); + Assert.assertEquals(new IndexSpec(), copy.getIndexSpec()); + Assert.assertTrue(copy.getBuildV9Directly()); + Assert.assertTrue(copy.isReportParseExceptions()); + Assert.assertEquals(5L, copy.getHandoffConditionTimeout()); + Assert.assertEquals(1000, copy.getRecordBufferSize()); + Assert.assertEquals(500, copy.getRecordBufferOfferTimeout()); + Assert.assertEquals(500, copy.getRecordBufferFullWait()); + Assert.assertEquals(6000, copy.getFetchSequenceNumberTimeout()); + Assert.assertEquals(2, (int) copy.getFetchThreads()); + Assert.assertFalse(copy.isSkipSequenceNumberAvailabilityCheck()); + Assert.assertTrue(copy.isResetOffsetAutomatically()); + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java index 8f9da95347d8..d3d2bfd5ffa5 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java @@ -1,161 +1,145 @@ -///* -// * Licensed to Metamarkets Group Inc. (Metamarkets) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. Metamarkets licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, -// * software distributed under the License is distributed on an -// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// * KIND, either express or implied. See the License for the -// * specific language governing permissions and limitations -// * under the License. -// */ -// -//package org.apache.druid.indexing.kinesis.supervisor; -// -//import com.fasterxml.jackson.databind.JsonMappingException; -//import com.fasterxml.jackson.databind.Module; -//import com.fasterxml.jackson.databind.ObjectMapper; -//import com.google.common.collect.ImmutableMap; -//import org.apache.druid.indexing.kinesis.KinesisIndexingServiceModule; -//import org.apache.druid.jackson.DefaultObjectMapper; -//import org.hamcrest.CoreMatchers; -//import org.joda.time.Duration; -//import org.junit.Assert; -//import org.junit.Rule; -//import org.junit.Test; -//import org.junit.rules.ExpectedException; -// -//public class KinesisSupervisorIOConfigTest -//{ -// private final ObjectMapper mapper; -// -// public KinesisSupervisorIOConfigTest() -// { -// mapper = new DefaultObjectMapper(); -// mapper.registerModules((Iterable) new KinesisIndexingServiceModule().getJacksonModules()); -// } -// -// @Rule -// public final ExpectedException exception = ExpectedException.none(); -// -// @Test -// public void testSerdeWithDefaults() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"topic\": \"my-topic\",\n" -// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"}\n" -// + "}"; -// -// KinesisSupervisorIOConfig config = mapper.readValue( -// mapper.writeValueAsString( -// mapper.readValue( -// jsonStr, -// KinesisSupervisorIOConfig.class -// ) -// ), KinesisSupervisorIOConfig.class -// ); -// -// Assert.assertEquals("my-topic", config.getStream()); -// Assert.assertEquals(1, (int) config.getReplicas()); -// Assert.assertEquals(1, (int) config.getTaskCount()); -// Assert.assertEquals(Duration.standardMinutes(60), config.getTaskDuration()); -// Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties()); -// Assert.assertEquals(Duration.standardSeconds(5), config.getStartDelay()); -// Assert.assertEquals(Duration.standardSeconds(30), config.getPeriod()); -// Assert.assertEquals(false, config.isUseEarliestOffset()); -// Assert.assertEquals(Duration.standardMinutes(30), config.getCompletionTimeout()); -// Assert.assertFalse("lateMessageRejectionPeriod", config.getLateMessageRejectionPeriod().isPresent()); -// Assert.assertFalse("skipOffsetGaps", config.isSkipOffsetGaps()); -// } -// -// @Test -// public void testSerdeWithNonDefaults() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"topic\": \"my-topic\",\n" -// + " \"replicas\": 3,\n" -// + " \"taskCount\": 9,\n" -// + " \"taskDuration\": \"PT30M\",\n" -// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" -// + " \"startDelay\": \"PT1M\",\n" -// + " \"period\": \"PT10S\",\n" -// + " \"useEarliestOffset\": true,\n" -// + " \"completionTimeout\": \"PT45M\",\n" -// + " \"lateMessageRejectionPeriod\": \"PT1H\",\n" -// + " \"skipOffsetGaps\": true\n" -// + "}"; -// -// KinesisSupervisorIOConfig config = mapper.readValue( -// mapper.writeValueAsString( -// mapper.readValue( -// jsonStr, -// KinesisSupervisorIOConfig.class -// ) -// ), KinesisSupervisorIOConfig.class -// ); -// -// Assert.assertEquals("my-topic", config.getStream()); -// Assert.assertEquals(3, (int) config.getReplicas()); -// Assert.assertEquals(9, (int) config.getTaskCount()); -// Assert.assertEquals(Duration.standardMinutes(30), config.getTaskDuration()); -// Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties()); -// Assert.assertEquals(Duration.standardMinutes(1), config.getStartDelay()); -// Assert.assertEquals(Duration.standardSeconds(10), config.getPeriod()); -// Assert.assertEquals(true, config.isUseEarliestOffset()); -// Assert.assertEquals(Duration.standardMinutes(45), config.getCompletionTimeout()); -// Assert.assertEquals(Duration.standardHours(1), config.getLateMessageRejectionPeriod().get()); -// Assert.assertTrue("skipOffsetGaps", config.isSkipOffsetGaps()); -// } -// -// @Test -// public void testTopicRequired() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"}\n" -// + "}"; -// -// exception.expect(JsonMappingException.class); -// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); -// exception.expectMessage(CoreMatchers.containsString("topic")); -// mapper.readValue(jsonStr, KinesisSupervisorIOConfig.class); -// } -// -// @Test -// public void testConsumerPropertiesRequired() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"topic\": \"my-topic\"\n" -// + "}"; -// -// exception.expect(JsonMappingException.class); -// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); -// exception.expectMessage(CoreMatchers.containsString("consumerProperties")); -// mapper.readValue(jsonStr, KinesisSupervisorIOConfig.class); -// } -// -// @Test -// public void testBootstrapServersRequired() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"topic\": \"my-topic\",\n" -// + " \"consumerProperties\": {}\n" -// + "}"; -// -// exception.expect(JsonMappingException.class); -// exception.expectCause(CoreMatchers.isA(NullPointerException.class)); -// exception.expectMessage(CoreMatchers.containsString("bootstrap.servers")); -// mapper.readValue(jsonStr, KinesisSupervisorIOConfig.class); -// } -//} +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis.supervisor; + +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexing.kinesis.KinesisIndexingServiceModule; +import org.apache.druid.indexing.kinesis.KinesisRegion; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.hamcrest.CoreMatchers; +import org.joda.time.Duration; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class KinesisSupervisorIOConfigTest +{ + private final ObjectMapper mapper; + + public KinesisSupervisorIOConfigTest() + { + mapper = new DefaultObjectMapper(); + mapper.registerModules((Iterable) new KinesisIndexingServiceModule().getJacksonModules()); + } + + @Rule + public final ExpectedException exception = ExpectedException.none(); + + @Test + public void testSerdeWithDefaults() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"stream\": \"my-stream\"\n" + + "}"; + + KinesisSupervisorIOConfig config = mapper.readValue( + jsonStr, + KinesisSupervisorIOConfig.class + ); + + Assert.assertEquals("my-stream", config.getStream()); + Assert.assertEquals(KinesisRegion.US_EAST_1.getEndpoint(), config.getEndpoint()); + Assert.assertEquals(1, (int) config.getReplicas()); + Assert.assertEquals(1, (int) config.getTaskCount()); + Assert.assertEquals(Duration.standardMinutes(60), config.getTaskDuration()); + Assert.assertEquals(Duration.standardSeconds(5), config.getStartDelay()); + Assert.assertEquals(Duration.standardSeconds(30), config.getPeriod()); + Assert.assertFalse(config.isUseEarliestSequenceNumber()); + Assert.assertEquals(Duration.standardMinutes(30), config.getCompletionTimeout()); + Assert.assertFalse("lateMessageRejectionPeriod", config.getLateMessageRejectionPeriod().isPresent()); + Assert.assertFalse("earlyMessageRejectionPeriod", config.getEarlyMessageRejectionPeriod().isPresent()); + Assert.assertEquals((Integer) 4000, config.getRecordsPerFetch()); + Assert.assertEquals((Integer) 0, config.getFetchDelayMillis()); + Assert.assertNull(config.getAwsAccessKeyId()); + Assert.assertNull(config.getAwsSecretAccessKey()); + Assert.assertNull(config.getAwsAssumedRoleArn()); + Assert.assertNull(config.getAwsExternalId()); + Assert.assertFalse(config.isDeaggregate()); + } + + @Test + public void testSerdeWithNonDefaults() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"stream\": \"my-stream\",\n" + + " \"region\": \"us-east-2\",\n" + + " \"endpoint\": \"kinesis.us-east-2.amazonaws.com\",\n" + + " \"replicas\": 3,\n" + + " \"taskCount\": 9,\n" + + " \"taskDuration\": \"PT30M\",\n" + + " \"startDelay\": \"PT1M\",\n" + + " \"period\": \"PT10S\",\n" + + " \"useEarliestSequenceNumber\": true,\n" + + " \"completionTimeout\": \"PT45M\",\n" + + " \"lateMessageRejectionPeriod\": \"PT1H\",\n" + + " \"earlyMessageRejectionPeriod\": \"PT1H\",\n" + + " \"recordsPerFetch\": 4000,\n" + + " \"fetchDelayMillis\": 1000,\n" + + " \"awsAccessKeyId\": \"awsid\",\n" + + " \"awsSecretAccessKey\": \"awskey\",\n" + + " \"awsAssumedRoleArn\": \"role\",\n" + + " \"awsExternalId\": \"awsexternalid\",\n" + + " \"deaggregate\": true\n" + + "}"; + + KinesisSupervisorIOConfig config = mapper.readValue( + jsonStr, + KinesisSupervisorIOConfig.class + ); + + Assert.assertEquals("my-stream", config.getStream()); + Assert.assertEquals(config.getEndpoint(), "kinesis.us-east-2.amazonaws.com"); + Assert.assertEquals(3, (int) config.getReplicas()); + Assert.assertEquals(9, (int) config.getTaskCount()); + Assert.assertEquals(Duration.standardMinutes(30), config.getTaskDuration()); + Assert.assertEquals(Duration.standardMinutes(1), config.getStartDelay()); + Assert.assertEquals(Duration.standardSeconds(10), config.getPeriod()); + Assert.assertTrue(config.isUseEarliestSequenceNumber()); + Assert.assertEquals(Duration.standardMinutes(45), config.getCompletionTimeout()); + Assert.assertEquals(Duration.standardHours(1), config.getLateMessageRejectionPeriod().get()); + Assert.assertEquals(Duration.standardHours(1), config.getEarlyMessageRejectionPeriod().get()); + Assert.assertEquals((Integer) 4000, config.getRecordsPerFetch()); + Assert.assertEquals((Integer) 1000, config.getFetchDelayMillis()); + Assert.assertEquals("awsid", config.getAwsAccessKeyId()); + Assert.assertEquals("awskey", config.getAwsSecretAccessKey()); + Assert.assertEquals("role", config.getAwsAssumedRoleArn()); + Assert.assertEquals("awsexternalid", config.getAwsExternalId()); + Assert.assertTrue(config.isDeaggregate()); + } + + @Test + public void testTopicRequired() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\"\n" + + "}"; + + exception.expect(JsonMappingException.class); + exception.expectCause(CoreMatchers.isA(NullPointerException.class)); + exception.expectMessage(CoreMatchers.containsString("stream")); + mapper.readValue(jsonStr, KinesisSupervisorIOConfig.class); + } + +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 5471dfc92e24..92f8eff5c702 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -1,21 +1,21 @@ -///* -// * Licensed to Metamarkets Group Inc. (Metamarkets) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. Metamarkets 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. -// */ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ // //package org.apache.druid.indexing.kinesis.supervisor; // diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java index 1169f7c5380c..36a02f4135ed 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java @@ -1,120 +1,120 @@ -///* -// * Licensed to Metamarkets Group Inc. (Metamarkets) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. Metamarkets licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, -// * software distributed under the License is distributed on an -// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// * KIND, either express or implied. See the License for the -// * specific language governing permissions and limitations -// * under the License. -// */ -// -//package org.apache.druid.indexing.kinesis.supervisor; -// -//import com.fasterxml.jackson.databind.Module; -//import com.fasterxml.jackson.databind.ObjectMapper; -//import org.apache.druid.indexing.kinesis.KinesisIndexingServiceModule; -//import org.apache.druid.jackson.DefaultObjectMapper; -//import org.apache.druid.segment.IndexSpec; -//import org.apache.druid.segment.indexing.TuningConfig; -//import org.joda.time.Duration; -//import org.joda.time.Period; -//import org.junit.Assert; -//import org.junit.Test; -// -//import java.io.File; -// -//public class KinesisSupervisorTuningConfigTest -//{ -// private final ObjectMapper mapper; -// -// public KinesisSupervisorTuningConfigTest() -// { -// mapper = new DefaultObjectMapper(); -// mapper.registerModules((Iterable) new KinesisIndexingServiceModule().getJacksonModules()); -// } -// -// @Test -// public void testSerdeWithDefaults() throws Exception -// { -// String jsonStr = "{\"type\": \"kinesis\"}"; -// -// KinesisSupervisorTuningConfig config = (KinesisSupervisorTuningConfig) mapper.readValue( -// mapper.writeValueAsString( -// mapper.readValue( -// jsonStr, -// TuningConfig.class -// ) -// ), -// TuningConfig.class -// ); -// -// Assert.assertNotNull(config.getBasePersistDirectory()); -// Assert.assertEquals(75000, config.getMaxRowsInMemory()); -// Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment()); -// Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); -// Assert.assertEquals(0, config.getMaxPendingPersists()); -// Assert.assertEquals(new IndexSpec(), config.getIndexSpec()); -// Assert.assertEquals(true, config.getBuildV9Directly()); -// Assert.assertEquals(false, config.isReportParseExceptions()); -// Assert.assertEquals(0, config.getHandoffConditionTimeout()); -// Assert.assertNull(config.getWorkerThreads()); -// Assert.assertNull(config.getChatThreads()); -// Assert.assertEquals(8L, (long) config.getChatRetries()); -// Assert.assertEquals(Duration.standardSeconds(10), config.getHttpTimeout()); -// Assert.assertEquals(Duration.standardSeconds(80), config.getShutdownTimeout()); -// } -// -// @Test -// public void testSerdeWithNonDefaults() throws Exception -// { -// String jsonStr = "{\n" -// + " \"type\": \"kinesis\",\n" -// + " \"basePersistDirectory\": \"/tmp/xxx\",\n" -// + " \"maxRowsInMemory\": 100,\n" -// + " \"maxRowsPerSegment\": 100,\n" -// + " \"intermediatePersistPeriod\": \"PT1H\",\n" -// + " \"maxPendingPersists\": 100,\n" -// + " \"buildV9Directly\": false,\n" -// + " \"reportParseExceptions\": true,\n" -// + " \"handoffConditionTimeout\": 100,\n" -// + " \"workerThreads\": 12,\n" -// + " \"chatThreads\": 13,\n" -// + " \"chatRetries\": 14,\n" -// + " \"httpTimeout\": \"PT15S\",\n" -// + " \"shutdownTimeout\": \"PT95S\"\n" -// + "}"; -// -// KinesisSupervisorTuningConfig config = (KinesisSupervisorTuningConfig) mapper.readValue( -// mapper.writeValueAsString( -// mapper.readValue( -// jsonStr, -// TuningConfig.class -// ) -// ), -// TuningConfig.class -// ); -// -// Assert.assertEquals(new File("/tmp/xxx"), config.getBasePersistDirectory()); -// Assert.assertEquals(100, config.getMaxRowsInMemory()); -// Assert.assertEquals(100, config.getMaxRowsPerSegment()); -// Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); -// Assert.assertEquals(100, config.getMaxPendingPersists()); -// Assert.assertEquals(false, config.getBuildV9Directly()); -// Assert.assertEquals(true, config.isReportParseExceptions()); -// Assert.assertEquals(100, config.getHandoffConditionTimeout()); -// Assert.assertEquals(12, (int) config.getWorkerThreads()); -// Assert.assertEquals(13, (int) config.getChatThreads()); -// Assert.assertEquals(14L, (long) config.getChatRetries()); -// Assert.assertEquals(Duration.standardSeconds(15), config.getHttpTimeout()); -// Assert.assertEquals(Duration.standardSeconds(95), config.getShutdownTimeout()); -// } -//} +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis.supervisor; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexing.kinesis.KinesisIndexingServiceModule; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.indexing.TuningConfig; +import org.joda.time.Duration; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; + +public class KinesisSupervisorTuningConfigTest +{ + private final ObjectMapper mapper; + + public KinesisSupervisorTuningConfigTest() + { + mapper = new DefaultObjectMapper(); + mapper.registerModules((Iterable) new KinesisIndexingServiceModule().getJacksonModules()); + } + + @Test + public void testSerdeWithDefaults() throws Exception + { + String jsonStr = "{\"type\": \"kinesis\"}"; + + KinesisSupervisorTuningConfig config = (KinesisSupervisorTuningConfig) mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + TuningConfig.class + ) + ), + TuningConfig.class + ); + + Assert.assertNotNull(config.getBasePersistDirectory()); + Assert.assertEquals(75000, config.getMaxRowsInMemory()); + Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment()); + Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); + Assert.assertEquals(0, config.getMaxPendingPersists()); + Assert.assertEquals(new IndexSpec(), config.getIndexSpec()); + Assert.assertEquals(true, config.getBuildV9Directly()); + Assert.assertEquals(false, config.isReportParseExceptions()); + Assert.assertEquals(0, config.getHandoffConditionTimeout()); + Assert.assertNull(config.getWorkerThreads()); + Assert.assertNull(config.getChatThreads()); + Assert.assertEquals(8L, (long) config.getChatRetries()); + Assert.assertEquals(Duration.standardSeconds(10), config.getHttpTimeout()); + Assert.assertEquals(Duration.standardSeconds(80), config.getShutdownTimeout()); + } + + @Test + public void testSerdeWithNonDefaults() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"basePersistDirectory\": \"/tmp/xxx\",\n" + + " \"maxRowsInMemory\": 100,\n" + + " \"maxRowsPerSegment\": 100,\n" + + " \"intermediatePersistPeriod\": \"PT1H\",\n" + + " \"maxPendingPersists\": 100,\n" + + " \"buildV9Directly\": false,\n" + + " \"reportParseExceptions\": true,\n" + + " \"handoffConditionTimeout\": 100,\n" + + " \"workerThreads\": 12,\n" + + " \"chatThreads\": 13,\n" + + " \"chatRetries\": 14,\n" + + " \"httpTimeout\": \"PT15S\",\n" + + " \"shutdownTimeout\": \"PT95S\"\n" + + "}"; + + KinesisSupervisorTuningConfig config = (KinesisSupervisorTuningConfig) mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + TuningConfig.class + ) + ), + TuningConfig.class + ); + + Assert.assertEquals(new File("/tmp/xxx"), config.getBasePersistDirectory()); + Assert.assertEquals(100, config.getMaxRowsInMemory()); + Assert.assertEquals(100, config.getMaxRowsPerSegment()); + Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); + Assert.assertEquals(100, config.getMaxPendingPersists()); + Assert.assertEquals(false, config.getBuildV9Directly()); + Assert.assertEquals(true, config.isReportParseExceptions()); + Assert.assertEquals(100, config.getHandoffConditionTimeout()); + Assert.assertEquals(12, (int) config.getWorkerThreads()); + Assert.assertEquals(13, (int) config.getChatThreads()); + Assert.assertEquals(14L, (long) config.getChatRetries()); + Assert.assertEquals(Duration.standardSeconds(15), config.getHttpTimeout()); + Assert.assertEquals(Duration.standardSeconds(95), config.getShutdownTimeout()); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java index e85595944011..4ee23863b8dc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java @@ -16,24 +16,6 @@ * specific language governing permissions and limitations * under the License. */ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ package org.apache.druid.indexing.seekablestream; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java index 3745746c35d6..b67bcf0b061b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java @@ -29,7 +29,7 @@ import javax.annotation.Nullable; import java.util.Set; -public abstract class SeekableStreamIOConfig implements IOConfig +public abstract class SeekableStreamIOConfig, T2 extends Comparable> implements IOConfig { private static final boolean DEFAULT_USE_TRANSACTION = true; @@ -60,6 +60,30 @@ public SeekableStreamIOConfig( this.useTransaction = useTransaction != null ? useTransaction : DEFAULT_USE_TRANSACTION; this.minimumMessageTime = Optional.fromNullable(minimumMessageTime); this.maximumMessageTime = Optional.fromNullable(maximumMessageTime); + + Preconditions.checkArgument( + startPartitions.getId().equals(endPartitions.getId()), + "start topic/stream and end topic/stream must match" + ); + + Preconditions.checkArgument( + startPartitions.getPartitionSequenceMap().keySet().equals(endPartitions.getPartitionSequenceMap().keySet()), + "start partition set and end partition set must match" + ); + + // are sequence numbers guranteed to be greater? + /* + for (T1 partition : endPartitions.getPartitionSequenceMap().keySet()) { + Preconditions.checkArgument( + endPartitions.getPartitionSequenceMap() + .get(partition) + .compareTo(startPartitions.getPartitionSequenceMap().get(partition)) >= 0, + "end offset must be >= start offset for partition[%s]", + partition + ); + } + */ + } @Nullable diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 2b5d958495ae..1c55aa5d3a7c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -47,7 +47,8 @@ import java.util.Random; //TODO: need more refactoring for run() -public abstract class SeekableStreamIndexTask extends AbstractTask implements ChatHandler +public abstract class SeekableStreamIndexTask, T2 extends Comparable> extends AbstractTask + implements ChatHandler { private static final Random RANDOM = new Random(); protected final DataSchema dataSchema; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java index bb4738719877..b926303542f9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java @@ -47,9 +47,6 @@ static Duration defaultDuration(final Period period, final String theDefault) @JsonProperty Duration getShutdownTimeout(); - @JsonProperty - Duration getOffsetFetchPeriod(); - @Override String toString(); From 511199f2f86525342a98b5168216bcbaf0b28818 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 26 Sep 2018 12:01:54 -0700 Subject: [PATCH 12/87] various bug fixes for kinesis-indexing-service --- ...ementalPublishingKafkaIndexTaskRunner.java | 5 +- .../druid/indexing/kafka/KafkaIndexTask.java | 5 +- .../indexing/kafka/KafkaIndexTaskRunner.java | 74 -------------- .../indexing/kafka/KafkaSequenceNumber.java | 55 +++++++++++ .../kafka/LegacyKafkaIndexTaskRunner.java | 5 +- .../kafka/supervisor/KafkaSupervisor.java | 10 ++ .../indexing/kinesis/KinesisIndexTask.java | 18 +++- .../kinesis/KinesisSequenceNumber.java | 68 +++++++++++++ .../kinesis/supervisor/KinesisSupervisor.java | 10 ++ .../SeekableStreamIOConfig.java | 2 +- .../SeekableStreamIndexTask.java | 2 +- .../SeekableStreamIndexTaskRunner.java | 6 +- ...nceNumberPlus.java => SequenceNumber.java} | 26 +++-- .../supervisor/SeekableStreamSupervisor.java | 98 +++++++++++-------- 14 files changed, 246 insertions(+), 138 deletions(-) delete mode 100644 extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java create mode 100644 extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java create mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java rename indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/{SequenceNumberPlus.java => SequenceNumber.java} (70%) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 3c017636cc21..0c65620ce812 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -44,7 +44,6 @@ import org.apache.druid.discovery.NodeType; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; @@ -57,6 +56,8 @@ import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.RealtimeIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -125,7 +126,7 @@ /** * Kafka indexing task runner supporting incremental segments publishing */ -public class IncrementalPublishingKafkaIndexTaskRunner implements KafkaIndexTaskRunner +public class IncrementalPublishingKafkaIndexTaskRunner implements SeekableStreamIndexTaskRunner { private static final EmittingLogger log = new EmittingLogger(IncrementalPublishingKafkaIndexTaskRunner.class); private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 7ee651aa7c86..792b95d8b7dd 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -34,6 +34,7 @@ import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; @@ -65,7 +66,7 @@ public class KafkaIndexTask extends SeekableStreamIndexTask static final long POLL_TIMEOUT_MILLIS = TimeUnit.MILLISECONDS.toMillis(100); static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15; - private final KafkaIndexTaskRunner runner; + private final SeekableStreamIndexTaskRunner runner; // This value can be tuned in some tests private long pollRetryMs = 30000; @@ -292,7 +293,7 @@ Appenderator getAppenderator() } @VisibleForTesting - KafkaIndexTaskRunner getRunner() + SeekableStreamIndexTaskRunner getRunner() { return runner; } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java deleted file mode 100644 index a85f18ecdd3a..000000000000 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.druid.indexing.kafka; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.stats.RowIngestionMeters; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; -import org.apache.druid.segment.realtime.appenderator.Appenderator; -import org.apache.druid.segment.realtime.firehose.ChatHandler; - -import javax.ws.rs.core.Response; -import java.util.Map; - -/** - * This class is used by only {@link KafkaIndexTask}. We currently have two implementations of this interface, i.e., - * {@link IncrementalPublishingKafkaIndexTaskRunner} and {@link LegacyKafkaIndexTaskRunner}. The latter one was used in - * the versions prior to 0.12.0, but being kept to support rolling update from them. - *

- * We don't have a good reason for having this interface except for better code maintenance for the latest kakfa - * indexing algorithm. As a result, this interface can be removed in the future when {@link LegacyKafkaIndexTaskRunner} - * is removed and it's no longer useful. - */ -public interface KafkaIndexTaskRunner extends ChatHandler -{ - Appenderator getAppenderator(); - - TaskStatus run(TaskToolbox toolbox); - - void stopGracefully(); - - // The below methods are mostly for unit testing. - - @VisibleForTesting - RowIngestionMeters getRowIngestionMeters(); - - @VisibleForTesting - SeekableStreamIndexTask.Status getStatus(); - - @VisibleForTesting - Map getCurrentOffsets(); - - @VisibleForTesting - Map getEndOffsets(); - - @VisibleForTesting - Response setEndOffsets( - Map offsets, - boolean finish // this field is only for internal purposes, shouldn't be usually set by users - ) throws InterruptedException; - - @VisibleForTesting - Response pause() throws InterruptedException; - - @VisibleForTesting - void resume() throws InterruptedException; -} diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java new file mode 100644 index 000000000000..626a8d8c6161 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kafka; + +import org.apache.druid.indexing.seekablestream.common.SequenceNumber; + +import javax.validation.constraints.NotNull; + +public class KafkaSequenceNumber extends SequenceNumber +{ + private KafkaSequenceNumber(Long sequenceNumber, boolean useExclusive, boolean isExclusive) + { + super(sequenceNumber, useExclusive, isExclusive); + } + + public static KafkaSequenceNumber of(Long sequenceNumber) + { + return new KafkaSequenceNumber(sequenceNumber, false, false); + } + + @Override + public int compareTo( + @NotNull SequenceNumber o + ) + { + return this.get().compareTo(o.get()); + } + + @Override + public boolean equals(Object o) + { + if (!(o instanceof KafkaSequenceNumber)) { + return false; + } + return this.compareTo((KafkaSequenceNumber) o) == 0; + } + +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index dc6ac9746485..3b23a478b79e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -35,7 +35,6 @@ import org.apache.druid.discovery.NodeType; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; @@ -47,6 +46,8 @@ import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.RealtimeIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -109,7 +110,7 @@ * This class will be removed in a future release. */ @Deprecated -public class LegacyKafkaIndexTaskRunner implements KafkaIndexTaskRunner +public class LegacyKafkaIndexTaskRunner implements SeekableStreamIndexTaskRunner { private static final EmittingLogger log = new EmittingLogger(LegacyKafkaIndexTaskRunner.class); private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index dab3e02b4ad7..2bb39fef0104 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -35,6 +35,7 @@ import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; import org.apache.druid.indexing.kafka.KafkaPartitions; import org.apache.druid.indexing.kafka.KafkaRecordSupplier; +import org.apache.druid.indexing.kafka.KafkaSequenceNumber; import org.apache.druid.indexing.kafka.KafkaTuningConfig; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -44,6 +45,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.SequenceNumber; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; @@ -329,6 +331,14 @@ protected Map createNewTaskEndPartitions(Set startPartit return endPartitions; } + @Override + protected SequenceNumber makeSequenceNumber( + Long seq, boolean useExclusive, boolean isExclusive + ) + { + return KafkaSequenceNumber.of(seq); + } + private Runnable emitLag() { return () -> { diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 3eedf7dc6996..4cba120bbfd8 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -990,6 +990,18 @@ public Response getUnparseableEvents( return Response.ok(events).build(); } + @VisibleForTesting + RowIngestionMeters getRowIngestionMeters() + { + return rowIngestionMeters; + } + + @VisibleForTesting + Appenderator getAppenderator() + { + return appenderator; + } + @VisibleForTesting FireDepartmentMetrics getFireDepartmentMetrics() { @@ -1091,7 +1103,8 @@ private Set assignPartitions(RecordSupplier recordSupplier, String topic final Set assignment = Sets.newHashSet(); for (Map.Entry entry : lastOffsets.entrySet()) { final String endOffset = endOffsets.get(entry.getKey()); - if (KinesisPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) || entry.getValue().compareTo(endOffset) < 0) { + if (KinesisPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) + || KinesisSequenceNumber.of(entry.getValue()).compareTo(KinesisSequenceNumber.of(endOffset)) < 0) { assignment.add(entry.getKey()); } else if (entry.getValue().equals(endOffset)) { log.info("Finished reading partition[%s].", entry.getKey()); @@ -1124,7 +1137,8 @@ private void seekToStartingRecords( if (!tuningConfig.isSkipSequenceNumberAvailabilityCheck()) { try { String earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); - if (earliestSequenceNumber == null || earliestSequenceNumber.compareTo(offset) > 0) { + if (earliestSequenceNumber == null + || KinesisSequenceNumber.of(earliestSequenceNumber).compareTo(KinesisSequenceNumber.of(offset)) > 0) { if (tuningConfig.isResetOffsetAutomatically()) { log.info("Attempting to reset offsets automatically for all partitions"); try { diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java new file mode 100644 index 000000000000..0071edf954d9 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + + +import org.apache.druid.indexing.seekablestream.common.SequenceNumber; + +import java.math.BigInteger; + +public class KinesisSequenceNumber extends SequenceNumber +{ + + private final BigInteger intSequence; + + private KinesisSequenceNumber(String sequenceNumber, boolean useExclusive, boolean isExclusive) + { + super(sequenceNumber, useExclusive, isExclusive); + this.intSequence = new BigInteger(sequenceNumber); + } + + public BigInteger getBigInteger() + { + return intSequence; + } + + public static KinesisSequenceNumber of(String sequenceNumber) + { + return new KinesisSequenceNumber(sequenceNumber, false, false); + } + + public static KinesisSequenceNumber of(String sequenceNumber, boolean useExclusive, boolean isExclusive) + { + return new KinesisSequenceNumber(sequenceNumber, useExclusive, isExclusive); + } + + @Override + public boolean equals(Object o) + { + if (!(o instanceof KinesisSequenceNumber)) { + return false; + } + return this.compareTo((KinesisSequenceNumber) o) == 0; + } + + + @Override + public int compareTo(SequenceNumber o) + { + return this.intSequence.compareTo(new BigInteger(o.get())); + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index a5373e2d8ffd..18264aaa7a66 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; import org.apache.druid.indexing.kinesis.KinesisPartitions; import org.apache.druid.indexing.kinesis.KinesisRecordSupplier; +import org.apache.druid.indexing.kinesis.KinesisSequenceNumber; import org.apache.druid.indexing.kinesis.KinesisTuningConfig; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -41,6 +42,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.SequenceNumber; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; @@ -274,6 +276,14 @@ protected Map createNewTaskEndPartitions(Set startPartit return endPartitions; } + @Override + protected SequenceNumber makeSequenceNumber( + String seq, boolean useExclusive, boolean isExclusive + ) + { + return KinesisSequenceNumber.of(seq, useExclusive, isExclusive); + } + // TODO: Implement this for Kinesis which uses approximate time from latest instead of offset lag /* diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java index b67bcf0b061b..b0f055c16b8e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java @@ -29,7 +29,7 @@ import javax.annotation.Nullable; import java.util.Set; -public abstract class SeekableStreamIOConfig, T2 extends Comparable> implements IOConfig +public abstract class SeekableStreamIOConfig implements IOConfig { private static final boolean DEFAULT_USE_TRANSACTION = true; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 1c55aa5d3a7c..ed5d6685abdf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -47,7 +47,7 @@ import java.util.Random; //TODO: need more refactoring for run() -public abstract class SeekableStreamIndexTask, T2 extends Comparable> extends AbstractTask +public abstract class SeekableStreamIndexTask extends AbstractTask implements ChatHandler { private static final Random RANDOM = new Random(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index b0fc2c478e39..f7c52c67b9e5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -55,11 +55,11 @@ public interface SeekableStreamIndexTaskRunner extends ChatHandler Response setEndOffsets( Map offsets, boolean finish // this field is only for internal purposes, shouldn't be usually set by users - ); + ) throws InterruptedException; @VisibleForTesting - Response pause(); + Response pause() throws InterruptedException; @VisibleForTesting - void resume(); + void resume() throws InterruptedException; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java similarity index 70% rename from indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java index fbb99cde814e..7f080b238fe5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumberPlus.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java @@ -20,22 +20,19 @@ package org.apache.druid.indexing.seekablestream.common; -public class SequenceNumberPlus +import java.util.Objects; + +public abstract class SequenceNumber implements Comparable> { private final T sequenceNumber; - private final boolean exclusive; + private final boolean isExclusive; private final boolean useExclusive; - private SequenceNumberPlus(T sequenceNumber, boolean useExclusive, boolean exclusive) + protected SequenceNumber(T sequenceNumber, boolean useExclusive, boolean isExclusive) { this.sequenceNumber = sequenceNumber; - this.exclusive = exclusive; this.useExclusive = useExclusive; - } - - public static SequenceNumberPlus of(T sequenceNumber, boolean useExclusive, boolean exclusive) - { - return new SequenceNumberPlus<>(sequenceNumber, useExclusive, exclusive); + this.isExclusive = isExclusive; } public T get() @@ -45,6 +42,15 @@ public T get() public boolean isExclusive() { - return useExclusive && exclusive; + return useExclusive && isExclusive; + } + + @Override + public abstract boolean equals(Object o); + + @Override + public int hashCode() + { + return Objects.hash(sequenceNumber, useExclusive, isExclusive); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 154209fe01f5..471f7ea69bff 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -65,7 +65,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.indexing.seekablestream.common.Record; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; -import org.apache.druid.indexing.seekablestream.common.SequenceNumberPlus; +import org.apache.druid.indexing.seekablestream.common.SequenceNumber; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; @@ -106,7 +106,7 @@ //TODO: documentation //TODO: compare with kinesis supervisor for subsequently discovered partitions -public abstract class SeekableStreamSupervisor, T2 extends Comparable> +public abstract class SeekableStreamSupervisor implements Supervisor { public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED"; @@ -878,7 +878,8 @@ public Boolean apply(SeekableStreamIndexTask.Status status) do { succeeded = true; T2 previousOffset = partitionOffsets.putIfAbsent(partition, offset); - if (previousOffset != null && (previousOffset.compareTo(offset)) < 0) { + if (previousOffset != null + && (makeSequenceNumber(previousOffset).compareTo(makeSequenceNumber(offset))) < 0) { succeeded = partitionOffsets.replace(partition, previousOffset, offset); } } while (!succeeded); @@ -1081,23 +1082,29 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) // store if (taskCheckpoints.entrySet().stream().anyMatch( sequenceCheckpoint -> sequenceCheckpoint.getValue().entrySet().stream().allMatch( - partitionOffset -> partitionOffset.getValue().compareTo(latestOffsetsFromDb == null ? - partitionOffset.getValue() : - latestOffsetsFromDb.getOrDefault( - partitionOffset - .getKey(), - partitionOffset - .getValue() - )) == 0) - && earliestConsistentSequenceId.compareAndSet(-1, sequenceCheckpoint.getKey())) || ( + partitionOffset -> { + SequenceNumber offset = makeSequenceNumber(partitionOffset.getValue()); + SequenceNumber latestOffset = makeSequenceNumber( + latestOffsetsFromDb == null ? partitionOffset.getValue() : + latestOffsetsFromDb.getOrDefault( + partitionOffset + .getKey(), + partitionOffset + .getValue() + ) + ); + + return offset.equals(latestOffset); + } + ) && earliestConsistentSequenceId.compareAndSet(-1, sequenceCheckpoint.getKey())) || ( pendingCompletionTaskGroups.getOrDefault(groupId, EMPTY_LIST).size() > 0 && earliestConsistentSequenceId.compareAndSet(-1, taskCheckpoints.firstKey()))) { final SortedMap> latestCheckpoints = new TreeMap<>( taskCheckpoints.tailMap(earliestConsistentSequenceId.get()) ); log.info("Setting taskGroup sequences to [%s] for group [%d]", latestCheckpoints, groupId); - taskGroup.currentSequences.clear(); - taskGroup.currentSequences.putAll(latestCheckpoints); + taskGroup.checkpointSequences.clear(); + taskGroup.checkpointSequences.putAll(latestCheckpoints); } else { log.debug( "Adding task [%s] to kill list, checkpoints[%s], latestoffsets from DB [%s]", @@ -1109,16 +1116,16 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) } } else { // check consistency with taskGroup sequences - if (taskCheckpoints.get(taskGroup.currentSequences.firstKey()) == null - || !(taskCheckpoints.get(taskGroup.currentSequences.firstKey()) - .equals(taskGroup.currentSequences.firstEntry().getValue())) - || taskCheckpoints.tailMap(taskGroup.currentSequences.firstKey()).size() - != taskGroup.currentSequences.size()) { + if (taskCheckpoints.get(taskGroup.checkpointSequences.firstKey()) == null + || !(taskCheckpoints.get(taskGroup.checkpointSequences.firstKey()) + .equals(taskGroup.checkpointSequences.firstEntry().getValue())) + || taskCheckpoints.tailMap(taskGroup.checkpointSequences.firstKey()).size() + != taskGroup.checkpointSequences.size()) { log.debug( "Adding task [%s] to kill list, checkpoints[%s], taskgroup checkpoints [%s]", taskId, taskCheckpoints, - taskGroup.currentSequences + taskGroup.checkpointSequences ); tasksToKill.add(taskId); } @@ -1142,7 +1149,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) + "persisted offsets in metadata store [%s]", sequenceCheckpoint.lhs, sequenceCheckpoint.rhs, - taskGroup.currentSequences, + taskGroup.checkpointSequences, latestOffsetsFromDb ); killTask(sequenceCheckpoint.lhs); @@ -1529,7 +1536,8 @@ public Map apply(List> input) } else { // otherwise build a map of the highest offsets seen for (Map.Entry offset : result.entrySet()) { if (!endOffsets.containsKey(offset.getKey()) - || endOffsets.get(offset.getKey()).compareTo(offset.getValue()) < 0) { + || makeSequenceNumber(endOffsets.get(offset.getKey())).compareTo( + makeSequenceNumber(offset.getValue())) < 0) { endOffsets.put(offset.getKey(), offset.getValue()); } } @@ -1548,11 +1556,11 @@ public Map apply(List> input) try { - if (endOffsets.equals(taskGroup.currentSequences.lastEntry().getValue())) { + if (endOffsets.equals(taskGroup.checkpointSequences.lastEntry().getValue())) { log.warn( "Checkpoint [%s] is same as the start offsets [%s] of latest sequence for the task group [%d]", endOffsets, - taskGroup.currentSequences.lastEntry().getValue(), + taskGroup.checkpointSequences.lastEntry().getValue(), taskGroup.groupId ); } @@ -1778,7 +1786,7 @@ private void createNewTasks() throws JsonProcessingException try { - Map> startingOffsets = generateStartingSequencesForPartitionGroup(groupId); + Map> startingOffsets = generateStartingSequencesForPartitionGroup(groupId); ImmutableMap simpleStartingOffsets = startingOffsets .entrySet() @@ -1872,23 +1880,23 @@ protected int getNoticesQueueSize() return notices.size(); } - private ImmutableMap> generateStartingSequencesForPartitionGroup(int groupId) + private ImmutableMap> generateStartingSequencesForPartitionGroup(int groupId) throws TimeoutException { - ImmutableMap.Builder> builder = ImmutableMap.builder(); + ImmutableMap.Builder> builder = ImmutableMap.builder(); for (Map.Entry entry : partitionGroups.get(groupId).entrySet()) { T1 partition = entry.getKey(); T2 offset = entry.getValue(); - if (NOT_SET.equals(offset)) { + if (!NOT_SET.equals(offset)) { // if we are given a startingOffset (set by a previous task group which is pending completion) then use it if (!Record.END_OF_SHARD_MARKER.equals(offset)) { - builder.put(partition, SequenceNumberPlus.of(offset, useExclusiveStartingSequence, true)); + builder.put(partition, makeSequenceNumber(offset, useExclusiveStartingSequence, true)); } } else { // if we don't have a startingOffset (first run or we had some previous failures and reset the offsets) then // get the offset from metadata storage (if available) or Kafka/Kinesis (otherwise) - SequenceNumberPlus offsetFromStorage = getOffsetFromStorageForPartition(partition); + SequenceNumber offsetFromStorage = getOffsetFromStorageForPartition(partition); if (!Record.END_OF_SHARD_MARKER.equals(offset)) { builder.put(partition, offsetFromStorage); @@ -1902,7 +1910,7 @@ private ImmutableMap> generateStartingSequencesForPar * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't * find any data, it will retrieve the latest or earliest Kafka/Kinesis offset depending on the useEarliestOffset config. */ - private SequenceNumberPlus getOffsetFromStorageForPartition(T1 partition) throws TimeoutException + private SequenceNumber getOffsetFromStorageForPartition(T1 partition) throws TimeoutException { final Map metadataOffsets = getOffsetsFromMetadataStorage(); T2 offset = metadataOffsets.get(partition); @@ -1912,7 +1920,7 @@ private SequenceNumberPlus getOffsetFromStorageForPartition(T1 partition) th try { T2 latestSequence = getOffsetFromStreamForPartition(partition, false); - if (latestSequence == null || offset.compareTo(latestSequence) > 0) { + if (latestSequence == null || makeSequenceNumber(offset).compareTo(makeSequenceNumber(latestSequence)) > 0) { if (taskTuningConfig.isResetOffsetAutomatically()) { resetInternal( createDataSourceMetaData(ioConfig.getId(), ImmutableMap.of(partition, offset)) @@ -1938,7 +1946,7 @@ private SequenceNumberPlus getOffsetFromStorageForPartition(T1 partition) th throw new ISE(e, "Timeout while fetching earliest sequence number for partition [%s]", partition); } } - return SequenceNumberPlus.of(offset, useExclusiveStartingSequence, true); + return makeSequenceNumber(offset, useExclusiveStartingSequence, true); } else { boolean useEarliestSequenceNumber = ioConfig.isUseEarliestSequenceNumber(); if (subsequentlyDiscoveredPartitions.contains(partition)) { @@ -1951,7 +1959,7 @@ private SequenceNumberPlus getOffsetFromStorageForPartition(T1 partition) th offset = getOffsetFromStreamForPartition(partition, useEarliestSequenceNumber); log.info("Getting sequence number [%s] for partition [%s]", offset, partition); - return SequenceNumberPlus.of(offset, useExclusiveStartingSequence, false); + return makeSequenceNumber(offset, useExclusiveStartingSequence, false); } } @@ -2019,7 +2027,7 @@ private void createTasksForGroup(int groupId, int replicas) throws JsonProcessin replicas, group.baseSequenceName, sortingMapper, - group.currentSequences, + group.checkpointSequences, newIoConfig, taskTuningConfig, rowIngestionMetersFactory @@ -2120,7 +2128,7 @@ protected Map getHighestCurrentOffsets() .collect(Collectors.toMap( Map.Entry::getKey, Map.Entry::getValue, - (v1, v2) -> v1.compareTo(v2) > 0 ? v1 : v2 + (v1, v2) -> makeSequenceNumber(v1).compareTo(makeSequenceNumber(v2)) > 0 ? v1 : v2 )); } @@ -2208,7 +2216,7 @@ private class TaskGroup final Optional minimumMessageTime; final Optional maximumMessageTime; final Set exclusiveStartSequenceNumberPartitions; //TODO: exclusiveSequence - final TreeMap> currentSequences = new TreeMap<>(); + final TreeMap> checkpointSequences = new TreeMap<>(); final String baseSequenceName; DateTime completionTimeout; @@ -2224,6 +2232,7 @@ public TaskGroup( this.startingSequences = startingSequences; this.minimumMessageTime = minimumMessageTime; this.maximumMessageTime = maximumMessageTime; + this.checkpointSequences.put(0, startingSequences); this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions != null ? exclusiveStartSequenceNumberPartitions : new HashSet<>(); @@ -2232,8 +2241,8 @@ public TaskGroup( int addNewCheckpoint(Map checkpoint) { - currentSequences.put(currentSequences.lastKey() + 1, checkpoint); - return currentSequences.lastKey(); + checkpointSequences.put(checkpointSequences.lastKey() + 1, checkpoint); + return checkpointSequences.lastKey(); } public Set taskIds() @@ -2255,7 +2264,7 @@ public String toString() return "TaskData{" + "status=" + status + ", startTime=" + startTime + - ", currentSequences=" + currentSequences + + ", checkpointSequences=" + currentSequences + '}'; } } @@ -2378,7 +2387,7 @@ public void handle() throws ExecutionException, InterruptedException final TaskGroup taskGroup = taskGroups.get(taskGroupId); if (isValidTaskGroup(taskGroupId, taskGroup)) { - final TreeMap> checkpoints = taskGroup.currentSequences; + final TreeMap> checkpoints = taskGroup.checkpointSequences; // check validity of previousCheckpoint int index = checkpoints.size(); @@ -2430,4 +2439,11 @@ protected boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGrou } } + private SequenceNumber makeSequenceNumber(T2 seq) + { + return makeSequenceNumber(seq, false, false); + } + + protected abstract SequenceNumber makeSequenceNumber(T2 seq, boolean useExclusive, boolean isExclusive); + } From 35bf8ee3c23aff6150535377218133c93e44cf26 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 27 Sep 2018 17:43:18 -0700 Subject: [PATCH 13/87] refactored kinesisindexingtask --- .../kafka/supervisor/KafkaSupervisor.java | 11 + .../imply-kinesis-lib/pom.xml | 35 - .../kinesis-indexing-service/pom.xml | 46 - .../kinesis-indexing-service/pom.xml | 21 +- .../kinesis/KinesisDataSourceMetadata.java | 0 .../indexing/kinesis/KinesisIOConfig.java | 0 .../indexing/kinesis/KinesisIndexTask.java | 7 +- .../kinesis/KinesisIndexTaskClient.java | 0 .../KinesisIndexTaskClientFactory.java | 0 .../kinesis/KinesisIndexTaskRunner.java | 101 + .../kinesis/KinesisIndexingServiceModule.java | 0 .../indexing/kinesis/KinesisPartitions.java | 0 .../kinesis/KinesisRecordSupplier.java | 0 .../druid/indexing/kinesis/KinesisRegion.java | 0 .../kinesis/KinesisSequenceNumber.java | 0 .../indexing/kinesis/KinesisTuningConfig.java | 0 .../ConstructibleAWSCredentialsConfig.java | 0 .../kinesis/supervisor/KinesisSupervisor.java | 58 + .../supervisor/KinesisSupervisorIOConfig.java | 0 .../KinesisSupervisorReportPayload.java | 0 .../supervisor/KinesisSupervisorSpec.java | 0 .../KinesisSupervisorTuningConfig.java | 0 ...rg.apache.druid.initialization.DruidModule | 0 .../KinesisDataSourceMetadataTest.java | 0 .../indexing/kinesis/KinesisIOConfigTest.java | 0 .../kinesis/KinesisIndexTaskClientTest.java | 0 .../kinesis/KinesisIndexTaskTest.java | 2022 +++++++++-------- .../kinesis/KinesisTuningConfigTest.java | 0 .../KinesisSupervisorIOConfigTest.java | 0 .../supervisor/KinesisSupervisorTest.java | 0 .../KinesisSupervisorTuningConfigTest.java | 0 .../indexing/kinesis/test/TestBroker.java | 0 .../supervisor/SeekableStreamSupervisor.java | 20 +- 33 files changed, 1238 insertions(+), 1083 deletions(-) delete mode 100644 extensions-core/kinesis-indexing-service/imply-kinesis-lib/pom.xml delete mode 100644 extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java (99%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java (100%) create mode 100644 extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java (100%) rename extensions-core/kinesis-indexing-service/{imply-kinesis-lib => }/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java (100%) rename extensions-core/kinesis-indexing-service/{imply-kinesis-lib => }/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java (100%) rename extensions-core/kinesis-indexing-service/{imply-kinesis-lib => }/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java (90%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java (57%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java (100%) rename extensions-core/kinesis-indexing-service/{kinesis-indexing-service => }/src/test/java/org/apache/druid/indexing/kinesis/test/TestBroker.java (100%) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 2bb39fef0104..b092daa9cfec 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -58,6 +58,7 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -419,5 +420,15 @@ protected int getNoticesQueueSize() return super.getNoticesQueueSize(); } + @Override + protected boolean checkSequenceAvailability( + @NotNull Integer partition, @NotNull Long sequenceFromMetadata + ) throws TimeoutException + { + Long latestOffset = getOffsetFromStreamForPartition(partition, false); + return latestOffset != null + && KafkaSequenceNumber.of(latestOffset).compareTo(KafkaSequenceNumber.of(sequenceFromMetadata)) >= 0; + } + } diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/pom.xml b/extensions-core/kinesis-indexing-service/imply-kinesis-lib/pom.xml deleted file mode 100644 index 39487c7f5ed0..000000000000 --- a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/pom.xml +++ /dev/null @@ -1,35 +0,0 @@ - - - - - 4.0.0 - - druid-imply-kinesis-lib - druid-imply-kinesis-lib - druid-imply-kinesis-lib - - - org.apache.druid.extensions - druid-kinesis-indexing-service - 0.13.0-SNAPSHOT - ../pom.xml - - diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml deleted file mode 100644 index 25707bd0f16d..000000000000 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/pom.xml +++ /dev/null @@ -1,46 +0,0 @@ - - - - - 4.0.0 - - kinesis-indexing-service - kinesis-indexing-service - kinesis-indexing-service - - - org.apache.druid.extensions - druid-kinesis-indexing-service - 0.13.0-SNAPSHOT - - - - - org.apache.druid.extensions - druid-imply-kinesis-lib - ${project.parent.version} - - - org.easymock - easymock - - - diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index bde56f2832f4..c25c2616e0f9 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -42,11 +42,6 @@ 1.10.61 - - imply-kinesis-lib - kinesis-indexing-service - - org.apache.druid @@ -122,6 +117,10 @@ + + org.easymock + easymock + junit junit @@ -135,6 +134,13 @@ test-jar test + + org.apache.druid + druid-processing + ${project.parent.version} + test-jar + test + org.apache.druid druid-indexing-service @@ -142,6 +148,11 @@ test-jar test + + cloud.localstack + localstack-utils + 0.1.14 + diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java similarity index 99% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 4cba120bbfd8..3b3ed50b0322 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -119,6 +119,7 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; +// TODO: kinesis task read from startPartitions to endPartitions inclusive, whereas kafka is exclusive, should change behavior to that of kafka's public class KinesisIndexTask extends SeekableStreamIndexTask { public static final long PAUSE_FOREVER = -1L; @@ -134,6 +135,7 @@ public class KinesisIndexTask extends SeekableStreamIndexTask private final Map endOffsets = new ConcurrentHashMap<>(); private final Map lastOffsets = new ConcurrentHashMap<>(); + private final Map nextOffsets = new ConcurrentHashMap<>(); // should use this instead of lastOffsets private final KinesisIOConfig ioConfig; private final KinesisTuningConfig tuningConfig; private ObjectMapper mapper; @@ -492,15 +494,16 @@ public void run() } lastOffsets.put(record.getPartitionId(), record.getSequenceNumber()); - } + + } if ((lastOffsets.get(record.getPartitionId()).equals(endOffsets.get(record.getPartitionId())) || Record.END_OF_SHARD_MARKER.equals(lastOffsets.get(record.getPartitionId()))) && assignment.remove(record.getPartitionId())) { log.info("Finished reading stream[%s], partition[%s].", record.getStreamName(), record.getPartitionId()); assignPartitions(recordSupplier, topic, assignment); - stillReading = ioConfig.isPauseAfterRead() || !assignment.isEmpty(); + stillReading = !assignment.isEmpty(); } } } diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java new file mode 100644 index 000000000000..c97f6d2c5b4c --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.stats.RowIngestionMeters; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.realtime.appenderator.Appenderator; + +import javax.ws.rs.core.Response; +import java.util.Map; + +/** + * Kinesis indexing task runner supporting incremental segments publishing + */ +public class KinesisIndexTaskRunner implements SeekableStreamIndexTaskRunner +{ + private static final EmittingLogger log = new EmittingLogger(KinesisIndexTask.class); + + + + @Override + public Appenderator getAppenderator() + { + return null; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) + { + return null; + } + + @Override + public void stopGracefully() + { + + } + + @Override + public RowIngestionMeters getRowIngestionMeters() + { + return null; + } + + @Override + public SeekableStreamIndexTask.Status getStatus() + { + return null; + } + + @Override + public Map getCurrentOffsets() + { + return null; + } + + @Override + public Map getEndOffsets() + { + return null; + } + + @Override + public Response setEndOffsets(Map offsets, boolean finish) throws InterruptedException + { + return null; + } + + @Override + public Response pause() throws InterruptedException + { + return null; + } + + @Override + public void resume() throws InterruptedException + { + + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java similarity index 100% rename from extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java similarity index 100% rename from extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java diff --git a/extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java similarity index 100% rename from extensions-core/kinesis-indexing-service/imply-kinesis-lib/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java similarity index 90% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 18264aaa7a66..902fff206299 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -19,7 +19,9 @@ package org.apache.druid.indexing.kinesis.supervisor; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -51,13 +53,16 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeoutException; /** * Supervisor responsible for managing the KafkaIndexTasks for a single dataSource. At a high level, the class accepts a @@ -284,6 +289,59 @@ protected SequenceNumber makeSequenceNumber( return KinesisSequenceNumber.of(seq, useExclusive, isExclusive); } + // the following are for unit testing purposes only + @Override + @VisibleForTesting + protected void runInternal() + throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException + { + super.runInternal(); + } + + @Override + @VisibleForTesting + protected Runnable updateCurrentAndLatestOffsets() + { + return super.updateCurrentAndLatestOffsets(); + } + + @Override + @VisibleForTesting + protected void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException + { + super.gracefulShutdownInternal(); + } + + @Override + @VisibleForTesting + protected void resetInternal(DataSourceMetadata dataSourceMetadata) + { + super.resetInternal(dataSourceMetadata); + } + + @Override + @VisibleForTesting + protected void moveTaskGroupToPendingCompletion(int taskGroupId) + { + super.moveTaskGroupToPendingCompletion(taskGroupId); + } + + @Override + @VisibleForTesting + protected int getNoticesQueueSize() + { + return super.getNoticesQueueSize(); + } + + @Override + protected boolean checkSequenceAvailability(String partition, @NotNull String sequenceFromMetadata) + throws TimeoutException + { + String earliestSequence = super.getOffsetFromStreamForPartition(partition, true); + return earliestSequence != null + && KinesisSequenceNumber.of(earliestSequence).compareTo(KinesisSequenceNumber.of(sequenceFromMetadata)) <= 0; + } + // TODO: Implement this for Kinesis which uses approximate time from latest instead of offset lag /* diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule rename to extensions-core/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java rename to extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java rename to extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java rename to extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java similarity index 57% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java rename to extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 74f708c40007..bdc94e9d1b96 100644 --- a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -1,430 +1,474 @@ -///* -// * Licensed to the Apache Software Foundation (ASF) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. The ASF licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, -// * software distributed under the License is distributed on an -// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// * KIND, either express or implied. See the License for the -// * specific language governing permissions and limitations -// * under the License. -// */ -// -//package org.apache.druid.indexing.kinesis; -// -//import com.fasterxml.jackson.core.type.TypeReference; -//import com.fasterxml.jackson.databind.Module; -//import com.fasterxml.jackson.databind.ObjectMapper; -//import com.google.common.base.Predicate; -//import com.google.common.base.Predicates; -//import com.google.common.base.Throwables; -//import com.google.common.collect.FluentIterable; -//import com.google.common.collect.ImmutableList; -//import com.google.common.collect.ImmutableMap; -//import com.google.common.collect.ImmutableSet; -//import com.google.common.collect.Iterables; -//import com.google.common.collect.Lists; -//import com.google.common.collect.Sets; -//import com.google.common.io.Files; -//import com.google.common.util.concurrent.ListenableFuture; -//import com.google.common.util.concurrent.ListeningExecutorService; -//import com.google.common.util.concurrent.MoreExecutors; -//import org.apache.curator.test.TestingCluster; -//import org.apache.druid.client.cache.CacheConfig; -//import org.apache.druid.client.cache.CachePopulatorStats; -//import org.apache.druid.client.cache.MapCache; -//import org.apache.druid.data.input.impl.DimensionsSpec; -//import org.apache.druid.data.input.impl.FloatDimensionSchema; -//import org.apache.druid.data.input.impl.JSONParseSpec; -//import org.apache.druid.data.input.impl.LongDimensionSchema; -//import org.apache.druid.data.input.impl.StringDimensionSchema; -//import org.apache.druid.data.input.impl.StringInputRowParser; -//import org.apache.druid.data.input.impl.TimestampSpec; -//import org.apache.druid.discovery.DataNodeService; -//import org.apache.druid.discovery.DruidNodeAnnouncer; -//import org.apache.druid.discovery.LookupNodeService; -//import org.apache.druid.indexer.TaskState; -//import org.apache.druid.indexer.TaskStatus; -//import org.apache.druid.indexing.common.Counters; -//import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; -//import org.apache.druid.indexing.common.SegmentLoaderFactory; -//import org.apache.druid.indexing.common.TaskLock; -//import org.apache.druid.indexing.common.TaskReport; -//import org.apache.druid.indexing.common.TaskReportFileWriter; -//import org.apache.druid.indexing.common.TaskToolbox; -//import org.apache.druid.indexing.common.TaskToolboxFactory; -//import org.apache.druid.indexing.common.TestUtils; -//import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; -//import org.apache.druid.indexing.common.actions.TaskActionClientFactory; -//import org.apache.druid.indexing.common.actions.TaskActionToolbox; -//import org.apache.druid.indexing.common.config.TaskConfig; -//import org.apache.druid.indexing.common.config.TaskStorageConfig; -//import org.apache.druid.indexing.common.stats.RowIngestionMeters; -//import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; -//import org.apache.druid.indexing.common.task.IndexTaskTest; -//import org.apache.druid.indexing.common.task.Task; -//import org.apache.druid.indexing.overlord.DataSourceMetadata; -//import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -//import org.apache.druid.indexing.overlord.MetadataTaskStorage; -//import org.apache.druid.indexing.overlord.TaskLockbox; -//import org.apache.druid.indexing.overlord.TaskStorage; -//import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; -//import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; -//import org.apache.druid.indexing.seekablestream.test.TestBroker; -//import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; -//import org.apache.druid.indexing.test.TestDataSegmentKiller; -//import org.apache.druid.java.util.common.CompressionUtils; -//import org.apache.druid.java.util.common.DateTimes; -//import org.apache.druid.java.util.common.ISE; -//import org.apache.druid.java.util.common.Intervals; -//import org.apache.druid.java.util.common.StringUtils; -//import org.apache.druid.java.util.common.concurrent.Execs; -//import org.apache.druid.java.util.common.granularity.Granularities; -//import org.apache.druid.java.util.common.logger.Logger; -//import org.apache.druid.java.util.common.parsers.JSONPathSpec; -//import org.apache.druid.java.util.emitter.EmittingLogger; -//import org.apache.druid.java.util.emitter.core.NoopEmitter; -//import org.apache.druid.java.util.emitter.service.ServiceEmitter; -//import org.apache.druid.java.util.metrics.MonitorScheduler; -//import org.apache.druid.math.expr.ExprMacroTable; -//import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; -//import org.apache.druid.metadata.EntryExistsException; -//import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; -//import org.apache.druid.metadata.TestDerbyConnector; -//import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; -//import org.apache.druid.query.Druids; -//import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; -//import org.apache.druid.query.Query; -//import org.apache.druid.query.QueryRunner; -//import org.apache.druid.query.QueryRunnerFactoryConglomerate; -//import org.apache.druid.query.QueryToolChest; -//import org.apache.druid.query.Result; -//import org.apache.druid.query.SegmentDescriptor; -//import org.apache.druid.query.aggregation.AggregatorFactory; -//import org.apache.druid.query.aggregation.CountAggregatorFactory; -//import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; -//import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -//import org.apache.druid.query.filter.SelectorDimFilter; -//import org.apache.druid.query.timeseries.TimeseriesQuery; -//import org.apache.druid.query.timeseries.TimeseriesQueryEngine; -//import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; -//import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; -//import org.apache.druid.query.timeseries.TimeseriesResultValue; -//import org.apache.druid.segment.DimensionHandlerUtils; -//import org.apache.druid.segment.IndexIO; -//import org.apache.druid.segment.QueryableIndex; -//import org.apache.druid.segment.TestHelper; -//import org.apache.druid.segment.column.DictionaryEncodedColumn; -//import org.apache.druid.segment.indexing.DataSchema; -//import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -//import org.apache.druid.segment.loading.DataSegmentPusher; -//import org.apache.druid.segment.loading.LocalDataSegmentPusher; -//import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; -//import org.apache.druid.segment.loading.SegmentLoaderConfig; -//import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager; -//import org.apache.druid.segment.loading.StorageLocationConfig; -//import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; -//import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; -//import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; -//import org.apache.druid.segment.transform.ExpressionTransform; -//import org.apache.druid.segment.transform.TransformSpec; -//import org.apache.druid.server.DruidNode; -//import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; -//import org.apache.druid.server.coordination.ServerType; -//import org.apache.druid.timeline.DataSegment; -//import org.apache.kafka.clients.producer.KafkaProducer; -//import org.apache.kafka.clients.producer.ProducerRecord; -//import org.easymock.EasyMock; -//import org.joda.time.Interval; -//import org.joda.time.Period; -//import org.junit.After; -//import org.junit.AfterClass; -//import org.junit.Assert; -//import org.junit.Before; -//import org.junit.BeforeClass; -//import org.junit.Rule; -//import org.junit.Test; -//import org.junit.rules.TemporaryFolder; -//import org.junit.runner.RunWith; -//import org.junit.runners.Parameterized; -// -//import javax.annotation.Nullable; -//import java.io.File; -//import java.io.IOException; -//import java.lang.reflect.InvocationTargetException; -//import java.lang.reflect.Method; -//import java.nio.charset.StandardCharsets; -//import java.util.Arrays; -//import java.util.HashMap; -//import java.util.List; -//import java.util.Map; -//import java.util.Objects; -//import java.util.Set; -//import java.util.TreeMap; -//import java.util.concurrent.Executor; -//import java.util.concurrent.Executors; -//import java.util.concurrent.TimeUnit; -//import java.util.concurrent.TimeoutException; -// -//import static org.apache.druid.query.QueryPlus.wrap; -// -//@RunWith(Parameterized.class) -//public class KinesisIndexTaskTest -//{ -// private static final Logger log = new Logger(KinesisIndexTask.class); -// private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); -// private static final long POLL_RETRY_MS = 100; -// -// private static TestingCluster zkServer; -// private static TestBroker kafkaServer; -// private static ServiceEmitter emitter; -// private static ListeningExecutorService taskExec; -// private static int topicPostfix; -// -// private final List runningTasks = Lists.newArrayList(); -// -// private long handoffConditionTimeout = 0; -// private boolean reportParseExceptions = false; -// private boolean logParseExceptions = true; -// private Integer maxParseExceptions = null; -// private Integer maxSavedParseExceptions = null; -// private boolean resetOffsetAutomatically = false; -// private boolean doHandoff = true; -// private Integer maxRowsPerSegment = null; -// private Long maxTotalRows = null; -// private Period intermediateHandoffPeriod = null; -// -// private TaskToolboxFactory toolboxFactory; -// private IndexerMetadataStorageCoordinator metadataStorageCoordinator; -// private TaskStorage taskStorage; -// private TaskLockbox taskLockbox; -// private File directory; -// private String topic; -// private List> records; -// private final boolean isIncrementalHandoffSupported; -// private final Set checkpointRequestsHash = Sets.newHashSet(); -// private File reportsFile; -// private RowIngestionMetersFactory rowIngestionMetersFactory; -// -// private int handoffCount = 0; -// -// // This should be removed in versions greater that 0.12.x -// // isIncrementalHandoffSupported should always be set to true in those later versions -// @Parameterized.Parameters(name = "isIncrementalHandoffSupported = {0}") -// public static Iterable constructorFeeder() -// { -// return ImmutableList.of(new Object[]{true}, new Object[]{false}); -// } -// -// public KafkaIndexTaskTest(boolean isIncrementalHandoffSupported) -// { -// this.isIncrementalHandoffSupported = isIncrementalHandoffSupported; -// } -// -// private static final DataSchema DATA_SCHEMA = new DataSchema( -// "test_ds", -// objectMapper.convertValue( -// new StringInputRowParser( -// new JSONParseSpec( -// new TimestampSpec("timestamp", "iso", null), -// new DimensionsSpec( -// Arrays.asList( -// new StringDimensionSchema("dim1"), -// new StringDimensionSchema("dim1t"), -// new StringDimensionSchema("dim2"), -// new LongDimensionSchema("dimLong"), -// new FloatDimensionSchema("dimFloat") -// ), -// null, -// null -// ), -// new JSONPathSpec(true, ImmutableList.of()), -// ImmutableMap.of() -// ), -// StandardCharsets.UTF_8.name() -// ), -// Map.class -// ), -// new AggregatorFactory[]{ -// new DoubleSumAggregatorFactory("met1sum", "met1"), -// new CountAggregatorFactory("rows") -// }, -// new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), -// null, -// objectMapper -// ); -// -// private static List> generateRecords(String topic) -// { -// return ImmutableList.of( -// new ProducerRecord<>(topic, 0, null, JB("2008", "a", "y", "10", "20.0", "1.0")), -// new ProducerRecord<>(topic, 0, null, JB("2009", "b", "y", "10", "20.0", "1.0")), -// new ProducerRecord<>(topic, 0, null, JB("2010", "c", "y", "10", "20.0", "1.0")), -// new ProducerRecord<>(topic, 0, null, JB("2011", "d", "y", "10", "20.0", "1.0")), -// new ProducerRecord<>(topic, 0, null, JB("2011", "e", "y", "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<>(topic, 0, null, StringUtils.toUtf8("unparseable")), -// new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")), -// new ProducerRecord<>(topic, 0, null, null), -// new ProducerRecord<>(topic, 0, null, JB("2013", "f", "y", "10", "20.0", "1.0")), -// new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "notanumber", "20.0", "1.0")), -// new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "10", "notanumber", "1.0")), -// new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "10", "20.0", "notanumber")), -// new ProducerRecord<>(topic, 1, null, JB("2012", "g", "y", "10", "20.0", "1.0")), -// new ProducerRecord<>(topic, 1, null, JB("2011", "h", "y", "10", "20.0", "1.0")) -// ); -// } -// -// private static String getTopicName() -// { -// return "topic" + topicPostfix++; -// } -// -// @Rule -// public final TemporaryFolder tempFolder = new TemporaryFolder(); -// -// @Rule -// public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); -// -// @BeforeClass -// public static void setupClass() throws Exception -// { -// emitter = new ServiceEmitter( -// "service", -// "host", -// new NoopEmitter() -// ); -// emitter.start(); -// EmittingLogger.registerEmitter(emitter); -// -// zkServer = new TestingCluster(1); -// zkServer.start(); -// -// kafkaServer = new TestBroker( -// zkServer.getConnectString(), -// null, -// 1, -// ImmutableMap.of("num.partitions", "2") -// ); -// kafkaServer.start(); -// -// taskExec = MoreExecutors.listeningDecorator( -// Executors.newCachedThreadPool( -// Execs.makeThreadFactory("kafka-task-test-%d") -// ) -// ); -// } -// -// @Before -// public void setupTest() throws IOException -// { -// handoffConditionTimeout = 0; -// reportParseExceptions = false; -// logParseExceptions = true; -// maxParseExceptions = null; -// maxSavedParseExceptions = null; -// doHandoff = true; -// topic = getTopicName(); -// records = generateRecords(topic); -// reportsFile = File.createTempFile("KafkaIndexTaskTestReports-" + System.currentTimeMillis(), "json"); -// makeToolboxFactory(); -// } -// -// @After -// public void tearDownTest() -// { -// synchronized (runningTasks) { -// for (Task task : runningTasks) { -// task.stopGracefully(); -// } -// -// runningTasks.clear(); -// } -// reportsFile.delete(); -// destroyToolboxFactory(); -// } -// -// @AfterClass -// public static void tearDownClass() throws Exception -// { -// taskExec.shutdown(); -// taskExec.awaitTermination(9999, TimeUnit.DAYS); -// -// kafkaServer.close(); -// kafkaServer = null; -// -// zkServer.stop(); -// zkServer = null; -// -// emitter.close(); -// } -// -// @Test(timeout = 60_000L) -// public void testRunAfterDataInserted() throws Exception -// { -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// final KafkaIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future = runTask(task); -// -// // Wait for task to exit -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); -// Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); -// } -// -// @Test(timeout = 60_000L) +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import cloud.localstack.Localstack; +import cloud.localstack.TestUtils; +import cloud.localstack.docker.LocalstackDockerTestRunner; +import cloud.localstack.docker.annotation.LocalstackDockerProperties; +import com.amazonaws.http.SdkHttpMetadata; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.model.GetRecordsRequest; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.PutRecordsRequest; +import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry; +import com.amazonaws.services.kinesis.model.Record; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import com.google.common.base.Throwables; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.io.Files; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.client.cache.MapCache; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.FloatDimensionSchema; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.discovery.DataNodeService; +import org.apache.druid.discovery.DruidNodeAnnouncer; +import org.apache.druid.discovery.LookupNodeService; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.Counters; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +import org.apache.druid.indexing.common.SegmentLoaderFactory; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.TaskReportFileWriter; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.TaskToolboxFactory; +import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; +import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.common.actions.TaskActionToolbox; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.config.TaskStorageConfig; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.MetadataTaskStorage; +import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; +import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; +import org.apache.druid.indexing.test.TestDataSegmentKiller; +import org.apache.druid.java.util.common.CompressionUtils; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.core.NoopEmitter; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.metrics.MonitorScheduler; +import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; +import org.apache.druid.metadata.EntryExistsException; +import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; +import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesQueryEngine; +import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.DictionaryEncodedColumn; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.loading.LocalDataSegmentPusher; +import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; +import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager; +import org.apache.druid.segment.loading.StorageLocationConfig; +import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; +import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; +import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.timeline.DataSegment; +import org.easymock.EasyMock; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +@RunWith(LocalstackDockerTestRunner.class) +@LocalstackDockerProperties(services = {"kinesis"}) +public class KinesisIndexTaskTest +{ + static { + /* + * Need to disable CBOR protocol, see: + * https://github.com/mhart/kinesalite/blob/master/README.md#cbor-protocol-issues-with-the-java-sdk + */ + TestUtils.setEnv("AWS_CBOR_DISABLE", "1"); + /* Disable SSL certificate checks for local testing */ + if (Localstack.useSSL()) { + TestUtils.disableSslCertChecking(); + } + } + + private static final Logger log = new Logger(KinesisIndexTaskTest.class); + private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + // private static final long POLL_RETRY_MS = 100; + private static int streamPosFix = 0; + + private static ServiceEmitter emitter; + private static ListeningExecutorService taskExec; + + private static final AmazonKinesis kinesis = TestUtils.getClientKinesis(); + private final List runningTasks = Lists.newArrayList(); + + private long handoffConditionTimeout = 0; + private boolean reportParseExceptions = false; + private boolean logParseExceptions = true; + private Integer maxParseExceptions = null; + private Integer maxSavedParseExceptions = null; + private boolean resetOffsetAutomatically = false; + private boolean doHandoff = true; + private Integer maxRowsPerSegment = null; + private Long maxTotalRows = null; + private Period intermediateHandoffPeriod = null; + + private TaskToolboxFactory toolboxFactory; + private IndexerMetadataStorageCoordinator metadataStorageCoordinator; + private TaskStorage taskStorage; + private TaskLockbox taskLockbox; + private File directory; + private String stream; + private PutRecordsRequest recordRequests; + private final boolean isIncrementalHandoffSupported = false; + private final Set checkpointRequestsHash = Sets.newHashSet(); + private File reportsFile; + private RowIngestionMetersFactory rowIngestionMetersFactory; + + private int handoffCount = 0; + + private static final DataSchema DATA_SCHEMA = new DataSchema( + "test_ds", + objectMapper.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ), + null, + null + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of() + ), + StandardCharsets.UTF_8.name() + ), + Map.class + ), + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + }, + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + null, + objectMapper + ); + + private static PutRecordsRequestEntry generateRequestEntry(String partition, String explicitHash, byte[] data) + { + return new PutRecordsRequestEntry().withPartitionKey(partition) + .withExplicitHashKey(explicitHash) + .withData(ByteBuffer.wrap(data)); + } + + private static PutRecordsRequest generateRecordsRequests(String stream) + { + return new PutRecordsRequest() + .withStreamName(stream) + .withRecords( + ImmutableList.of( + generateRequestEntry("0", "0", JB("2008", "a", "y", "10", "20.0", "1.0")), + generateRequestEntry("0", "0", JB("2009", "b", "y", "10", "20.0", "1.0")), + generateRequestEntry("0", "0", JB("2010", "c", "y", "10", "20.0", "1.0")), + generateRequestEntry("0", "0", JB("2011", "d", "y", "10", "20.0", "1.0")), + generateRequestEntry("0", "0", JB("2011", "e", "y", "10", "20.0", "1.0")), + generateRequestEntry("0", "0", JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")), + generateRequestEntry("0", "0", StringUtils.toUtf8("unparseable")), + generateRequestEntry("0", "0", StringUtils.toUtf8("unparseable2")), + generateRequestEntry("0", "0", "{}".getBytes()), + generateRequestEntry("0", "0", JB("2013", "f", "y", "10", "20.0", "1.0")), + generateRequestEntry("0", "0", JB("2049", "f", "y", "notanumber", "20.0", "1.0")), + generateRequestEntry("0", "0", JB("2049", "f", "y", "10", "notanumber", "1.0")), + generateRequestEntry("0", "0", JB("2049", "f", "y", "10", "20.0", "notanumber")), + generateRequestEntry("1", "1", JB("2012", "g", "y", "10", "20.0", "1.0")), + generateRequestEntry("1", "1", JB("2011", "h", "y", "10", "20.0", "1.0")) + ) + ); + } + + private static String getStreamName() + { + return "stream-" + streamPosFix++; + } + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @Rule + public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); + + @BeforeClass + public static void setupClass() + { + emitter = new ServiceEmitter( + "service", + "host", + new NoopEmitter() + ); + emitter.start(); + EmittingLogger.registerEmitter(emitter); + taskExec = MoreExecutors.listeningDecorator( + Executors.newCachedThreadPool( + Execs.makeThreadFactory("kafka-task-test-%d") + ) + ); + } + + @Before + public void setupTest() throws IOException, InterruptedException + { + handoffConditionTimeout = 0; + reportParseExceptions = false; + logParseExceptions = true; + maxParseExceptions = null; + maxSavedParseExceptions = null; + doHandoff = true; + stream = getStreamName(); + recordRequests = generateRecordsRequests(stream); + reportsFile = File.createTempFile("KinesisIndexTaskTestReports-" + System.currentTimeMillis(), "json"); + + // sleep required because of kinesalite + Thread.sleep(500); + makeToolboxFactory(); + } + + @After + public void tearDownTest() + { + synchronized (runningTasks) { + for (Task task : runningTasks) { + task.stopGracefully(); + } + + runningTasks.clear(); + } + reportsFile.delete(); + destroyToolboxFactory(); + } + + @AfterClass + public static void tearDownClass() throws Exception + { + taskExec.shutdown(); + taskExec.awaitTermination(9999, TimeUnit.DAYS); + emitter.close(); + } + + private String getSequenceNumber(AmazonKinesis kinesis, String shardId, ShardIteratorType itType, long recordsOffset) + throws Exception + { + String iterator = kinesis.getShardIterator(stream, shardId, itType.toString()).getShardIterator(); + Record record = null; + + while (iterator != null && recordsOffset >= 0) { + GetRecordsResult res = kinesis.getRecords(new GetRecordsRequest().withLimit(1).withShardIterator(iterator)); + List records = res.getRecords(); + if (!records.isEmpty()) { + record = records.get(0); + iterator = res.getNextShardIterator(); + } else { + throw new Exception("failed to get record for specified offset???"); + } + --recordsOffset; + } + + if (record == null) { + throw new Exception("record is null???"); + } + + return record.getSequenceNumber(); + } + + private static boolean isResponseOk(SdkHttpMetadata sdkHttpMetadata) + { + return sdkHttpMetadata.getHttpStatusCode() == 200; + } + + @Test(timeout = 60_000L) + public void testRunAfterDataInserted() throws Exception + { + // Insert data + SdkHttpMetadata createRes = kinesis.createStream(stream, 2).getSdkHttpMetadata(); + // sleep required because of kinesalite + Thread.sleep(500); + Assert.assertTrue(isResponseOk(createRes)); + + kinesis.putRecords(recordRequests); + + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + "0", + getSequenceNumber(kinesis, "0", ShardIteratorType.TRIM_HORIZON, 2) + )), + new KinesisPartitions(stream, ImmutableMap.of( + "0", + getSequenceNumber(kinesis, "0", ShardIteratorType.TRIM_HORIZON, 4) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata(new KinesisPartitions( + stream, + ImmutableMap.of( + "0", + getSequenceNumber( + kinesis, + "0", + ShardIteratorType.TRIM_HORIZON, + 4 + ) + ) + )), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + + SdkHttpMetadata delRes = kinesis.deleteStream(stream).getSdkHttpMetadata(); + Thread.sleep(500); + Assert.assertTrue(isResponseOk(delRes)); + } + + // @Test(timeout = 60_000L) // public void testRunBeforeDataInserted() throws Exception // { -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -460,7 +504,7 @@ // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -488,13 +532,13 @@ // Map consumerProps = kafkaServer.consumerProperties(); // consumerProps.put("max.poll.records", "1"); // -// final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L)); +// final KinesisPartitions startPartitions = new KinesisPartitions(stream, ImmutableMap.of(0, 0L, 1, 0L)); // // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering // // of events fetched across two partitions from Kafka -// final KafkaPartitions checkpoint1 = new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 0L)); -// final KafkaPartitions checkpoint2 = new KafkaPartitions(topic, ImmutableMap.of(0, 4L, 1, 2L)); -// final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L)); -// final KafkaIndexTask task = createTask( +// final KinesisPartitions checkpoint1 = new KinesisPartitions(stream, ImmutableMap.of(0, 5L, 1, 0L)); +// final KinesisPartitions checkpoint2 = new KinesisPartitions(stream, ImmutableMap.of(0, 4L, 1, 2L)); +// final KinesisPartitions endPartitions = new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 2L)); +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, @@ -524,8 +568,8 @@ // Objects.hash( // DATA_SCHEMA.getDataSource(), // 0, -// new KafkaDataSourceMetadata(startPartitions), -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets)) +// new KinesisDataSourceMetadata(startPartitions), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, currentOffsets)) // ) // ) // ); @@ -545,7 +589,7 @@ // SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 2L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -583,12 +627,12 @@ // Map consumerProps = kafkaServer.consumerProperties(); // consumerProps.put("max.poll.records", "1"); // -// final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L)); -// final KafkaPartitions checkpoint1 = new KafkaPartitions(topic, ImmutableMap.of(0, 3L, 1, 0L)); -// final KafkaPartitions checkpoint2 = new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 0L)); +// final KinesisPartitions startPartitions = new KinesisPartitions(stream, ImmutableMap.of(0, 0L, 1, 0L)); +// final KinesisPartitions checkpoint1 = new KinesisPartitions(stream, ImmutableMap.of(0, 3L, 1, 0L)); +// final KinesisPartitions checkpoint2 = new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 0L)); // -// final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L)); -// final KafkaIndexTask task = createTask( +// final KinesisPartitions endPartitions = new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 2L)); +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, @@ -632,8 +676,8 @@ // Objects.hash( // DATA_SCHEMA.getDataSource(), // 0, -// new KafkaDataSourceMetadata(startPartitions), -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets)) +// new KinesisDataSourceMetadata(startPartitions), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, currentOffsets)) // ) // ) // ); @@ -642,8 +686,8 @@ // Objects.hash( // DATA_SCHEMA.getDataSource(), // 0, -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets)), -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, nextOffsets)) +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, currentOffsets)), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, nextOffsets)) // ) // ) // ); @@ -663,7 +707,7 @@ // SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 2L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -700,11 +744,11 @@ // Map consumerProps = kafkaServer.consumerProperties(); // consumerProps.put("max.poll.records", "1"); // -// final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L)); +// final KinesisPartitions startPartitions = new KinesisPartitions(stream, ImmutableMap.of(0, 0L, 1, 0L)); // // Checkpointing will happen at checkpoint -// final KafkaPartitions checkpoint = new KafkaPartitions(topic, ImmutableMap.of(0, 1L, 1, 0L)); -// final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L)); -// final KafkaIndexTask task = createTask( +// final KinesisPartitions checkpoint = new KinesisPartitions(stream, ImmutableMap.of(0, 1L, 1, 0L)); +// final KinesisPartitions endPartitions = new KinesisPartitions(stream, ImmutableMap.of(0, 2L, 1, 0L)); +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, @@ -735,8 +779,8 @@ // Objects.hash( // DATA_SCHEMA.getDataSource(), // 0, -// new KafkaDataSourceMetadata(startPartitions), -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, checkpoint.getPartitionOffsetMap())) +// new KinesisDataSourceMetadata(startPartitions), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, checkpoint.getPartitionOffsetMap())) // ) // ) // ); @@ -751,7 +795,7 @@ // SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 2L, 1, 0L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -763,13 +807,13 @@ // @Test(timeout = 60_000L) // public void testRunWithMinimumMessageTime() throws Exception // { -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // DateTimes.of("2010"), @@ -805,7 +849,7 @@ // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -817,13 +861,13 @@ // @Test(timeout = 60_000L) // public void testRunWithMaximumMessageTime() throws Exception // { -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -860,7 +904,7 @@ // SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -873,7 +917,7 @@ // @Test(timeout = 60_000L) // public void testRunWithTransformSpec() throws Exception // { -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // DATA_SCHEMA.withTransformSpec( // new TransformSpec( @@ -886,8 +930,8 @@ // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -922,7 +966,7 @@ // SegmentDescriptor desc1 = SD(task, "2009/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -941,13 +985,13 @@ // } // } // -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), // kafkaServer.consumerProperties(), // true, // null, @@ -982,13 +1026,13 @@ // } // } // -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1012,7 +1056,7 @@ // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -1034,13 +1078,13 @@ // } // } // -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1064,7 +1108,7 @@ // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -1089,13 +1133,13 @@ // } // } // -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 7L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 7L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1133,13 +1177,13 @@ // } // } // -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 13L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 13L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1169,7 +1213,7 @@ // SegmentDescriptor desc4 = SD(task, "2049/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 13L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 13L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -1215,13 +1259,13 @@ // } // } // -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 10L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 10L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1275,13 +1319,13 @@ // @Test(timeout = 60_000L) // public void testRunReplicas() throws Exception // { -// final KafkaIndexTask task1 = createTask( +// final KinesisIndexTask task1 = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1289,13 +1333,13 @@ // false // ) // ); -// final KafkaIndexTask task2 = createTask( +// final KinesisIndexTask task2 = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1331,7 +1375,7 @@ // SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -1343,13 +1387,13 @@ // @Test(timeout = 60_000L) // public void testRunConflicting() throws Exception // { -// final KafkaIndexTask task1 = createTask( +// final KinesisIndexTask task1 = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1357,13 +1401,13 @@ // false // ) // ); -// final KafkaIndexTask task2 = createTask( +// final KinesisIndexTask task2 = createTask( // null, // new KafkaIOConfig( // 1, // "sequence1", -// new KafkaPartitions(topic, ImmutableMap.of(0, 3L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 10L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 3L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 10L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1400,7 +1444,7 @@ // SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -1412,13 +1456,13 @@ // @Test(timeout = 60_000L) // public void testRunConflictingWithoutTransactions() throws Exception // { -// final KafkaIndexTask task1 = createTask( +// final KinesisIndexTask task1 = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // false, // null, @@ -1426,13 +1470,13 @@ // false // ) // ); -// final KafkaIndexTask task2 = createTask( +// final KinesisIndexTask task2 = createTask( // null, // new KafkaIOConfig( // 1, // "sequence1", -// new KafkaPartitions(topic, ImmutableMap.of(0, 3L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 10L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 3L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 10L)), // kafkaServer.consumerProperties(), // false, // null, @@ -1486,13 +1530,13 @@ // @Test(timeout = 60_000L) // public void testRunOneTaskTwoPartitions() throws Exception // { -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L, 1, 0L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L, 1, 2L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1522,14 +1566,14 @@ // // Check published segments & metadata // SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); -// // desc3 will not be created in KafkaIndexTask (0.12.x) as it does not create per Kafka partition Druid segments +// // desc3 will not be created in KinesisIndexTask (0.12.x) as it does not create per Kafka partition Druid segments // SegmentDescriptor desc3 = SD(task, "2011/P1D", 1); // SegmentDescriptor desc4 = SD(task, "2012/P1D", 0); // Assert.assertEquals(isIncrementalHandoffSupported // ? ImmutableSet.of(desc1, desc2, desc4) // : ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 2L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L, 1, 2L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -1551,13 +1595,13 @@ // @Test(timeout = 60_000L) // public void testRunTwoTasksTwoPartitions() throws Exception // { -// final KafkaIndexTask task1 = createTask( +// final KinesisIndexTask task1 = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1565,13 +1609,13 @@ // false // ) // ); -// final KafkaIndexTask task2 = createTask( +// final KinesisIndexTask task2 = createTask( // null, // new KafkaIOConfig( // 1, // "sequence1", -// new KafkaPartitions(topic, ImmutableMap.of(1, 0L)), -// new KafkaPartitions(topic, ImmutableMap.of(1, 1L)), +// new KinesisPartitions(stream, ImmutableMap.of(1, 0L)), +// new KinesisPartitions(stream, ImmutableMap.of(1, 1L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1608,7 +1652,7 @@ // SegmentDescriptor desc3 = SD(task2, "2012/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 1L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L, 1, 1L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -1621,13 +1665,13 @@ // @Test(timeout = 60_000L) // public void testRestore() throws Exception // { -// final KafkaIndexTask task1 = createTask( +// final KinesisIndexTask task1 = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1658,13 +1702,13 @@ // Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); // // // Start a new task -// final KafkaIndexTask task2 = createTask( +// final KinesisIndexTask task2 = createTask( // task1.getId(), // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1698,7 +1742,7 @@ // SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -1710,13 +1754,13 @@ // @Test(timeout = 60_000L) // public void testRunWithPauseAndResume() throws Exception // { -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1782,7 +1826,7 @@ // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -1794,13 +1838,13 @@ // @Test(timeout = 60_000L) // public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception // { -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1833,13 +1877,13 @@ // } // } // -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", -// new KafkaPartitions(topic, ImmutableMap.of(0, 200L)), -// new KafkaPartitions(topic, ImmutableMap.of(0, 500L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 200L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 500L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1886,14 +1930,14 @@ // { // }).writeValueAsString(sequences)); // -// final KafkaIndexTask task = createTask( +// final KinesisIndexTask task = createTask( // null, // new KafkaIOConfig( // 0, // "sequence0", // // 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, 5L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), // kafkaServer.consumerProperties(), // true, // null, @@ -1918,7 +1962,7 @@ // SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); // Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); // Assert.assertEquals( -// new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), // metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) // ); // @@ -1926,420 +1970,428 @@ // Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); // Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); // } -// -// private ListenableFuture runTask(final Task task) -// { -// try { -// taskStorage.insert(task, TaskStatus.running(task.getId())); -// } -// catch (EntryExistsException e) { -// // suppress -// } -// taskLockbox.syncFromStorage(); -// final TaskToolbox toolbox = toolboxFactory.build(task); -// synchronized (runningTasks) { -// runningTasks.add(task); -// } -// return taskExec.submit( -// () -> { -// try { -// if (task.isReady(toolbox.getTaskActionClient())) { -// return task.run(toolbox); -// } else { -// throw new ISE("Task is not ready"); -// } -// } -// catch (Exception e) { -// log.warn(e, "Task failed"); -// return TaskStatus.failure(task.getId()); -// } -// } -// ); -// } -// -// private TaskLock getLock(final Task task, final Interval interval) -// { -// return Iterables.find( -// taskLockbox.findLocksForTask(task), -// new Predicate() -// { -// @Override -// public boolean apply(TaskLock lock) -// { -// return lock.getInterval().contains(interval); -// } -// } -// ); -// } -// -// private KafkaIndexTask createTask( -// final String taskId, -// final KafkaIOConfig ioConfig -// ) -// { -// return createTask(taskId, DATA_SCHEMA, ioConfig); -// } -// -// private KafkaIndexTask createTask( -// final String taskId, -// final KafkaIOConfig ioConfig, -// final Map context -// ) -// { -// return createTask(taskId, DATA_SCHEMA, ioConfig, context); -// } -// -// private KafkaIndexTask createTask( -// final String taskId, -// final DataSchema dataSchema, -// final KafkaIOConfig ioConfig -// ) -// { -// final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( -// 1000, -// null, -// maxRowsPerSegment, -// maxTotalRows, -// new Period("P1Y"), -// null, -// null, -// null, -// true, -// reportParseExceptions, -// handoffConditionTimeout, -// resetOffsetAutomatically, -// null, -// intermediateHandoffPeriod, -// logParseExceptions, -// maxParseExceptions, -// maxSavedParseExceptions -// ); -// final Map context = isIncrementalHandoffSupported -// ? ImmutableMap.of(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true) -// : null; -// final KafkaIndexTask task = new KafkaIndexTask( -// taskId, -// null, -// cloneDataSchema(dataSchema), -// tuningConfig, -// ioConfig, -// context, -// null, -// null, -// rowIngestionMetersFactory -// ); -// task.setPollRetryMs(POLL_RETRY_MS); -// return task; -// } -// -// -// private KafkaIndexTask createTask( -// final String taskId, -// final DataSchema dataSchema, -// final KafkaIOConfig ioConfig, -// final Map context -// ) -// { -// final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( -// 1000, -// null, -// maxRowsPerSegment, -// null, -// new Period("P1Y"), -// null, -// null, -// null, -// true, -// reportParseExceptions, -// handoffConditionTimeout, -// resetOffsetAutomatically, -// null, -// null, -// logParseExceptions, -// maxParseExceptions, -// maxSavedParseExceptions -// ); -// if (isIncrementalHandoffSupported) { -// context.put(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true); -// } -// -// final KafkaIndexTask task = new KafkaIndexTask( -// taskId, -// null, -// cloneDataSchema(dataSchema), -// tuningConfig, -// ioConfig, -// context, -// null, -// null, -// rowIngestionMetersFactory -// ); -// task.setPollRetryMs(POLL_RETRY_MS); -// return task; -// } -// -// private static DataSchema cloneDataSchema(final DataSchema dataSchema) -// { -// return new DataSchema( -// dataSchema.getDataSource(), -// dataSchema.getParserMap(), -// dataSchema.getAggregators(), -// dataSchema.getGranularitySpec(), -// dataSchema.getTransformSpec(), -// objectMapper -// ); -// } -// -// private QueryRunnerFactoryConglomerate makeTimeseriesOnlyConglomerate() -// { -// IntervalChunkingQueryRunnerDecorator queryRunnerDecorator = new IntervalChunkingQueryRunnerDecorator( -// null, -// null, -// null -// ) -// { -// @Override -// public QueryRunner decorate( -// QueryRunner delegate, QueryToolChest> toolChest -// ) -// { -// return delegate; -// } -// }; -// return new DefaultQueryRunnerFactoryConglomerate( -// ImmutableMap.of( -// TimeseriesQuery.class, -// new TimeseriesQueryRunnerFactory( -// new TimeseriesQueryQueryToolChest(queryRunnerDecorator), -// new TimeseriesQueryEngine(), -// (query, future) -> { -// // do nothing -// } -// ) -// ) -// ); -// } -// -// private void makeToolboxFactory() throws IOException -// { -// directory = tempFolder.newFolder(); -// final TestUtils testUtils = new TestUtils(); -// rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory(); -// final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); -// for (Module module : new KafkaIndexTaskModule().getJacksonModules()) { -// objectMapper.registerModule(module); -// } -// final TaskConfig taskConfig = new TaskConfig( -// new File(directory, "taskBaseDir").getPath(), -// null, -// null, -// 50000, -// null, -// false, -// null, -// null -// ); -// final TestDerbyConnector derbyConnector = derby.getConnector(); -// derbyConnector.createDataSourceTable(); -// derbyConnector.createPendingSegmentsTable(); -// derbyConnector.createSegmentTable(); -// derbyConnector.createRulesTable(); -// derbyConnector.createConfigTable(); -// derbyConnector.createTaskTables(); -// derbyConnector.createAuditTable(); -// taskStorage = new MetadataTaskStorage( -// derbyConnector, -// new TaskStorageConfig(null), -// new DerbyMetadataStorageActionHandlerFactory( -// derbyConnector, -// derby.metadataTablesConfigSupplier().get(), -// objectMapper -// ) -// ); -// metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( -// testUtils.getTestObjectMapper(), -// derby.metadataTablesConfigSupplier().get(), -// derbyConnector -// ); -// taskLockbox = new TaskLockbox(taskStorage); -// final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( -// taskLockbox, -// taskStorage, -// metadataStorageCoordinator, -// emitter, -// new SupervisorManager(null) -// { -// @Override -// public boolean checkPointDataSourceMetadata( -// String supervisorId, -// @Nullable Integer taskGroupId, -// String baseSequenceName, -// @Nullable DataSourceMetadata previousDataSourceMetadata, -// @Nullable DataSourceMetadata currentDataSourceMetadata -// ) -// { -// log.info("Adding checkpoint hash to the set"); -// checkpointRequestsHash.add( -// Objects.hash( -// supervisorId, -// taskGroupId, -// previousDataSourceMetadata, -// currentDataSourceMetadata -// ) -// ); -// return true; -// } -// }, -// new Counters() -// ); -// final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( -// taskStorage, -// taskActionToolbox -// ); -// final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier() -// { -// @Override -// public boolean registerSegmentHandoffCallback( -// SegmentDescriptor descriptor, Executor exec, Runnable handOffRunnable -// ) -// { -// if (doHandoff) { -// // Simulate immediate handoff -// exec.execute(handOffRunnable); -// } -// return true; -// } -// -// @Override -// public void start() -// { -// //Noop -// } -// -// @Override -// public void close() -// { -// //Noop -// } -// }; -// final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig(); -// dataSegmentPusherConfig.storageDirectory = getSegmentDirectory(); -// final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig, objectMapper); -// SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig() -// { -// @Override -// public List getLocations() -// { -// return Lists.newArrayList(); -// } -// }; -// toolboxFactory = new TaskToolboxFactory( -// taskConfig, -// taskActionClientFactory, -// emitter, -// dataSegmentPusher, -// new TestDataSegmentKiller(), -// null, // DataSegmentMover -// null, // DataSegmentArchiver -// new TestDataSegmentAnnouncer(), -// EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), -// handoffNotifierFactory, -// this::makeTimeseriesOnlyConglomerate, -// MoreExecutors.sameThreadExecutor(), // queryExecutorService -// EasyMock.createMock(MonitorScheduler.class), -// new SegmentLoaderFactory( -// new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper()) -// ), -// testUtils.getTestObjectMapper(), -// testUtils.getTestIndexIO(), -// MapCache.create(1024), -// new CacheConfig(), -// new CachePopulatorStats(), -// testUtils.getTestIndexMergerV9(), -// EasyMock.createNiceMock(DruidNodeAnnouncer.class), -// EasyMock.createNiceMock(DruidNode.class), -// new LookupNodeService("tier"), -// new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), -// new TaskReportFileWriter(reportsFile) -// ); -// } -// -// private void destroyToolboxFactory() -// { -// toolboxFactory = null; -// taskStorage = null; -// taskLockbox = null; -// metadataStorageCoordinator = null; -// } -// -// private Set publishedDescriptors() -// { -// return FluentIterable.from( -// metadataStorageCoordinator.getUsedSegmentsForInterval( -// DATA_SCHEMA.getDataSource(), -// Intervals.of("0000/3000") -// ) -// ).transform(DataSegment::toDescriptor).toSet(); -// } -// -// private void unlockAppenderatorBasePersistDirForTask(KafkaIndexTask task) -// throws NoSuchMethodException, InvocationTargetException, IllegalAccessException -// { -// Method unlockBasePersistDir = ((AppenderatorImpl) task.getAppenderator()).getClass() -// .getDeclaredMethod( -// "unlockBasePersistDirectory"); -// unlockBasePersistDir.setAccessible(true); -// unlockBasePersistDir.invoke(task.getAppenderator()); -// } -// -// private File getSegmentDirectory() -// { -// return new File(directory, "segments"); -// } -// -// private List readSegmentColumn(final String column, final SegmentDescriptor descriptor) throws IOException -// { -// File indexBasePath = new File( -// StringUtils.format( -// "%s/%s/%s_%s/%s/%d", -// getSegmentDirectory(), -// DATA_SCHEMA.getDataSource(), -// descriptor.getInterval().getStart(), -// descriptor.getInterval().getEnd(), -// descriptor.getVersion(), -// descriptor.getPartitionNumber() -// ) -// ); -// -// File outputLocation = new File( -// directory, -// StringUtils.format( -// "%s_%s_%s_%s", -// descriptor.getInterval().getStart(), -// descriptor.getInterval().getEnd(), -// descriptor.getVersion(), -// descriptor.getPartitionNumber() -// ) -// ); -// outputLocation.mkdir(); -// CompressionUtils.unzip( -// Files.asByteSource(new File(indexBasePath.listFiles()[0], "index.zip")), -// outputLocation, -// Predicates.alwaysFalse(), -// false -// ); -// IndexIO indexIO = new TestUtils().getTestIndexIO(); -// QueryableIndex index = indexIO.loadIndex(outputLocation); -// DictionaryEncodedColumn theColumn = index.getColumn(column).getDictionaryEncoding(); -// List values = Lists.newArrayList(); -// for (int i = 0; i < theColumn.length(); i++) { -// int id = theColumn.getSingleValueRow(i); -// String value = theColumn.lookupName(id); -// values.add(value); -// } -// return values; -// } -// + + private ListenableFuture runTask(final Task task) + { + try { + taskStorage.insert(task, TaskStatus.running(task.getId())); + } + catch (EntryExistsException e) { + // suppress + } + taskLockbox.syncFromStorage(); + final TaskToolbox toolbox = toolboxFactory.build(task); + synchronized (runningTasks) { + runningTasks.add(task); + } + return taskExec.submit( + () -> { + try { + if (task.isReady(toolbox.getTaskActionClient())) { + return task.run(toolbox); + } else { + throw new ISE("Task is not ready"); + } + } + catch (Exception e) { + log.warn(e, "Task failed"); + return TaskStatus.failure(task.getId()); + } + } + ); + } + + + private TaskLock getLock(final Task task, final Interval interval) + { + return Iterables.find( + taskLockbox.findLocksForTask(task), + new Predicate() + { + @Override + public boolean apply(TaskLock lock) + { + return lock.getInterval().contains(interval); + } + } + ); + } + + private KinesisIndexTask createTask( + final String taskId, + final KinesisIOConfig ioConfig + ) + { + return createTask(taskId, DATA_SCHEMA, ioConfig); + } + + private KinesisIndexTask createTask( + final String taskId, + final KinesisIOConfig ioConfig, + final Map context + ) + { + return createTask(taskId, DATA_SCHEMA, ioConfig, context); + } + + private KinesisIndexTask createTask( + final String taskId, + final DataSchema dataSchema, + final KinesisIOConfig ioConfig + ) + { + final KinesisTuningConfig tuningConfig = new KinesisTuningConfig( + 1000, + null, + null, + new Period("P1Y"), + null, + null, + null, + true, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + false, + null, + null, + null, + null, + null, + null, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions + ); + final Map context = null; + final KinesisIndexTask task = new KinesisIndexTask( + taskId, + null, + cloneDataSchema(dataSchema), + tuningConfig, + ioConfig, + context, + null, + null, + rowIngestionMetersFactory + ); + return task; + } + + + private KinesisIndexTask createTask( + final String taskId, + final DataSchema dataSchema, + final KinesisIOConfig ioConfig, + final Map context + ) + { + final KinesisTuningConfig tuningConfig = new KinesisTuningConfig( + 1000, + null, + null, + new Period("P1Y"), + null, + null, + null, + true, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + null, + null, + null, + null, + null, + null, + null, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions + ); + if (isIncrementalHandoffSupported) { + context.put(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true); + } + + final KinesisIndexTask task = new KinesisIndexTask( + taskId, + null, + cloneDataSchema(dataSchema), + tuningConfig, + ioConfig, + context, + null, + null, + rowIngestionMetersFactory + ); + return task; + } + + private static DataSchema cloneDataSchema(final DataSchema dataSchema) + { + return new DataSchema( + dataSchema.getDataSource(), + dataSchema.getParserMap(), + dataSchema.getAggregators(), + dataSchema.getGranularitySpec(), + dataSchema.getTransformSpec(), + objectMapper + ); + } + + private QueryRunnerFactoryConglomerate makeTimeseriesOnlyConglomerate() + { + IntervalChunkingQueryRunnerDecorator queryRunnerDecorator = new IntervalChunkingQueryRunnerDecorator( + null, + null, + null + ) + { + @Override + public QueryRunner decorate( + QueryRunner delegate, QueryToolChest> toolChest + ) + { + return delegate; + } + }; + return new DefaultQueryRunnerFactoryConglomerate( + ImmutableMap.of( + TimeseriesQuery.class, + new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest(queryRunnerDecorator), + new TimeseriesQueryEngine(), + (query, future) -> { + // do nothing + } + ) + ) + ); + } + + private void makeToolboxFactory() throws IOException + { + directory = tempFolder.newFolder(); + final org.apache.druid.indexing.common.TestUtils testUtils = new org.apache.druid.indexing.common.TestUtils(); + rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory(); + final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); + for (Module module : new KinesisIndexingServiceModule().getJacksonModules()) { + objectMapper.registerModule(module); + } + final TaskConfig taskConfig = new TaskConfig( + new File(directory, "taskBaseDir").getPath(), + null, + null, + 50000, + null, + false, + null, + null + ); + final TestDerbyConnector derbyConnector = derby.getConnector(); + derbyConnector.createDataSourceTable(); + derbyConnector.createPendingSegmentsTable(); + derbyConnector.createSegmentTable(); + derbyConnector.createRulesTable(); + derbyConnector.createConfigTable(); + derbyConnector.createTaskTables(); + derbyConnector.createAuditTable(); + taskStorage = new MetadataTaskStorage( + derbyConnector, + new TaskStorageConfig(null), + new DerbyMetadataStorageActionHandlerFactory( + derbyConnector, + derby.metadataTablesConfigSupplier().get(), + objectMapper + ) + ); + metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( + testUtils.getTestObjectMapper(), + derby.metadataTablesConfigSupplier().get(), + derbyConnector + ); + taskLockbox = new TaskLockbox(taskStorage); + final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( + taskLockbox, + taskStorage, + metadataStorageCoordinator, + emitter, + new SupervisorManager(null) + { + @Override + public boolean checkPointDataSourceMetadata( + String supervisorId, + @Nullable Integer taskGroupId, + String baseSequenceName, + @Nullable DataSourceMetadata previousDataSourceMetadata, + @Nullable DataSourceMetadata currentDataSourceMetadata + ) + { + log.info("Adding checkpoint hash to the set"); + checkpointRequestsHash.add( + Objects.hash( + supervisorId, + taskGroupId, + previousDataSourceMetadata, + currentDataSourceMetadata + ) + ); + return true; + } + }, + new Counters() + ); + final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( + taskStorage, + taskActionToolbox + ); + final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier() + { + @Override + public boolean registerSegmentHandoffCallback( + SegmentDescriptor descriptor, Executor exec, Runnable handOffRunnable + ) + { + if (doHandoff) { + // Simulate immediate handoff + exec.execute(handOffRunnable); + } + return true; + } + + @Override + public void start() + { + //Noop + } + + @Override + public void close() + { + //Noop + } + }; + final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig(); + dataSegmentPusherConfig.storageDirectory = getSegmentDirectory(); + final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig, objectMapper); + SegmentLoaderConfig segmentLoaderConfig = new SegmentLoaderConfig() + { + @Override + public List getLocations() + { + return Lists.newArrayList(); + } + }; + toolboxFactory = new TaskToolboxFactory( + taskConfig, + taskActionClientFactory, + emitter, + dataSegmentPusher, + new TestDataSegmentKiller(), + null, // DataSegmentMover + null, // DataSegmentArchiver + new TestDataSegmentAnnouncer(), + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), + handoffNotifierFactory, + this::makeTimeseriesOnlyConglomerate, + MoreExecutors.sameThreadExecutor(), // queryExecutorService + EasyMock.createMock(MonitorScheduler.class), + new SegmentLoaderFactory( + new SegmentLoaderLocalCacheManager(null, segmentLoaderConfig, testUtils.getTestObjectMapper()) + ), + testUtils.getTestObjectMapper(), + testUtils.getTestIndexIO(), + MapCache.create(1024), + new CacheConfig(), + new CachePopulatorStats(), + testUtils.getTestIndexMergerV9(), + EasyMock.createNiceMock(DruidNodeAnnouncer.class), + EasyMock.createNiceMock(DruidNode.class), + new LookupNodeService("tier"), + new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), + new TaskReportFileWriter(reportsFile) + ); + } + + private void destroyToolboxFactory() + { + toolboxFactory = null; + taskStorage = null; + taskLockbox = null; + metadataStorageCoordinator = null; + } + + + private Set publishedDescriptors() + { + return FluentIterable.from( + metadataStorageCoordinator.getUsedSegmentsForInterval( + DATA_SCHEMA.getDataSource(), + Intervals.of("0000/3000") + ) + ).transform(DataSegment::toDescriptor).toSet(); + } + + private void unlockAppenderatorBasePersistDirForTask(KinesisIndexTask task) + throws NoSuchMethodException, InvocationTargetException, IllegalAccessException + { + Method unlockBasePersistDir = ((AppenderatorImpl) task.getAppenderator()).getClass() + .getDeclaredMethod( + "unlockBasePersistDirectory"); + unlockBasePersistDir.setAccessible(true); + unlockBasePersistDir.invoke(task.getAppenderator()); + } + + private File getSegmentDirectory() + { + return new File(directory, "segments"); + } + + + private List readSegmentColumn(final String column, final SegmentDescriptor descriptor) throws IOException + { + File indexBasePath = new File( + StringUtils.format( + "%s/%s/%s_%s/%s/%d", + getSegmentDirectory(), + DATA_SCHEMA.getDataSource(), + descriptor.getInterval().getStart(), + descriptor.getInterval().getEnd(), + descriptor.getVersion(), + descriptor.getPartitionNumber() + ) + ); + + File outputLocation = new File( + directory, + StringUtils.format( + "%s_%s_%s_%s", + descriptor.getInterval().getStart(), + descriptor.getInterval().getEnd(), + descriptor.getVersion(), + descriptor.getPartitionNumber() + ) + ); + outputLocation.mkdir(); + CompressionUtils.unzip( + Files.asByteSource(new File(indexBasePath.listFiles()[0], "index.zip")), + outputLocation, + Predicates.alwaysFalse(), + false + ); + IndexIO indexIO = new org.apache.druid.indexing.common.TestUtils().getTestIndexIO(); + QueryableIndex index = indexIO.loadIndex(outputLocation); + DictionaryEncodedColumn theColumn = index.getColumn(column).getDictionaryEncoding(); + List values = Lists.newArrayList(); + for (int i = 0; i < theColumn.length(); i++) { + int id = theColumn.getSingleValueRow(i); + String value = theColumn.lookupName(id); + values.add(value); + } + return values; + } + + // // public long countEvents(final Task task) // { // // Do a query. @@ -2359,41 +2411,41 @@ // return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); // } // -// private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) -// { -// try { -// return new ObjectMapper().writeValueAsBytes( -// ImmutableMap.builder() -// .put("timestamp", timestamp) -// .put("dim1", dim1) -// .put("dim2", dim2) -// .put("dimLong", dimLong) -// .put("dimFloat", dimFloat) -// .put("met1", met1) -// .build() -// ); -// } -// catch (Exception e) { -// throw Throwables.propagate(e); -// } -// } -// -// private SegmentDescriptor SD(final Task task, final String intervalString, final int partitionNum) -// { -// final Interval interval = Intervals.of(intervalString); -// return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum); -// } -// -// private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException -// { -// Map taskReports = objectMapper.readValue( -// reportsFile, -// new TypeReference>() -// { -// } -// ); -// return IngestionStatsAndErrorsTaskReportData.getPayloadFromTaskReports( -// taskReports -// ); -// } -//} + private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) + { + try { + return new ObjectMapper().writeValueAsBytes( + ImmutableMap.builder() + .put("timestamp", timestamp) + .put("dim1", dim1) + .put("dim2", dim2) + .put("dimLong", dimLong) + .put("dimFloat", dimFloat) + .put("met1", met1) + .build() + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + private SegmentDescriptor SD(final Task task, final String intervalString, final int partitionNum) + { + final Interval interval = Intervals.of(intervalString); + return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum); + } + + private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException + { + Map taskReports = objectMapper.readValue( + reportsFile, + new TypeReference>() + { + } + ); + return IngestionStatsAndErrorsTaskReportData.getPayloadFromTaskReports( + taskReports + ); + } +} diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java rename to extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java rename to extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java rename to extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java rename to extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java diff --git a/extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestBroker.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestBroker.java similarity index 100% rename from extensions-core/kinesis-indexing-service/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestBroker.java rename to extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestBroker.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 471f7ea69bff..05a85e54cde0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -78,6 +78,7 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -1906,6 +1907,9 @@ private ImmutableMap> generateStartingSequencesForPartiti return builder.build(); } + protected abstract boolean checkSequenceAvailability(@NotNull T1 partition, @NotNull T2 sequenceFromMetadata) + throws TimeoutException; + /** * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't * find any data, it will retrieve the latest or earliest Kafka/Kinesis offset depending on the useEarliestOffset config. @@ -1918,26 +1922,22 @@ private SequenceNumber getOffsetFromStorageForPartition(T1 partition) throws log.debug("Getting offset [%s] from metadata storage for partition [%s]", offset, partition); if (!taskTuningConfig.isSkipSequenceNumberAvailabilityCheck()) { try { - T2 latestSequence = getOffsetFromStreamForPartition(partition, false); - - if (latestSequence == null || makeSequenceNumber(offset).compareTo(makeSequenceNumber(latestSequence)) > 0) { + if (!checkSequenceAvailability(partition, offset)) { if (taskTuningConfig.isResetOffsetAutomatically()) { resetInternal( createDataSourceMetaData(ioConfig.getId(), ImmutableMap.of(partition, offset)) ); throw new ISE( - "Previous sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) - automatically resetting offset", + "Previous sequenceNumber [%s] is no longer available for partition [%s] - automatically resetting offset", offset, - partition, - latestSequence + partition ); } else { throw new ISE( - "Previous sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]). You can clear the previous sequenceNumber and start reading from a valid message by using the supervisor's reset API.", + "Previous sequenceNumber [%s] is no longer available for partition [%s]. You can clear the previous sequenceNumber and start reading from a valid message by using the supervisor's reset API.", offset, - partition, - latestSequence + partition ); } } @@ -1987,7 +1987,7 @@ && checkSourceMetaDataMatch(dataSourceMetadata)) { return Collections.emptyMap(); } - private T2 getOffsetFromStreamForPartition(T1 partition, boolean useEarliestOffset) throws TimeoutException + protected T2 getOffsetFromStreamForPartition(T1 partition, boolean useEarliestOffset) throws TimeoutException { synchronized (recordSupplierLock) { StreamPartition topicPartition = new StreamPartition<>(ioConfig.getId(), partition); From 7313dbe32908097d15451620447a66ed49c85c1a Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 2 Oct 2018 18:15:17 -0700 Subject: [PATCH 14/87] finished up more kinesis unit tests --- .../indexing/kafka/KafkaRecordSupplier.java | 1 + .../indexing/kinesis/KinesisIndexTask.java | 73 +- .../kinesis/KinesisRecordSupplier.java | 6 +- .../indexing/kinesis/KinesisTuningConfig.java | 2 +- .../indexing/kinesis/KinesisIOConfigTest.java | 2 +- .../kinesis/KinesisIndexTaskTest.java | 3113 ++++++++-------- .../supervisor/KinesisSupervisorTest.java | 3186 +++++++++++------ .../KinesisSupervisorTuningConfigTest.java | 4 +- .../seekablestream/common/RecordSupplier.java | 2 +- 9 files changed, 3636 insertions(+), 2753 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 6f68284d43cf..af18b2338e4c 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -39,6 +39,7 @@ import java.util.Set; import java.util.stream.Collectors; +// TODO: in the future, should refactor to do smt similar to KinesisRecordSupplier public class KafkaRecordSupplier implements RecordSupplier { private static final Random RANDOM = new Random(); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 3b3ed50b0322..4e48eb03b873 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -42,7 +42,10 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; +import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.ResetDataSourceMetadataAction; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; @@ -90,6 +93,7 @@ import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; +import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; import javax.ws.rs.DefaultValue; @@ -101,7 +105,6 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -135,7 +138,6 @@ public class KinesisIndexTask extends SeekableStreamIndexTask private final Map endOffsets = new ConcurrentHashMap<>(); private final Map lastOffsets = new ConcurrentHashMap<>(); - private final Map nextOffsets = new ConcurrentHashMap<>(); // should use this instead of lastOffsets private final KinesisIOConfig ioConfig; private final KinesisTuningConfig tuningConfig; private ObjectMapper mapper; @@ -264,13 +266,14 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ); try ( - final Appenderator appenderator0 = newAppenderator(fireDepartmentMetrics, toolbox); - final StreamAppenderatorDriver driver = newDriver(appenderator0, toolbox, fireDepartmentMetrics); final RecordSupplier recordSupplier = getRecordSupplier() ) { toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); toolbox.getDataSegmentServerAnnouncer().announce(); + final Appenderator appenderator0 = newAppenderator(fireDepartmentMetrics, toolbox); + final StreamAppenderatorDriver driver = newDriver(appenderator0, toolbox, fireDepartmentMetrics); + appenderator = appenderator0; final String topic = ioConfig.getStartPartitions().getId(); @@ -359,7 +362,7 @@ public void run() // Main loop. // Could eventually support leader/follower mode (for keeping replicas more in sync) - boolean stillReading = ioConfig.isPauseAfterRead() || !assignment.isEmpty(); + boolean stillReading = !assignment.isEmpty(); status = Status.READING; try { while (stillReading) { @@ -507,7 +510,17 @@ public void run() } } } + catch (Exception e) { + log.error(e, "Encountered exception while running task."); + final String errorMsg = Throwables.getStackTraceAsString(e); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(errorMsg)); + return TaskStatus.failure( + getId(), + errorMsg + ); + } finally { + log.info("Persisting all pending data"); driver.persist(committerSupplier.get()); // persist pending data } @@ -587,6 +600,13 @@ public String apply(DataSegment input) } } catch (InterruptedException | RejectedExecutionException e) { + final String errorMsg = Throwables.getStackTraceAsString(e); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(errorMsg)); + + if (appenderator != null) { + appenderator.closeNow(); + } + // handle the InterruptedException that gets wrapped in a RejectedExecutionException if (e instanceof RejectedExecutionException && (e.getCause() == null || !(e.getCause() instanceof InterruptedException))) { @@ -610,9 +630,37 @@ public String apply(DataSegment input) toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); toolbox.getDataSegmentServerAnnouncer().unannounce(); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(null)); return success(); } + private Map getTaskCompletionReports(@Nullable String errorMsg) + { + return TaskReport.buildTaskReports( + new IngestionStatsAndErrorsTaskReport( + getId(), + new IngestionStatsAndErrorsTaskReportData( + ingestionState, + getTaskCompletionUnparseableEvents(), + getTaskCompletionRowStats(), + errorMsg + ) + ) + ); + } + + private Map getTaskCompletionUnparseableEvents() + { + Map unparseableEventsMap = Maps.newHashMap(); + List buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions( + savedParseExceptions + ); + if (buildSegmentsParseExceptionMessages != null) { + unparseableEventsMap.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegmentsParseExceptionMessages); + } + return unparseableEventsMap; + } + @Override public boolean canRestore() { @@ -1022,8 +1070,7 @@ private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox ioConfig.getStartPartitions().getPartitionSequenceMap().size()); return Appenderators.createRealtime( dataSchema, - tuningConfig.withBasePersistDirectory(new File(toolbox.getPersistDir(), "persist")) - .withMaxRowsInMemory(maxRowsInMemoryPerPartition), + tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), metrics, toolbox.getSegmentPusher(), toolbox.getObjectMapper(), @@ -1106,7 +1153,8 @@ private Set assignPartitions(RecordSupplier recordSupplier, String topic final Set assignment = Sets.newHashSet(); for (Map.Entry entry : lastOffsets.entrySet()) { final String endOffset = endOffsets.get(entry.getKey()); - if (KinesisPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) + if (Record.END_OF_SHARD_MARKER.equals(endOffset) + || KinesisPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) || KinesisSequenceNumber.of(entry.getValue()).compareTo(KinesisSequenceNumber.of(endOffset)) < 0) { assignment.add(entry.getKey()); } else if (entry.getValue().equals(endOffset)) { @@ -1197,10 +1245,12 @@ private boolean possiblyPause(Set assignment) throws InterruptedExceptio { pauseLock.lockInterruptibly(); try { + /* if (ioConfig.isPauseAfterRead() && assignment.isEmpty()) { pauseMillis = PAUSE_FOREVER; pauseRequested = true; } + */ if (pauseRequested) { status = Status.PAUSED; @@ -1317,4 +1367,11 @@ private Map getTaskCompletionRowStats() ); return metrics; } + + @Override + @JsonProperty("ioConfig") + public KinesisIOConfig getIOConfig() + { + return (KinesisIOConfig) super.getIOConfig(); + } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index d04cec5911ec..de8628fee808 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -172,6 +172,7 @@ private Runnable getRecordRunnable() data ); + if (log.isTraceEnabled()) { log.trace( "Stream[%s] / partition[%s] / sequenceNum[%s] / bufferRemainingCapacity[%d]: %s", @@ -352,7 +353,7 @@ public void seekToLatest(Set> partitions) @Override public Collection> getAssignment() { - return null; + return partitionResources.keySet(); } @Override @@ -399,10 +400,11 @@ public String getEarliestSequenceNumber(StreamPartition partition) throw return getSequenceNumberInternal(partition, ShardIteratorType.TRIM_HORIZON); } + //TODO: remove in the future @Override public String position(StreamPartition partition) { - return null; + throw new UnsupportedOperationException("position in KinesisRecordSupplier not supported"); } @Override diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java index 2b406ba4acf9..83d83cccf908 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java @@ -177,7 +177,7 @@ public KinesisTuningConfig withBasePersistDirectory(File dir) getMaxBytesInMemory(), getMaxRowsPerSegment(), getIntermediatePersistPeriod(), - getBasePersistDirectory(), + dir, getMaxPendingPersists(), getIndexSpec(), true, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index 2c26506f0118..f12bb52ebb93 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -142,7 +142,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(DateTimes.of("2016-05-31T14:00Z"), config.getMaximumMessageTime().get()); Assert.assertEquals(config.getEndpoint(), "kinesis.us-east-2.amazonaws.com"); Assert.assertEquals(config.getExclusiveStartSequenceNumberPartitions(), ImmutableSet.of("0")); - Assert.assertEquals(4000, config.getRecordsPerFetch()); + Assert.assertEquals(1000, config.getRecordsPerFetch()); Assert.assertEquals(1000, config.getFetchDelayMillis()); Assert.assertEquals("awsid", config.getAwsAccessKeyId()); Assert.assertEquals("awskey", config.getAwsSecretAccessKey()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index bdc94e9d1b96..2c4cb226867a 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -25,12 +25,10 @@ import cloud.localstack.docker.annotation.LocalstackDockerProperties; import com.amazonaws.http.SdkHttpMetadata; import com.amazonaws.services.kinesis.AmazonKinesis; -import com.amazonaws.services.kinesis.model.GetRecordsRequest; -import com.amazonaws.services.kinesis.model.GetRecordsResult; import com.amazonaws.services.kinesis.model.PutRecordsRequest; import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry; -import com.amazonaws.services.kinesis.model.Record; -import com.amazonaws.services.kinesis.model.ShardIteratorType; +import com.amazonaws.services.kinesis.model.PutRecordsResult; +import com.amazonaws.services.kinesis.model.PutRecordsResultEntry; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; @@ -76,7 +74,9 @@ import org.apache.druid.indexing.common.actions.TaskActionToolbox; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskStorageConfig; +import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.IndexTaskTest; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -84,10 +84,13 @@ import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.common.Record; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; import org.apache.druid.indexing.test.TestDataSegmentKiller; import org.apache.druid.java.util.common.CompressionUtils; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -99,24 +102,31 @@ import org.apache.druid.java.util.emitter.core.NoopEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.MonitorScheduler; +import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; +import org.apache.druid.query.Druids; import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryEngine; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.TestHelper; @@ -132,6 +142,8 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import org.apache.druid.segment.transform.ExpressionTransform; +import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.coordination.ServerType; @@ -157,6 +169,7 @@ import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.Arrays; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -164,7 +177,11 @@ import java.util.concurrent.Executor; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import static org.apache.druid.query.QueryPlus.wrap; + +// TODO: improve helper methods like insertData(...) @RunWith(LocalstackDockerTestRunner.class) @LocalstackDockerProperties(services = {"kinesis"}) public class KinesisIndexTaskTest @@ -185,11 +202,69 @@ public class KinesisIndexTaskTest private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); // private static final long POLL_RETRY_MS = 100; private static int streamPosFix = 0; + private static String shardId1 = "shardId-000000000001"; + private static String shardId0 = "shardId-000000000000"; + private static final List records = ImmutableList.of( + generateRequestEntry( + "1", + JB("2008", "a", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2009", "b", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2010", "c", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2011", "d", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2011", "e", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0") + ), + generateRequestEntry("1", StringUtils.toUtf8("unparseable")), + generateRequestEntry( + "1", + StringUtils.toUtf8("unparseable2") + ), + generateRequestEntry("1", "{}".getBytes()), + generateRequestEntry( + "1", + JB("2013", "f", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2049", "f", "y", "notanumber", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2049", "f", "y", "10", "notanumber", "1.0") + ), + generateRequestEntry( + "1", + JB("2049", "f", "y", "10", "20.0", "notanumber") + ), + generateRequestEntry( + "123123", + JB("2012", "g", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "123123", + JB("2011", "h", "y", "10", "20.0", "1.0") + ) + ); + private static ServiceEmitter emitter; private static ListeningExecutorService taskExec; - private static final AmazonKinesis kinesis = TestUtils.getClientKinesis(); private final List runningTasks = Lists.newArrayList(); private long handoffConditionTimeout = 0; @@ -199,7 +274,7 @@ public class KinesisIndexTaskTest private Integer maxSavedParseExceptions = null; private boolean resetOffsetAutomatically = false; private boolean doHandoff = true; - private Integer maxRowsPerSegment = null; + private int maxRowsInMemory = 1000; private Long maxTotalRows = null; private Period intermediateHandoffPeriod = null; @@ -209,7 +284,6 @@ public class KinesisIndexTaskTest private TaskLockbox taskLockbox; private File directory; private String stream; - private PutRecordsRequest recordRequests; private final boolean isIncrementalHandoffSupported = false; private final Set checkpointRequestsHash = Sets.newHashSet(); private File reportsFile; @@ -250,158 +324,1260 @@ public class KinesisIndexTaskTest objectMapper ); - private static PutRecordsRequestEntry generateRequestEntry(String partition, String explicitHash, byte[] data) + private static PutRecordsRequestEntry generateRequestEntry(String partition, byte[] data) { return new PutRecordsRequestEntry().withPartitionKey(partition) - .withExplicitHashKey(explicitHash) .withData(ByteBuffer.wrap(data)); } - private static PutRecordsRequest generateRecordsRequests(String stream) - { - return new PutRecordsRequest() - .withStreamName(stream) - .withRecords( - ImmutableList.of( - generateRequestEntry("0", "0", JB("2008", "a", "y", "10", "20.0", "1.0")), - generateRequestEntry("0", "0", JB("2009", "b", "y", "10", "20.0", "1.0")), - generateRequestEntry("0", "0", JB("2010", "c", "y", "10", "20.0", "1.0")), - generateRequestEntry("0", "0", JB("2011", "d", "y", "10", "20.0", "1.0")), - generateRequestEntry("0", "0", JB("2011", "e", "y", "10", "20.0", "1.0")), - generateRequestEntry("0", "0", JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")), - generateRequestEntry("0", "0", StringUtils.toUtf8("unparseable")), - generateRequestEntry("0", "0", StringUtils.toUtf8("unparseable2")), - generateRequestEntry("0", "0", "{}".getBytes()), - generateRequestEntry("0", "0", JB("2013", "f", "y", "10", "20.0", "1.0")), - generateRequestEntry("0", "0", JB("2049", "f", "y", "notanumber", "20.0", "1.0")), - generateRequestEntry("0", "0", JB("2049", "f", "y", "10", "notanumber", "1.0")), - generateRequestEntry("0", "0", JB("2049", "f", "y", "10", "20.0", "notanumber")), - generateRequestEntry("1", "1", JB("2012", "g", "y", "10", "20.0", "1.0")), - generateRequestEntry("1", "1", JB("2011", "h", "y", "10", "20.0", "1.0")) - ) - ); + private static PutRecordsRequest generateRecordsRequests(String stream, int first, int last) + { + return new PutRecordsRequest() + .withStreamName(stream) + .withRecords(records.subList(first, last)); + } + + private static PutRecordsRequest generateRecordsRequests(String stream) + { + return new PutRecordsRequest() + .withStreamName(stream) + .withRecords(records); + } + + private static String getStreamName() + { + return "stream-" + streamPosFix++; + } + + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @Rule + public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); + + @BeforeClass + public static void setupClass() + { + emitter = new ServiceEmitter( + "service", + "host", + new NoopEmitter() + ); + emitter.start(); + EmittingLogger.registerEmitter(emitter); + taskExec = MoreExecutors.listeningDecorator( + Executors.newCachedThreadPool( + Execs.makeThreadFactory("kafka-task-test-%d") + ) + ); + } + + @Before + public void setupTest() throws IOException, InterruptedException + { + handoffConditionTimeout = 0; + reportParseExceptions = false; + logParseExceptions = true; + maxParseExceptions = null; + maxSavedParseExceptions = null; + doHandoff = true; + stream = getStreamName(); + reportsFile = File.createTempFile("KinesisIndexTaskTestReports-" + System.currentTimeMillis(), "json"); + + // sleep required because of kinesalite + Thread.sleep(500); + makeToolboxFactory(); + } + + @After + public void tearDownTest() + { + synchronized (runningTasks) { + for (Task task : runningTasks) { + task.stopGracefully(); + } + + runningTasks.clear(); + } + reportsFile.delete(); + destroyToolboxFactory(); + } + + @AfterClass + public static void tearDownClass() throws Exception + { + taskExec.shutdown(); + taskExec.awaitTermination(9999, TimeUnit.DAYS); + emitter.close(); + } + + + @Test(timeout = 60_000L) + public void testRunAfterDataInserted() throws Exception + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata(new KinesisPartitions( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + ) + )), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testRunBeforeDataInserted() throws Exception + { + AmazonKinesis kinesis = getKinesisClientInstance(); + + // insert 1 row to get starting seq number + List res = insertData(kinesis, new PutRecordsRequest() + .withStreamName(stream) + .withRecords( + ImmutableList.of( + generateRequestEntry( + "123123", + JB("2055", "z", "y", "10", "20.0", "1.0") + ) + ) + ) + ); + + + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + Record.END_OF_SHARD_MARKER + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + + ); + + final ListenableFuture future = runTask(task); + + // Wait for the task to start reading + while (task.getStatus() != SeekableStreamIndexTask.Status.READING) { + Thread.sleep(10); + } + + // Insert data + List res2 = insertData(kinesis, generateRecordsRequests(stream)); + + // force shard 0 to close + kinesis.splitShard(stream, shardId0, "somenewshardpls234234234"); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2011/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2012/P1D", 0); + SegmentDescriptor desc3 = SD(task, "2055/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata( + new KinesisPartitions( + stream, + ImmutableMap.of( + shardId0, + Record.END_OF_SHARD_MARKER + ) + )), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("h"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("z"), readSegmentColumn("dim1", desc3)); + } + + + @Test(timeout = 60_000L) + public void testRunWithMinimumMessageTime() throws Exception + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + )), + true, + null, + DateTimes.of("2010"), + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for the task to start reading + while (task.getStatus() != SeekableStreamIndexTask.Status.READING) { + Thread.sleep(10); + } + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(2, task.getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata( + new KinesisPartitions( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + ) + )), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testRunWithMaximumMessageTime() throws Exception + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + )), + true, + null, + null, + DateTimes.of("2010"), + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for the task to start reading + while (task.getStatus() != SeekableStreamIndexTask.Status.READING) { + Thread.sleep(10); + } + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(2, task.getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); + SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata( + new KinesisPartitions( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + ) + )), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); + } + + @Test(timeout = 60_000L) + public void testRunWithTransformSpec() throws Exception + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final KinesisIndexTask task = createTask( + null, + DATA_SCHEMA.withTransformSpec( + new TransformSpec( + new SelectorDimFilter("dim1", "b", null), + ImmutableList.of( + new ExpressionTransform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) + ) + ) + ), + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for the task to start reading + while (task.getStatus() != SeekableStreamIndexTask.Status.READING) { + Thread.sleep(10); + } + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(1, task.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(4, task.getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2009/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata( + new KinesisPartitions( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + ) + )), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", desc1)); + } + + @Test(timeout = 60_000L) + public void testRunOnNothing() throws Exception + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(0, task.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + + // Check published metadata + Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); + } + + @Test(timeout = 60_000L) + public void testReportParseExceptions() throws Exception + { + reportParseExceptions = true; + + // these will be ignored because reportParseExceptions is true + maxParseExceptions = 1000; + maxSavedParseExceptions = 2; + + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 6) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.FAILED, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(1, task.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + + // Check published metadata + Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + } + + @Test(timeout = 60_000L) + public void testMultipleParseExceptionsSuccess() throws Exception + { + reportParseExceptions = false; + maxParseExceptions = 7; + maxSavedParseExceptions = 7; + + // Insert data + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 12) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + TaskStatus status = future.get(); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); + Assert.assertEquals(null, status.getErrorMsg()); + + // Check metrics + Assert.assertEquals(4, task.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(3, task.getRowIngestionMeters().getProcessedWithError()); + Assert.assertEquals(4, task.getRowIngestionMeters().getUnparseable()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + SegmentDescriptor desc3 = SD(task, "2013/P1D", 0); + SegmentDescriptor desc4 = SD(task, "2049/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata( + new KinesisPartitions( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 12) + ) + )), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Map expectedMetrics = ImmutableMap.of( + RowIngestionMeters.BUILD_SEGMENTS, + ImmutableMap.of( + RowIngestionMeters.PROCESSED, 4, + RowIngestionMeters.PROCESSED_WITH_ERROR, 3, + RowIngestionMeters.UNPARSEABLE, 4, + RowIngestionMeters.THROWN_AWAY, 0 + ) + ); + Assert.assertEquals(expectedMetrics, reportData.getRowStats()); + + Map unparseableEvents = ImmutableMap.of( + RowIngestionMeters.BUILD_SEGMENTS, + Arrays.asList( + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=20.0, met1=notanumber}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [Unable to parse value[notanumber] for field[met1],]", + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=notanumber, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to float,]", + "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=notanumber, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to long,]", + "Unparseable timestamp found! Event: {}", + "Unable to parse row [unparseable2]", + "Unable to parse row [unparseable]", + "Encountered row with timestamp that cannot be represented as a long: [MapBasedInputRow{timestamp=246140482-04-24T15:36:27.903Z, event={timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}]" + ) + ); + + Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); + } + + @Test(timeout = 60_000L) + public void testMultipleParseExceptionsFailure() throws Exception + { + reportParseExceptions = false; + maxParseExceptions = 2; + maxSavedParseExceptions = 2; + + // Insert data + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 9) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + TaskStatus status = future.get(); + + // Wait for task to exit + Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); + IndexTaskTest.checkTaskStatusErrorMsgForParseExceptionsExceeded(status); + + // Check metrics + Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRowIngestionMeters().getProcessedWithError()); + Assert.assertEquals(3, task.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + + // Check published metadata + Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + + IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); + + Map expectedMetrics = ImmutableMap.of( + RowIngestionMeters.BUILD_SEGMENTS, + ImmutableMap.of( + RowIngestionMeters.PROCESSED, 3, + RowIngestionMeters.PROCESSED_WITH_ERROR, 0, + RowIngestionMeters.UNPARSEABLE, 3, + RowIngestionMeters.THROWN_AWAY, 0 + ) + ); + Assert.assertEquals(expectedMetrics, reportData.getRowStats()); + + Map unparseableEvents = ImmutableMap.of( + RowIngestionMeters.BUILD_SEGMENTS, + Arrays.asList( + "Unable to parse row [unparseable2]", + "Unable to parse row [unparseable]" + ) + ); + + Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); + } + + @Test(timeout = 60_000L) + public void testRunReplicas() throws Exception + { + // Insert data + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final KinesisIndexTask task1 = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + final KinesisIndexTask task2 = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + final ListenableFuture future1 = runTask(task1); + final ListenableFuture future2 = runTask(task2); + + // Wait for tasks to exit + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task1.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task2.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task2.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRowIngestionMeters().getThrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata( + new KinesisPartitions( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + ) + )), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testRunConflicting() throws Exception + { + // Insert data + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final KinesisIndexTask task1 = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + final KinesisIndexTask task2 = createTask( + null, + new KinesisIOConfig( + "sequence1", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 9) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + // Run first task + final ListenableFuture future1 = runTask(task1); + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + + // Run second task + final ListenableFuture future2 = runTask(task2); + Assert.assertEquals(TaskState.FAILED, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task1.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task2.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(4, task2.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRowIngestionMeters().getThrownAway()); + + // Check published segments & metadata, should all be from the first task + SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata( + new KinesisPartitions( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + ) + )), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 60_000L) + public void testRunConflictingWithoutTransactions() throws Exception + { + // Insert data + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final KinesisIndexTask task1 = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + )), + false, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + final KinesisIndexTask task2 = createTask( + null, + new KinesisIOConfig( + "sequence1", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 9) + )), + false, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + // Run first task + final ListenableFuture future1 = runTask(task1); + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + + // Check published segments & metadata + SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + + // Run second task + final ListenableFuture future2 = runTask(task2); + Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task1.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task2.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(4, task2.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRowIngestionMeters().getThrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1); + SegmentDescriptor desc4 = SD(task2, "2013/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc3)); + Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4)); } - private static String getStreamName() + @Test(timeout = 60_000L) + public void testRunOneTaskTwoPartitions() throws Exception { - return "stream-" + streamPosFix++; - } + // Insert data + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); - @Rule - public final TemporaryFolder tempFolder = new TemporaryFolder(); + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + "sequence1", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4), + shardId0, + getSequenceNumber(res, shardId0, 1) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); - @Rule - public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); + final ListenableFuture future = runTask(task); - @BeforeClass - public static void setupClass() - { - emitter = new ServiceEmitter( - "service", - "host", - new NoopEmitter() - ); - emitter.start(); - EmittingLogger.registerEmitter(emitter); - taskExec = MoreExecutors.listeningDecorator( - Executors.newCachedThreadPool( - Execs.makeThreadFactory("kafka-task-test-%d") - ) + + // Wait for tasks to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(5, task.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + + // Check published segments & metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + // desc3 will not be created in KinesisIndexTask (0.12.x) as it does not create per Kafka partition Druid segments + SegmentDescriptor desc3 = SD(task, "2011/P1D", 1); + SegmentDescriptor desc4 = SD(task, "2012/P1D", 0); + Assert.assertEquals(isIncrementalHandoffSupported + ? ImmutableSet.of(desc1, desc2, desc4) + : ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - } - @Before - public void setupTest() throws IOException, InterruptedException - { - handoffConditionTimeout = 0; - reportParseExceptions = false; - logParseExceptions = true; - maxParseExceptions = null; - maxSavedParseExceptions = null; - doHandoff = true; - stream = getStreamName(); - recordRequests = generateRecordsRequests(stream); - reportsFile = File.createTempFile("KinesisIndexTaskTestReports-" + System.currentTimeMillis(), "json"); + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4)); - // sleep required because of kinesalite - Thread.sleep(500); - makeToolboxFactory(); + // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically + Assert.assertEquals( + isIncrementalHandoffSupported + ? ImmutableSet.of(ImmutableList.of("d", "e", "h")) + : ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), + isIncrementalHandoffSupported + ? ImmutableSet.of(readSegmentColumn("dim1", desc2)) + : ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3)) + ); } - @After - public void tearDownTest() + @Test(timeout = 60_000L) + public void testRunTwoTasksTwoPartitions() throws Exception { - synchronized (runningTasks) { - for (Task task : runningTasks) { - task.stopGracefully(); - } + // Insert data + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); - runningTasks.clear(); - } - reportsFile.delete(); - destroyToolboxFactory(); - } + final KinesisIndexTask task1 = createTask( + null, + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + final KinesisIndexTask task2 = createTask( + null, + new KinesisIOConfig( + "sequence1", + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 1) + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); - @AfterClass - public static void tearDownClass() throws Exception - { - taskExec.shutdown(); - taskExec.awaitTermination(9999, TimeUnit.DAYS); - emitter.close(); - } + final ListenableFuture future1 = runTask(task1); + final ListenableFuture future2 = runTask(task2); - private String getSequenceNumber(AmazonKinesis kinesis, String shardId, ShardIteratorType itType, long recordsOffset) - throws Exception - { - String iterator = kinesis.getShardIterator(stream, shardId, itType.toString()).getShardIterator(); - Record record = null; - - while (iterator != null && recordsOffset >= 0) { - GetRecordsResult res = kinesis.getRecords(new GetRecordsRequest().withLimit(1).withShardIterator(iterator)); - List records = res.getRecords(); - if (!records.isEmpty()) { - record = records.get(0); - iterator = res.getNextShardIterator(); - } else { - throw new Exception("failed to get record for specified offset???"); - } - --recordsOffset; - } + // Wait for tasks to exit + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); - if (record == null) { - throw new Exception("record is null???"); - } + // Check metrics + Assert.assertEquals(3, task1.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(2, task2.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task2.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRowIngestionMeters().getThrownAway()); - return record.getSequenceNumber(); - } + // Check published segments & metadata + SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); + SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1); + SegmentDescriptor desc4 = SD(task2, "2012/P1D", 0); - private static boolean isResponseOk(SdkHttpMetadata sdkHttpMetadata) - { - return sdkHttpMetadata.getHttpStatusCode() == 200; + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata( + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically + Assert.assertEquals( + isIncrementalHandoffSupported + ? ImmutableSet.of(ImmutableList.of("d", "e", "h")) + : ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), + isIncrementalHandoffSupported + ? ImmutableSet.of(readSegmentColumn("dim1", desc2)) + : ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3)) + ); + Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4)); } @Test(timeout = 60_000L) - public void testRunAfterDataInserted() throws Exception + public void testRestore() throws Exception { // Insert data - SdkHttpMetadata createRes = kinesis.createStream(stream, 2).getSdkHttpMetadata(); - // sleep required because of kinesalite - Thread.sleep(500); - Assert.assertTrue(isResponseOk(createRes)); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream, 0, 4)); - kinesis.putRecords(recordRequests); - - final KinesisIndexTask task = createTask( - null, + final KinesisIndexTask task1 = createTask( + "task1", new KinesisIOConfig( "sequence0", new KinesisPartitions(stream, ImmutableMap.of( - "0", - getSequenceNumber(kinesis, "0", ShardIteratorType.TRIM_HORIZON, 2) + shardId1, + getSequenceNumber(res, shardId1, 2) )), new KinesisPartitions(stream, ImmutableMap.of( - "0", - getSequenceNumber(kinesis, "0", ShardIteratorType.TRIM_HORIZON, 4) + shardId1, + Record.END_OF_SHARD_MARKER )), true, null, @@ -417,1560 +1593,91 @@ public void testRunAfterDataInserted() throws Exception null, false ) + ); + + final ListenableFuture future1 = runTask(task1); + + while (countEvents(task1) != 2) { + Thread.sleep(25); + } + + Assert.assertEquals(2, countEvents(task1)); + + // Stop without publishing segment + task1.stopGracefully(); + unlockAppenderatorBasePersistDirForTask(task1); + + Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + List res2 = insertData(kinesis, generateRecordsRequests(stream, 4, 5)); + + // Start a new task + final KinesisIndexTask task2 = createTask( + task1.getId(), + new KinesisIOConfig( + "sequence0", + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + Record.END_OF_SHARD_MARKER + )), + true, + null, + null, + null, + Localstack.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + ImmutableSet.of(shardId1), + null, + null, + false + ) ); - final ListenableFuture future = runTask(task); + final ListenableFuture future2 = runTask(task2); + + while (countEvents(task2) < 1) { + Thread.sleep(25); + } + + // force shard to close + kinesis.splitShard(stream, shardId1, "somerandomshardidhah1213123"); // Wait for task to exit - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); // Check metrics - Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(2, task1.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(1, task2.getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task2.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); - SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + // Check published segments & metadata + SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KinesisDataSourceMetadata(new KinesisPartitions( - stream, - ImmutableMap.of( - "0", - getSequenceNumber( - kinesis, - "0", - ShardIteratorType.TRIM_HORIZON, - 4 - ) - ) - )), + new KinesisDataSourceMetadata( + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + Record.END_OF_SHARD_MARKER + ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); // Check segments in deep storage Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); - - SdkHttpMetadata delRes = kinesis.deleteStream(stream).getSdkHttpMetadata(); - Thread.sleep(500); - Assert.assertTrue(isResponseOk(delRes)); } - // @Test(timeout = 60_000L) -// public void testRunBeforeDataInserted() throws Exception -// { -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future = runTask(task); -// -// // Wait for the task to start reading -// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { -// Thread.sleep(10); -// } -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// // Wait for task to exit -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); -// } -// -// @Test(timeout = 60_000L) -// public void testIncrementalHandOff() throws Exception -// { -// if (!isIncrementalHandoffSupported) { -// return; -// } -// final String baseSequenceName = "sequence0"; -// // as soon as any segment has more than one record, incremental publishing should happen -// maxRowsPerSegment = 2; -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// Map consumerProps = kafkaServer.consumerProperties(); -// consumerProps.put("max.poll.records", "1"); -// -// final KinesisPartitions startPartitions = new KinesisPartitions(stream, ImmutableMap.of(0, 0L, 1, 0L)); -// // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering -// // of events fetched across two partitions from Kafka -// final KinesisPartitions checkpoint1 = new KinesisPartitions(stream, ImmutableMap.of(0, 5L, 1, 0L)); -// final KinesisPartitions checkpoint2 = new KinesisPartitions(stream, ImmutableMap.of(0, 4L, 1, 2L)); -// final KinesisPartitions endPartitions = new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 2L)); -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// baseSequenceName, -// startPartitions, -// endPartitions, -// consumerProps, -// true, -// null, -// null, -// false -// ) -// ); -// final ListenableFuture future = runTask(task); -// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { -// Thread.sleep(10); -// } -// final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); -// Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets) || checkpoint2.getPartitionOffsetMap() -// .equals(currentOffsets)); -// task.getRunner().setEndOffsets(currentOffsets, false); -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// -// Assert.assertEquals(1, checkpointRequestsHash.size()); -// Assert.assertTrue( -// checkpointRequestsHash.contains( -// Objects.hash( -// DATA_SCHEMA.getDataSource(), -// 0, -// new KinesisDataSourceMetadata(startPartitions), -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, currentOffsets)) -// ) -// ) -// ); -// -// // Check metrics -// Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); -// SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); -// SegmentDescriptor desc4 = SD(task, "2011/P1D", 0); -// SegmentDescriptor desc5 = SD(task, "2011/P1D", 1); -// SegmentDescriptor desc6 = SD(task, "2012/P1D", 0); -// SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 2L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); -// Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4)) -// && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) || -// (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4)) -// && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5)))); -// Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6)); -// Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); -// } -// -// @Test(timeout = 60_000L) -// public void testIncrementalHandOffMaxTotalRows() throws Exception -// { -// if (!isIncrementalHandoffSupported) { -// return; -// } -// final String baseSequenceName = "sequence0"; -// // incremental publish should happen every 3 records -// maxRowsPerSegment = Integer.MAX_VALUE; -// maxTotalRows = 3L; -// -// // Insert data -// int numToAdd = records.size() - 2; -// -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (int i = 0; i < numToAdd; i++) { -// kafkaProducer.send(records.get(i)).get(); -// } -// -// Map consumerProps = kafkaServer.consumerProperties(); -// consumerProps.put("max.poll.records", "1"); -// -// final KinesisPartitions startPartitions = new KinesisPartitions(stream, ImmutableMap.of(0, 0L, 1, 0L)); -// final KinesisPartitions checkpoint1 = new KinesisPartitions(stream, ImmutableMap.of(0, 3L, 1, 0L)); -// final KinesisPartitions checkpoint2 = new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 0L)); -// -// final KinesisPartitions endPartitions = new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 2L)); -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// baseSequenceName, -// startPartitions, -// endPartitions, -// consumerProps, -// true, -// null, -// null, -// false -// ) -// ); -// final ListenableFuture future = runTask(task); -// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { -// Thread.sleep(10); -// } -// final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); -// -// Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets)); -// task.getRunner().setEndOffsets(currentOffsets, false); -// -// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { -// Thread.sleep(10); -// } -// -// // add remaining records -// for (int i = numToAdd; i < records.size(); i++) { -// kafkaProducer.send(records.get(i)).get(); -// } -// final Map nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); -// -// Assert.assertTrue(checkpoint2.getPartitionOffsetMap().equals(nextOffsets)); -// task.getRunner().setEndOffsets(nextOffsets, false); -// -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// -// Assert.assertEquals(2, checkpointRequestsHash.size()); -// Assert.assertTrue( -// checkpointRequestsHash.contains( -// Objects.hash( -// DATA_SCHEMA.getDataSource(), -// 0, -// new KinesisDataSourceMetadata(startPartitions), -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, currentOffsets)) -// ) -// ) -// ); -// Assert.assertTrue( -// checkpointRequestsHash.contains( -// Objects.hash( -// DATA_SCHEMA.getDataSource(), -// 0, -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, currentOffsets)), -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, nextOffsets)) -// ) -// ) -// ); -// -// // Check metrics -// Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); -// SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); -// SegmentDescriptor desc4 = SD(task, "2011/P1D", 0); -// SegmentDescriptor desc5 = SD(task, "2011/P1D", 1); -// SegmentDescriptor desc6 = SD(task, "2012/P1D", 0); -// SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 2L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); -// Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4)) -// && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) || -// (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4)) -// && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5)))); -// Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6)); -// Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); -// } -// } -// -// @Test(timeout = 60_000L) -// public void testTimeBasedIncrementalHandOff() throws Exception -// { -// if (!isIncrementalHandoffSupported) { -// return; -// } -// final String baseSequenceName = "sequence0"; -// // as soon as any segment hits maxRowsPerSegment or intermediateHandoffPeriod, incremental publishing should happen -// maxRowsPerSegment = Integer.MAX_VALUE; -// intermediateHandoffPeriod = new Period().withSeconds(0); -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records.subList(0, 2)) { -// kafkaProducer.send(record).get(); -// } -// } -// Map consumerProps = kafkaServer.consumerProperties(); -// consumerProps.put("max.poll.records", "1"); -// -// final KinesisPartitions startPartitions = new KinesisPartitions(stream, ImmutableMap.of(0, 0L, 1, 0L)); -// // Checkpointing will happen at checkpoint -// final KinesisPartitions checkpoint = new KinesisPartitions(stream, ImmutableMap.of(0, 1L, 1, 0L)); -// final KinesisPartitions endPartitions = new KinesisPartitions(stream, ImmutableMap.of(0, 2L, 1, 0L)); -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// baseSequenceName, -// startPartitions, -// endPartitions, -// consumerProps, -// true, -// null, -// null, -// false -// ) -// ); -// final ListenableFuture future = runTask(task); -// -// // task will pause for checkpointing -// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { -// Thread.sleep(10); -// } -// final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); -// Assert.assertTrue(checkpoint.getPartitionOffsetMap().equals(currentOffsets)); -// task.getRunner().setEndOffsets(currentOffsets, false); -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// -// Assert.assertEquals(1, checkpointRequestsHash.size()); -// Assert.assertTrue( -// checkpointRequestsHash.contains( -// Objects.hash( -// DATA_SCHEMA.getDataSource(), -// 0, -// new KinesisDataSourceMetadata(startPartitions), -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, checkpoint.getPartitionOffsetMap())) -// ) -// ) -// ); -// -// // Check metrics -// Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 2L, 1, 0L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); -// } -// -// @Test(timeout = 60_000L) -// public void testRunWithMinimumMessageTime() throws Exception -// { -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// DateTimes.of("2010"), -// null, -// false -// ) -// ); -// -// final ListenableFuture future = runTask(task); -// -// // Wait for the task to start reading -// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { -// Thread.sleep(10); -// } -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// // Wait for task to exit -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); -// } -// -// @Test(timeout = 60_000L) -// public void testRunWithMaximumMessageTime() throws Exception -// { -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// DateTimes.of("2010"), -// false -// ) -// ); -// -// final ListenableFuture future = runTask(task); -// -// // Wait for the task to start reading -// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { -// Thread.sleep(10); -// } -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// // Wait for task to exit -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); -// SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); -// } -// -// @Test(timeout = 60_000L) -// public void testRunWithTransformSpec() throws Exception -// { -// final KinesisIndexTask task = createTask( -// null, -// DATA_SCHEMA.withTransformSpec( -// new TransformSpec( -// new SelectorDimFilter("dim1", "b", null), -// ImmutableList.of( -// new ExpressionTransform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) -// ) -// ) -// ), -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future = runTask(task); -// -// // Wait for the task to start reading -// while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { -// Thread.sleep(10); -// } -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// // Wait for task to exit -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2009/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", desc1)); -// } -// -// @Test(timeout = 60_000L) -// public void testRunOnNothing() throws Exception -// { -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future = runTask(task); -// -// // Wait for task to exit -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); -// } -// -// @Test(timeout = 60_000L) -// public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception -// { -// handoffConditionTimeout = 5_000; -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future = runTask(task); -// -// // Wait for task to exit -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); -// } -// -// @Test(timeout = 60_000L) -// public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exception -// { -// doHandoff = false; -// handoffConditionTimeout = 100; -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future = runTask(task); -// -// // Wait for task to exit -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); -// } -// -// @Test(timeout = 60_000L) -// public void testReportParseExceptions() throws Exception -// { -// reportParseExceptions = true; -// -// // these will be ignored because reportParseExceptions is true -// maxParseExceptions = 1000; -// maxSavedParseExceptions = 2; -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 7L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future = runTask(task); -// -// // Wait for task to exit -// Assert.assertEquals(TaskState.FAILED, future.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); -// Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); -// } -// -// @Test(timeout = 60_000L) -// public void testMultipleParseExceptionsSuccess() throws Exception -// { -// reportParseExceptions = false; -// maxParseExceptions = 6; -// maxSavedParseExceptions = 6; -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 13L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future = runTask(task); -// -// TaskStatus status = future.get(); -// -// // Wait for task to exit -// Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); -// Assert.assertEquals(null, status.getErrorMsg()); -// -// // Check metrics -// Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessedWithError()); -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); -// SegmentDescriptor desc3 = SD(task, "2013/P1D", 0); -// SegmentDescriptor desc4 = SD(task, "2049/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 13L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); -// -// Map expectedMetrics = ImmutableMap.of( -// RowIngestionMeters.BUILD_SEGMENTS, -// ImmutableMap.of( -// RowIngestionMeters.PROCESSED, 4, -// RowIngestionMeters.PROCESSED_WITH_ERROR, 3, -// RowIngestionMeters.UNPARSEABLE, 3, -// RowIngestionMeters.THROWN_AWAY, 1 -// ) -// ); -// Assert.assertEquals(expectedMetrics, reportData.getRowStats()); -// -// Map unparseableEvents = ImmutableMap.of( -// RowIngestionMeters.BUILD_SEGMENTS, -// Arrays.asList( -// "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=20.0, met1=notanumber}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [Unable to parse value[notanumber] for field[met1],]", -// "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=10, dimFloat=notanumber, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to float,]", -// "Found unparseable columns in row: [MapBasedInputRow{timestamp=2049-01-01T00:00:00.000Z, event={timestamp=2049, dim1=f, dim2=y, dimLong=notanumber, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}], exceptions: [could not convert value [notanumber] to long,]", -// "Unable to parse row [unparseable2]", -// "Unable to parse row [unparseable]", -// "Encountered row with timestamp that cannot be represented as a long: [MapBasedInputRow{timestamp=246140482-04-24T15:36:27.903Z, event={timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}]" -// ) -// ); -// -// Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); -// } -// -// @Test(timeout = 60_000L) -// public void testMultipleParseExceptionsFailure() throws Exception -// { -// reportParseExceptions = false; -// maxParseExceptions = 2; -// maxSavedParseExceptions = 2; -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 10L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future = runTask(task); -// -// TaskStatus status = future.get(); -// -// // Wait for task to exit -// Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); -// IndexTaskTest.checkTaskStatusErrorMsgForParseExceptionsExceeded(status); -// -// // Check metrics -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessedWithError()); -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); -// Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); -// -// IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); -// -// Map expectedMetrics = ImmutableMap.of( -// RowIngestionMeters.BUILD_SEGMENTS, -// ImmutableMap.of( -// RowIngestionMeters.PROCESSED, 3, -// RowIngestionMeters.PROCESSED_WITH_ERROR, 0, -// RowIngestionMeters.UNPARSEABLE, 3, -// RowIngestionMeters.THROWN_AWAY, 0 -// ) -// ); -// Assert.assertEquals(expectedMetrics, reportData.getRowStats()); -// -// Map unparseableEvents = ImmutableMap.of( -// RowIngestionMeters.BUILD_SEGMENTS, -// Arrays.asList( -// "Unable to parse row [unparseable2]", -// "Unable to parse row [unparseable]" -// ) -// ); -// -// Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); -// } -// -// @Test(timeout = 60_000L) -// public void testRunReplicas() throws Exception -// { -// final KinesisIndexTask task1 = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// final KinesisIndexTask task2 = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future1 = runTask(task1); -// final ListenableFuture future2 = runTask(task2); -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// // Wait for tasks to exit -// Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); -// Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); -// Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published segments & metadata -// SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); -// } -// -// @Test(timeout = 60_000L) -// public void testRunConflicting() throws Exception -// { -// final KinesisIndexTask task1 = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// final KinesisIndexTask task2 = createTask( -// null, -// new KafkaIOConfig( -// 1, -// "sequence1", -// new KinesisPartitions(stream, ImmutableMap.of(0, 3L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 10L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// // Run first task -// final ListenableFuture future1 = runTask(task1); -// Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); -// -// // Run second task -// final ListenableFuture future2 = runTask(task2); -// Assert.assertEquals(TaskState.FAILED, future2.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); -// Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published segments & metadata, should all be from the first task -// SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); -// } -// -// @Test(timeout = 60_000L) -// public void testRunConflictingWithoutTransactions() throws Exception -// { -// final KinesisIndexTask task1 = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// false, -// null, -// null, -// false -// ) -// ); -// final KinesisIndexTask task2 = createTask( -// null, -// new KafkaIOConfig( -// 1, -// "sequence1", -// new KinesisPartitions(stream, ImmutableMap.of(0, 3L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 10L)), -// kafkaServer.consumerProperties(), -// false, -// null, -// null, -// false -// ) -// ); -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// // Run first task -// final ListenableFuture future1 = runTask(task1); -// Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); -// -// // Check published segments & metadata -// SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); -// Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); -// -// // Run second task -// final ListenableFuture future2 = runTask(task2); -// Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); -// Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published segments & metadata -// SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1); -// SegmentDescriptor desc4 = SD(task2, "2013/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); -// Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc3)); -// Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4)); -// } -// -// @Test(timeout = 60_000L) -// public void testRunOneTaskTwoPartitions() throws Exception -// { -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L, 1, 0L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L, 1, 2L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future = runTask(task); -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// kafkaProducer.flush(); -// } -// -// // Wait for tasks to exit -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(5, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published segments & metadata -// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); -// // desc3 will not be created in KinesisIndexTask (0.12.x) as it does not create per Kafka partition Druid segments -// SegmentDescriptor desc3 = SD(task, "2011/P1D", 1); -// SegmentDescriptor desc4 = SD(task, "2012/P1D", 0); -// Assert.assertEquals(isIncrementalHandoffSupported -// ? ImmutableSet.of(desc1, desc2, desc4) -// : ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L, 1, 2L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4)); -// -// // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically -// Assert.assertEquals( -// isIncrementalHandoffSupported -// ? ImmutableSet.of(ImmutableList.of("d", "e", "h")) -// : ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), -// isIncrementalHandoffSupported -// ? ImmutableSet.of(readSegmentColumn("dim1", desc2)) -// : ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3)) -// ); -// } -// -// @Test(timeout = 60_000L) -// public void testRunTwoTasksTwoPartitions() throws Exception -// { -// final KinesisIndexTask task1 = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// final KinesisIndexTask task2 = createTask( -// null, -// new KafkaIOConfig( -// 1, -// "sequence1", -// new KinesisPartitions(stream, ImmutableMap.of(1, 0L)), -// new KinesisPartitions(stream, ImmutableMap.of(1, 1L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future1 = runTask(task1); -// final ListenableFuture future2 = runTask(task2); -// -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// // Wait for tasks to exit -// Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); -// Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); -// Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published segments & metadata -// SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); -// SegmentDescriptor desc3 = SD(task2, "2012/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L, 1, 1L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); -// Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc3)); -// } -// -// @Test(timeout = 60_000L) -// public void testRestore() throws Exception -// { -// final KinesisIndexTask task1 = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future1 = runTask(task1); -// -// // Insert some data, but not enough for the task to finish -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : Iterables.limit(records, 4)) { -// kafkaProducer.send(record).get(); -// } -// } -// -// while (countEvents(task1) != 2) { -// Thread.sleep(25); -// } -// -// Assert.assertEquals(2, countEvents(task1)); -// -// // Stop without publishing segment -// task1.stopGracefully(); -// unlockAppenderatorBasePersistDirForTask(task1); -// -// Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); -// -// // Start a new task -// final KinesisIndexTask task2 = createTask( -// task1.getId(), -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future2 = runTask(task2); -// -// // Insert remaining data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : Iterables.skip(records, 4)) { -// kafkaProducer.send(record).get(); -// } -// } -// -// // Wait for task to exit -// Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(2, task1.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); -// Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published segments & metadata -// SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); -// } -// -// @Test(timeout = 60_000L) -// public void testRunWithPauseAndResume() throws Exception -// { -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// final ListenableFuture future = runTask(task); -// -// // Insert some data, but not enough for the task to finish -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : Iterables.limit(records, 4)) { -// kafkaProducer.send(record).get(); -// } -// kafkaProducer.flush(); -// } -// -// while (countEvents(task) != 2) { -// Thread.sleep(25); -// } -// -// Assert.assertEquals(2, countEvents(task)); -// Assert.assertEquals(SeekableStreamIndexTask.Status.READING, task.getRunner().getStatus()); -// -// Map currentOffsets = objectMapper.readValue( -// task.getRunner().pause().getEntity().toString(), -// new TypeReference>() -// { -// } -// ); -// Assert.assertEquals(SeekableStreamIndexTask.Status.PAUSED, task.getRunner().getStatus()); -// -// // Insert remaining data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : Iterables.skip(records, 4)) { -// kafkaProducer.send(record).get(); -// } -// } -// -// try { -// future.get(10, TimeUnit.SECONDS); -// Assert.fail("Task completed when it should have been paused"); -// } -// catch (TimeoutException e) { -// // carry on.. -// } -// -// Assert.assertEquals(currentOffsets, task.getRunner().getCurrentOffsets()); -// -// task.getRunner().resume(); -// -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// Assert.assertEquals(task.getRunner().getEndOffsets(), task.getRunner().getCurrentOffsets()); -// -// // Check metrics -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); -// } -// -// @Test(timeout = 60_000L) -// public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception -// { -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 2L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// runTask(task); -// -// while (!task.getRunner().getStatus().equals(SeekableStreamIndexTask.Status.READING)) { -// Thread.sleep(2000); -// } -// -// task.getRunner().pause(); -// -// while (!task.getRunner().getStatus().equals(SeekableStreamIndexTask.Status.PAUSED)) { -// Thread.sleep(25); -// } -// } -// -// @Test(timeout = 60_000L) -// public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAvailable() throws Exception -// { -// resetOffsetAutomatically = true; -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// new KinesisPartitions(stream, ImmutableMap.of(0, 200L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 500L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ) -// ); -// -// runTask(task); -// -// while (!task.getRunner().getStatus().equals(SeekableStreamIndexTask.Status.READING)) { -// Thread.sleep(20); -// } -// -// for (int i = 0; i < 5; i++) { -// Assert.assertEquals(task.getRunner().getStatus(), SeekableStreamIndexTask.Status.READING); -// // Offset should not be reset -// Assert.assertTrue(task.getRunner().getCurrentOffsets().get(0) == 200L); -// } -// } -// -// @Test(timeout = 60_000L) -// public void testRunContextSequenceAheadOfStartingOffsets() throws Exception -// { -// // This tests the case when a replacement task is created in place of a failed test -// // which has done some incremental handoffs, thus the context will contain starting -// // sequence offsets from which the task should start reading and ignore the start offsets -// if (!isIncrementalHandoffSupported) { -// return; -// } -// // Insert data -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (ProducerRecord record : records) { -// kafkaProducer.send(record).get(); -// } -// } -// -// final TreeMap> sequences = new TreeMap<>(); -// // Here the sequence number is 1 meaning that one incremental handoff was done by the failed task -// // and this task should start reading from offset 2 for partition 0 -// sequences.put(1, ImmutableMap.of(0, 2L)); -// final Map context = new HashMap<>(); -// context.put("checkpoints", objectMapper.writerWithType(new TypeReference>>() -// { -// }).writeValueAsString(sequences)); -// -// final KinesisIndexTask task = createTask( -// null, -// new KafkaIOConfig( -// 0, -// "sequence0", -// // task should ignore these and use sequence info sent in the context -// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 5L)), -// kafkaServer.consumerProperties(), -// true, -// null, -// null, -// false -// ), -// context -// ); -// -// final ListenableFuture future = runTask(task); -// -// // Wait for task to exit -// Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); -// -// // Check metrics -// Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); -// Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); -// -// // Check published metadata -// SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); -// SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); -// Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); -// Assert.assertEquals( -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of(0, 5L))), -// metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) -// ); -// -// // Check segments in deep storage -// Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); -// Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); -// } - private ListenableFuture runTask(final Task task) { try { @@ -1995,7 +1702,7 @@ private ListenableFuture runTask(final Task task) } catch (Exception e) { log.warn(e, "Task failed"); - return TaskStatus.failure(task.getId()); + return TaskStatus.failure(task.getId(), Throwables.getStackTraceAsString(e)); } } ); @@ -2052,7 +1759,7 @@ private KinesisIndexTask createTask( reportParseExceptions, handoffConditionTimeout, resetOffsetAutomatically, - false, + true, null, null, null, @@ -2087,7 +1794,7 @@ private KinesisIndexTask createTask( ) { final KinesisTuningConfig tuningConfig = new KinesisTuningConfig( - 1000, + maxRowsInMemory, null, null, new Period("P1Y"), @@ -2098,7 +1805,7 @@ private KinesisIndexTask createTask( reportParseExceptions, handoffConditionTimeout, resetOffsetAutomatically, - null, + true, null, null, null, @@ -2391,26 +2098,26 @@ private List readSegmentColumn(final String column, final SegmentDescrip return values; } - // -// public long countEvents(final Task task) -// { -// // Do a query. -// TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() -// .dataSource(DATA_SCHEMA.getDataSource()) -// .aggregators( -// ImmutableList.of( -// new LongSumAggregatorFactory("rows", "rows") -// ) -// ).granularity(Granularities.ALL) -// .intervals("0000/3000") -// .build(); -// -// List> results = -// task.getQueryRunner(query).run(wrap(query), ImmutableMap.of()).toList(); -// -// return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); -// } -// + + public long countEvents(final Task task) + { + // Do a query. + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(DATA_SCHEMA.getDataSource()) + .aggregators( + ImmutableList.of( + new LongSumAggregatorFactory("rows", "rows") + ) + ).granularity(Granularities.ALL) + .intervals("0000/3000") + .build(); + + List> results = + task.getQueryRunner(query).run(wrap(query), ImmutableMap.of()).toList(); + + return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); + } + private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) { try { @@ -2448,4 +2155,40 @@ private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOExcep taskReports ); } + + private AmazonKinesis getKinesisClientInstance() throws InterruptedException + { + AmazonKinesis kinesis = TestUtils.getClientKinesis(); + SdkHttpMetadata createRes = kinesis.createStream(stream, 2).getSdkHttpMetadata(); + // sleep required because of kinesalite + Thread.sleep(500); + Assert.assertTrue(isResponseOk(createRes)); + return kinesis; + } + + private static String getSequenceNumber(List entries, String shardId, int offset) + { + List sortedEntries = entries.stream() + .filter(e -> e.getShardId().equals(shardId)) + .sorted(Comparator.comparing(e -> KinesisSequenceNumber.of(e.getSequenceNumber()))) + .collect(Collectors.toList()); + return sortedEntries.get(offset).getSequenceNumber(); + } + + + private static boolean isResponseOk(SdkHttpMetadata sdkHttpMetadata) + { + return sdkHttpMetadata.getHttpStatusCode() == 200; + } + + private static List insertData( + AmazonKinesis kinesis, + PutRecordsRequest req + ) + { + PutRecordsResult res = kinesis.putRecords(req); + Assert.assertTrue(isResponseOk(res.getSdkHttpMetadata())); + Assert.assertEquals((int) res.getFailedRecordCount(), 0); + return res.getRecords(); + } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 92f8eff5c702..d6abd3649d17 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -16,390 +16,532 @@ * specific language governing permissions and limitations * under the License. */ -// -//package org.apache.druid.indexing.kinesis.supervisor; -// -//import com.fasterxml.jackson.databind.ObjectMapper; -//import com.google.common.base.Charsets; -//import com.google.common.base.Optional; -//import com.google.common.collect.ImmutableList; -//import com.google.common.collect.ImmutableMap; -//import com.google.common.util.concurrent.Futures; -//import com.google.common.util.concurrent.ListenableFuture; -//import org.apache.druid.data.input.impl.DimensionSchema; -//import org.apache.druid.data.input.impl.DimensionsSpec; -//import org.apache.druid.data.input.impl.JSONParseSpec; -//import org.apache.druid.data.input.impl.JSONPathFieldSpec; -//import org.apache.druid.data.input.impl.JSONPathSpec; -//import org.apache.druid.data.input.impl.StringDimensionSchema; -//import org.apache.druid.data.input.impl.StringInputRowParser; -//import org.apache.druid.data.input.impl.TimestampSpec; -//import org.apache.druid.indexing.common.TaskInfoProvider; -//import org.apache.druid.indexing.common.TaskLocation; -//import org.apache.druid.indexing.common.TaskStatus; -//import org.apache.druid.indexing.common.task.RealtimeIndexTask; -//import org.apache.druid.indexing.common.task.Task; -//import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata; -//import org.apache.druid.indexing.kinesis.KinesisIOConfig; -//import org.apache.druid.indexing.kinesis.KinesisIndexTask; -//import org.apache.druid.indexing.kinesis.KinesisIndexTaskClient; -//import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; -//import org.apache.druid.indexing.kinesis.KinesisPartitions; -//import org.apache.druid.indexing.kinesis.KinesisTuningConfig; -//import org.apache.druid.indexing.kinesis.test.TestBroker; -//import org.apache.druid.indexing.overlord.DataSourceMetadata; -//import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -//import org.apache.druid.indexing.overlord.TaskMaster; -//import org.apache.druid.indexing.overlord.TaskQueue; -//import org.apache.druid.indexing.overlord.TaskRunner; -//import org.apache.druid.indexing.overlord.TaskRunnerListener; -//import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; -//import org.apache.druid.indexing.overlord.TaskStorage; -//import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; -//import org.apache.druid.jackson.DefaultObjectMapper; -//import org.apache.druid.java.util.common.ISE; -//import org.apache.druid.java.util.common.granularity.Granularities; -//import org.apache.druid.query.aggregation.AggregatorFactory; -//import org.apache.druid.query.aggregation.CountAggregatorFactory; -//import org.apache.druid.segment.indexing.DataSchema; -//import org.apache.druid.segment.indexing.RealtimeIOConfig; -//import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -//import org.apache.druid.segment.realtime.FireDepartment; -//import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; -//import org.apache.druid.server.metrics.NoopServiceEmitter; -//import org.apache.curator.test.TestingCluster; -//import org.apache.kafka.clients.producer.KafkaProducer; -//import org.apache.kafka.clients.producer.ProducerRecord; -//import org.easymock.Capture; -//import org.easymock.CaptureType; -//import org.easymock.EasyMock; -//import org.easymock.EasyMockSupport; -//import org.joda.time.DateTime; -//import org.joda.time.Duration; -//import org.joda.time.Interval; -//import org.joda.time.Period; -//import org.junit.After; -//import org.junit.Assert; -//import org.junit.Before; -//import org.junit.Rule; -//import org.junit.Test; -//import org.junit.rules.TemporaryFolder; -//import org.junit.runner.RunWith; -//import org.junit.runners.Parameterized; -// -//import java.io.File; -//import java.util.ArrayList; -//import java.util.Collection; -//import java.util.Collections; -//import java.util.List; -//import java.util.Map; -//import java.util.concurrent.Executor; -// -//import static org.easymock.EasyMock.anyObject; -//import static org.easymock.EasyMock.anyString; -//import static org.easymock.EasyMock.capture; -//import static org.easymock.EasyMock.eq; -//import static org.easymock.EasyMock.expect; -//import static org.easymock.EasyMock.expectLastCall; -//import static org.easymock.EasyMock.replay; -//import static org.easymock.EasyMock.reset; -// -//@RunWith(Parameterized.class) -//public class KinesisSupervisorTest extends EasyMockSupport -//{ -// private static final ObjectMapper objectMapper = new DefaultObjectMapper(); -// private static final String KAFKA_TOPIC = "testTopic"; -// private static final String DATASOURCE = "testDS"; -// private static final int NUM_PARTITIONS = 3; -// private static final int TEST_CHAT_THREADS = 3; -// private static final long TEST_CHAT_RETRIES = 9L; -// private static final Period TEST_HTTP_TIMEOUT = new Period("PT10S"); -// private static final Period TEST_SHUTDOWN_TIMEOUT = new Period("PT80S"); -// -// private int numThreads; -// private TestingCluster zkServer; -// private TestBroker kafkaServer; -// private KinesisSupervisor supervisor; -// private String kafkaHost; -// private DataSchema dataSchema; -// private KinesisSupervisorTuningConfig tuningConfig; -// private TaskStorage taskStorage; -// private TaskMaster taskMaster; -// private TaskRunner taskRunner; -// private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; -// private KinesisIndexTaskClient taskClient; -// private TaskQueue taskQueue; -// -// @Rule -// public final TemporaryFolder tempFolder = new TemporaryFolder(); -// -// @Parameterized.Parameters(name = "numThreads = {0}") -// public static Iterable constructorFeeder() -// { -// return ImmutableList.of(new Object[]{1}, new Object[]{8}); -// } -// -// public KinesisSupervisorTest(int numThreads) -// { -// this.numThreads = numThreads; -// } -// -// @Before -// public void setUp() throws Exception -// { -// taskStorage = createMock(TaskStorage.class); -// taskMaster = createMock(TaskMaster.class); -// taskRunner = createMock(TaskRunner.class); -// indexerMetadataStorageCoordinator = createMock(IndexerMetadataStorageCoordinator.class); -// taskClient = createMock(KinesisIndexTaskClient.class); -// taskQueue = createMock(TaskQueue.class); -// -// zkServer = new TestingCluster(1); -// zkServer.start(); -// -// kafkaServer = new TestBroker( -// zkServer.getConnectString(), -// tempFolder.newFolder(), -// 1, -// ImmutableMap.of("num.partitions", String.valueOf(NUM_PARTITIONS)) -// ); -// kafkaServer.start(); -// kafkaHost = String.format("localhost:%d", kafkaServer.getPort()); -// -// dataSchema = getDataSchema(DATASOURCE); -// tuningConfig = new KinesisSupervisorTuningConfig( -// 1000, -// 50000, -// new Period("P1Y"), -// new File("/test"), -// null, -// null, -// true, -// false, -// null, -// null, -// numThreads, -// TEST_CHAT_THREADS, -// TEST_CHAT_RETRIES, -// TEST_HTTP_TIMEOUT, -// TEST_SHUTDOWN_TIMEOUT -// ); -// } -// -// @After -// public void tearDown() throws Exception -// { -// kafkaServer.close(); -// kafkaServer = null; -// -// zkServer.stop(); -// zkServer = null; -// -// supervisor = null; -// } -// -// @Test -// public void testNoInitialState() throws Exception -// { -// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); -// addSomeEvents(1); -// -// Capture captured = Capture.newInstance(); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// KinesisIndexTask task = captured.getValue(); -// Assert.assertEquals(dataSchema, task.getDataSchema()); -// Assert.assertEquals(KinesisTuningConfig.copyOf(tuningConfig), task.getTuningConfig()); -// -// KinesisIOConfig taskConfig = task.getIOConfig(); -// Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); -// Assert.assertEquals("myCustomValue", taskConfig.getConsumerProperties().get("myCustomKey")); -// Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); -// Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); -// Assert.assertFalse("pauseAfterRead", taskConfig.isPauseAfterRead()); -// Assert.assertFalse("minimumMessageTime", taskConfig.getMinimumMessageTime().isPresent()); -// Assert.assertFalse("skipOffsetGaps", taskConfig.isSkipOffsetGaps()); -// -// Assert.assertEquals(KAFKA_TOPIC, taskConfig.getStartPartitions().getStream()); -// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); -// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); -// -// Assert.assertEquals(KAFKA_TOPIC, taskConfig.getEndPartitions().getStream()); -// Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)); -// Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)); -// } -// -// @Test -// public void testSkipOffsetGaps() throws Exception -// { -// supervisor = getSupervisor(1, 1, true, "PT1H", null, true); -// addSomeEvents(1); -// -// Capture captured = Capture.newInstance(); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// KinesisIndexTask task = captured.getValue(); -// KinesisIOConfig taskConfig = task.getIOConfig(); -// -// Assert.assertTrue("skipOffsetGaps", taskConfig.isSkipOffsetGaps()); -// } -// -// @Test -// public void testMultiTask() throws Exception -// { -// supervisor = getSupervisor(1, 2, true, "PT1H", null, false); -// addSomeEvents(1); -// -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true).times(2); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// KinesisIndexTask task1 = captured.getValues().get(0); -// Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); -// Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); -// Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(Long.MAX_VALUE, (long) task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2)); -// Assert.assertEquals(Long.MAX_VALUE, (long) task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(2)); -// -// KinesisIndexTask task2 = captured.getValues().get(1); -// Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); -// Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); -// Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1)); -// Assert.assertEquals(Long.MAX_VALUE, (long) task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(1)); -// } -// -// @Test -// public void testReplicas() throws Exception -// { -// supervisor = getSupervisor(2, 1, true, "PT1H", null, false); -// addSomeEvents(1); -// -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true).times(2); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// KinesisIndexTask task1 = captured.getValues().get(0); -// Assert.assertEquals(3, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); -// Assert.assertEquals(3, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); -// Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1)); -// Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2)); -// -// KinesisIndexTask task2 = captured.getValues().get(1); -// Assert.assertEquals(3, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); -// Assert.assertEquals(3, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); -// Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1)); -// Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2)); -// } -// -// @Test -// public void testLateMessageRejectionPeriod() throws Exception -// { -// supervisor = getSupervisor(2, 1, true, "PT1H", new Period("PT1H"), false); -// addSomeEvents(1); -// -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true).times(2); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// KinesisIndexTask task1 = captured.getValues().get(0); -// KinesisIndexTask task2 = captured.getValues().get(1); -// -// Assert.assertTrue( -// "minimumMessageTime", -// task1.getIOConfig().getMinimumMessageTime().get().plusMinutes(59).isBeforeNow() -// ); -// Assert.assertTrue( -// "minimumMessageTime", -// task1.getIOConfig().getMinimumMessageTime().get().plusMinutes(61).isAfterNow() -// ); -// Assert.assertEquals( -// task1.getIOConfig().getMinimumMessageTime().get(), -// task2.getIOConfig().getMinimumMessageTime().get() -// ); -// } -// -// @Test + +package org.apache.druid.indexing.kinesis.supervisor; + +import cloud.localstack.Localstack; +import cloud.localstack.docker.LocalstackDockerTestRunner; +import cloud.localstack.docker.annotation.LocalstackDockerProperties; +import com.amazonaws.http.SdkHttpMetadata; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.model.PutRecordsRequest; +import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry; +import com.amazonaws.services.kinesis.model.PutRecordsResult; +import com.amazonaws.services.kinesis.model.PutRecordsResultEntry; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskInfoProvider; +import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.RealtimeIndexTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata; +import org.apache.druid.indexing.kinesis.KinesisIOConfig; +import org.apache.druid.indexing.kinesis.KinesisIndexTask; +import org.apache.druid.indexing.kinesis.KinesisIndexTaskClient; +import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; +import org.apache.druid.indexing.kinesis.KinesisPartitions; +import org.apache.druid.indexing.kinesis.KinesisSequenceNumber; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskQueue; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.RealtimeIOConfig; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.realtime.FireDepartment; +import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; +import org.apache.druid.server.metrics.ExceptionCapturingServiceEmitter; +import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.easymock.Capture; +import org.easymock.CaptureType; +import org.easymock.EasyMock; +import org.easymock.EasyMockSupport; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; + +import java.io.File; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.Executor; +import java.util.stream.Collectors; + +import static org.easymock.EasyMock.anyBoolean; +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.anyString; +import static org.easymock.EasyMock.capture; +import static org.easymock.EasyMock.expect; + +// TODO: improve helper methods like insertData(...) +@RunWith(LocalstackDockerTestRunner.class) +@LocalstackDockerProperties(services = {"kinesis"}) +public class KinesisSupervisorTest extends EasyMockSupport +{ + private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + private static final String TOPIC_PREFIX = "testTopic"; + private static final String DATASOURCE = "testDS"; + private static final int TEST_CHAT_THREADS = 3; + private static final long TEST_CHAT_RETRIES = 9L; + private static final Period TEST_HTTP_TIMEOUT = new Period("PT10S"); + private static final Period TEST_SHUTDOWN_TIMEOUT = new Period("PT80S"); + private static final List records = ImmutableList.of( + generateRequestEntry( + "1", + JB("2008", "a", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2009", "b", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2010", "c", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2011", "d", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2011", "e", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0") + ), + generateRequestEntry("1", StringUtils.toUtf8("unparseable")), + generateRequestEntry( + "1", + StringUtils.toUtf8("unparseable2") + ), + generateRequestEntry("1", "{}".getBytes()), + generateRequestEntry( + "1", + JB("2013", "f", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2049", "f", "y", "notanumber", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2049", "f", "y", "10", "notanumber", "1.0") + ), + generateRequestEntry( + "1", + JB("2049", "f", "y", "10", "20.0", "notanumber") + ), + generateRequestEntry( + "123123", + JB("2012", "g", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "123123", + JB("2011", "h", "y", "10", "20.0", "1.0") + ) + ); + private static String shardId1 = "shardId-000000000001"; + private static String shardId0 = "shardId-000000000000"; + + + private static DataSchema dataSchema; + private static int topicPostfix; + + private final int numThreads; + + private KinesisSupervisor supervisor; + private KinesisSupervisorTuningConfig tuningConfig; + private TaskStorage taskStorage; + private TaskMaster taskMaster; + private TaskRunner taskRunner; + private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; + private KinesisIndexTaskClient taskClient; + private TaskQueue taskQueue; + private String stream; + private RowIngestionMetersFactory rowIngestionMetersFactory; + private ExceptionCapturingServiceEmitter serviceEmitter; + + private static String getStream() + { + return TOPIC_PREFIX + topicPostfix++; + } + + public KinesisSupervisorTest() + { + this.numThreads = 1; + } + + @BeforeClass + public static void setupClass() + { + /* + * Need to disable CBOR protocol, see: + * https://github.com/mhart/kinesalite/blob/master/README.md#cbor-protocol-issues-with-the-java-sdk + */ + cloud.localstack.TestUtils.setEnv("AWS_CBOR_DISABLE", "1"); + /* Disable SSL certificate checks for local testing */ + if (Localstack.useSSL()) { + cloud.localstack.TestUtils.disableSslCertChecking(); + } + + dataSchema = getDataSchema(DATASOURCE); + } + + @Before + public void setupTest() + { + taskStorage = createMock(TaskStorage.class); + taskMaster = createMock(TaskMaster.class); + taskRunner = createMock(TaskRunner.class); + indexerMetadataStorageCoordinator = createMock(IndexerMetadataStorageCoordinator.class); + taskClient = createMock(KinesisIndexTaskClient.class); + taskQueue = createMock(TaskQueue.class); + + tuningConfig = new KinesisSupervisorTuningConfig( + 1000, + null, + 50000, + new Period("P1Y"), + new File("/test"), + null, + null, + true, + false, + null, + null, + null, + null, + numThreads, + TEST_CHAT_THREADS, + TEST_CHAT_RETRIES, + TEST_HTTP_TIMEOUT, + TEST_SHUTDOWN_TIMEOUT, + null, + null, + null, + null, + null, + null, + null, + null + ); + + stream = getStream(); + rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); + serviceEmitter = new ExceptionCapturingServiceEmitter(); + EmittingLogger.registerEmitter(serviceEmitter); + } + + @After + public void tearDownTest() + { + supervisor = null; + } + + + @Test + public void testNoInitialState() throws Exception + { + supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Capture captured = Capture.newInstance(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + KinesisIndexTask task = captured.getValue(); + Assert.assertEquals(dataSchema, task.getDataSchema()); + Assert.assertEquals(tuningConfig.copyOf(), task.getTuningConfig()); + + KinesisIOConfig taskConfig = task.getIOConfig(); + Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); + Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); + Assert.assertFalse("minimumMessageTime", taskConfig.getMinimumMessageTime().isPresent()); + Assert.assertFalse("maximumMessageTime", taskConfig.getMaximumMessageTime().isPresent()); + + Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); + Assert.assertEquals( + getSequenceNumber(res, shardId1, 0), + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + Assert.assertEquals( + getSequenceNumber(res, shardId0, 0), + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + + Assert.assertEquals(stream, taskConfig.getEndPartitions().getStream()); + Assert.assertEquals( + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + Assert.assertEquals( + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + } + + + @Test + public void testMultiTask() throws Exception + { + supervisor = getSupervisor(1, 2, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + KinesisIndexTask task1 = captured.getValues().get(0); + Assert.assertEquals(1, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(1, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals( + getSequenceNumber(res, shardId1, 0), + task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + Assert.assertEquals( + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + + KinesisIndexTask task2 = captured.getValues().get(1); + Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals( + getSequenceNumber(res, shardId0, 0), + task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + Assert.assertEquals( + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + } + + @Test + public void testReplicas() throws Exception + { + supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + KinesisIndexTask task1 = captured.getValues().get(0); + Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals( + getSequenceNumber(res, shardId0, 0), + task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + Assert.assertEquals( + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + Assert.assertEquals( + getSequenceNumber(res, shardId1, 0), + task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + Assert.assertEquals( + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + + KinesisIndexTask task2 = captured.getValues().get(1); + Assert.assertEquals(2, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(2, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals( + getSequenceNumber(res, shardId0, 0), + task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + Assert.assertEquals( + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + Assert.assertEquals( + getSequenceNumber(res, shardId1, 0), + task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + Assert.assertEquals( + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + + } + + @Test + public void testLateMessageRejectionPeriod() throws Exception + { + supervisor = getSupervisor(2, 1, true, "PT1H", new Period("PT1H"), null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + KinesisIndexTask task1 = captured.getValues().get(0); + KinesisIndexTask task2 = captured.getValues().get(1); + + Assert.assertTrue( + "minimumMessageTime", + task1.getIOConfig().getMinimumMessageTime().get().plusMinutes(59).isBeforeNow() + ); + Assert.assertTrue( + "minimumMessageTime", + task1.getIOConfig().getMinimumMessageTime().get().plusMinutes(61).isAfterNow() + ); + Assert.assertEquals( + task1.getIOConfig().getMinimumMessageTime().get(), + task2.getIOConfig().getMinimumMessageTime().get() + ); + } + + @Test + public void testEarlyMessageRejectionPeriod() throws Exception + { + supervisor = getSupervisor(2, 1, true, "PT1H", null, new Period("PT1H")); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + KinesisIndexTask task1 = captured.getValues().get(0); + KinesisIndexTask task2 = captured.getValues().get(1); + + Assert.assertTrue( + "maximumMessageTime", + task1.getIOConfig().getMaximumMessageTime().get().minusMinutes(59 + 60).isAfterNow() + ); + Assert.assertTrue( + "maximumMessageTime", + task1.getIOConfig().getMaximumMessageTime().get().minusMinutes(61 + 60).isBeforeNow() + ); + Assert.assertEquals( + task1.getIOConfig().getMaximumMessageTime().get(), + task2.getIOConfig().getMaximumMessageTime().get() + ); + } + + // @Test // /** -// * Test generating the starting offsets from the partition high water marks in Kafka. +// * Test generating the starting offsets from the partition high water marks in Kinesis. // */ // public void testLatestOffset() throws Exception // { -// supervisor = getSupervisor(1, 1, false, "PT1H", null, false); -// addSomeEvents(1100); +// final ExecutorService exec = Execs.singleThreaded("workerexec"); +// supervisor = getSupervisor(1, 1, false, "PT1H", null, null); +// AmazonKinesis kinesis = getKinesisClientInstance(); +// // // Capture captured = Capture.newInstance(); // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); // expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( // new KinesisDataSourceMetadata( // null @@ -409,258 +551,416 @@ // replayAll(); // // supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// KinesisIndexTask task = captured.getValue(); -// Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1)); -// Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2)); -// } -// -// @Test -// /** -// * Test generating the starting offsets from the partition data stored in druid_dataSource which contains the -// * offsets of the last built segments. -// */ -// public void testDatasourceMetadata() throws Exception -// { -// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); -// addSomeEvents(100); +// try { +// supervisor.runInternal(); +// } +// catch (TimeoutException e) { +// supervisor. +// List res = insertData(kinesis, generateRecordsRequests(stream)); +// supervisor.runInternal(); +// } // -// Capture captured = Capture.newInstance(); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// new KinesisPartitions(KAFKA_TOPIC, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) -// ) -// ).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true); -// replayAll(); // -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); +// verifyAll(); // // KinesisIndexTask task = captured.getValue(); // KinesisIOConfig taskConfig = task.getIOConfig(); -// Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); -// Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); -// Assert.assertEquals(30L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); -// } -// -// @Test(expected = ISE.class) -// public void testBadMetadataOffsets() throws Exception -// { -// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); -// addSomeEvents(1); -// -// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// new KinesisPartitions(KAFKA_TOPIC, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) -// ) -// ).anyTimes(); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// } -// -// @Test -// public void testKillIncompatibleTasks() throws Exception -// { -// supervisor = getSupervisor(2, 1, true, "PT1H", null, false); -// addSomeEvents(1); -// -// Task id1 = createKafkaIndexTask( // unexpected # of partitions (kill) -// "id1", -// DATASOURCE, -// "index_kinesis_testDS__some_other_sequenceName", -// new KinesisPartitions("topic", ImmutableMap.of(0, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, 10L)), -// null -// ); -// -// Task id2 = createKafkaIndexTask( // correct number of partitions and ranges (don't kill) -// "id2", -// DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 333L)), -// null -// ); -// -// Task id3 = createKafkaIndexTask( // unexpected range on partition 2 (kill) -// "id3", -// DATASOURCE, -// "index_kinesis_testDS__some_other_sequenceName", -// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 1L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 330L)), -// null -// ); -// -// Task id4 = createKafkaIndexTask( // different datasource (don't kill) -// "id4", -// "other-datasource", -// "index_kinesis_testDS_d927edff33c4b3f", -// new KinesisPartitions("topic", ImmutableMap.of(0, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, 10L)), -// null +// Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); +// Assert.assertEquals( +// getSequenceNumber(res, shardId1, 12), +// taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) // ); -// -// Task id5 = new RealtimeIndexTask( // non KinesisIndexTask (don't kill) -// "id5", -// null, -// new FireDepartment( -// dataSchema, -// new RealtimeIOConfig(null, null, null), -// null -// ), -// null +// Assert.assertEquals( +// getSequenceNumber(res, shardId0, 1), +// taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) // ); -// -// List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); -// -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.NOT_STARTED)) -// .anyTimes(); -// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); -// expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(false)); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// taskQueue.shutdown("id3"); -// -// expect(taskQueue.add(anyObject(Task.class))).andReturn(true); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); // } // -// @Test -// public void testKillBadPartitionAssignment() throws Exception -// { -// supervisor = getSupervisor(1, 2, true, "PT1H", null, false); -// addSomeEvents(1); -// -// Task id1 = createKafkaIndexTask( -// "id1", -// DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null -// ); -// Task id2 = createKafkaIndexTask( -// "id2", -// DATASOURCE, -// "sequenceName-1", -// new KinesisPartitions("topic", ImmutableMap.of(1, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE)), -// null -// ); -// Task id3 = createKafkaIndexTask( -// "id3", -// DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null -// ); -// Task id4 = createKafkaIndexTask( -// "id4", -// DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE)), -// null -// ); -// Task id5 = createKafkaIndexTask( -// "id5", -// DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(1, 0L, 2, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null -// ); -// -// List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); -// -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); -// expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).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(taskStorage.getTask("id4")).andReturn(Optional.of(id3)).anyTimes(); -// expect(taskStorage.getTask("id5")).andReturn(Optional.of(id3)).anyTimes(); -// expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.NOT_STARTED)) -// .anyTimes(); -// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); -// expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); -// expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// taskQueue.shutdown("id4"); -// taskQueue.shutdown("id5"); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// } + @Test + /** + * Test generating the starting offsets from the partition data stored in druid_dataSource which contains the + * offsets of the last built segments. + */ + public void testDatasourceMetadata() throws Exception + { + supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Capture captured = Capture.newInstance(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + )) + ) + ).anyTimes(); + + expect(taskQueue.add(capture(captured))).andReturn(true); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + KinesisIndexTask task = captured.getValue(); + KinesisIOConfig taskConfig = task.getIOConfig(); + Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); + Assert.assertEquals( + getSequenceNumber(res, shardId1, 2), + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + Assert.assertEquals( + getSequenceNumber(res, shardId0, 1), + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + } + + @Test(expected = ISE.class) + public void testBadMetadataOffsets() throws Exception + { + supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + "00000000000000000000000000000000000000000000000000000000", + shardId0, + "00000000000000000000000000000000000000000000000000000000" + )) + ) + ).anyTimes(); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + } + + @Test + public void testKillIncompatibleTasks() throws Exception + { + supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + // unexpected # of partitions (kill) + Task id1 = createKinesisIndexTask( + "id1", + DATASOURCE, + 1, + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 1) + )), + null, + null + ); + + // correct number of partitions and ranges (don't kill) + Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0), + shardId1, + getSequenceNumber(res, shardId1, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 1), + shardId1, + getSequenceNumber(res, shardId1, 12) + )), + null, + null + ); + + // unexpected range on partition 2 (kill) + Task id3 = createKinesisIndexTask( + "id3", + DATASOURCE, + 1, + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0), + shardId1, + getSequenceNumber(res, shardId1, 1) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 1), + shardId1, + getSequenceNumber(res, shardId1, 11) + )), + null, + null + ); + + // different datasource (don't kill) + Task id4 = createKinesisIndexTask( + "id4", + "other-datasource", + 2, + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0), + shardId1, + getSequenceNumber(res, shardId1, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 1), + shardId1, + getSequenceNumber(res, shardId1, 12) + )), + null, + null + ); + + // non KinesisIndexTask (don't kill) + Task id5 = new RealtimeIndexTask( + "id5", + null, + new FireDepartment( + dataSchema, + new RealtimeIOConfig(null, null, null), + null + ), + null + ); + + List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); + + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); + expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(false)); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskQueue.shutdown("id3"); + + expect(taskQueue.add(anyObject(Task.class))).andReturn(true); + + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0), + shardId1, + getSequenceNumber(res, shardId1, 0) + )); + + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(2); + + replayAll(); + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + + // TODO: delete redundant tasks + @Test + public void testKillBadPartitionAssignment() throws Exception + { + supervisor = getSupervisor(1, 2, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Task id1 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 12) + )), + null, + null + ); + Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 1, + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 1) + )), + null, + null + ); + Task id3 = createKinesisIndexTask( + "id3", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0), + shardId1, + getSequenceNumber(res, shardId1, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 1), + shardId1, + getSequenceNumber(res, shardId1, 12) + )), + null, + null + ); + Task id4 = createKinesisIndexTask( + "id4", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 1) + )), + null, + null + ); + Task id5 = createKinesisIndexTask( + "id5", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 1) + )), + null, + null + ); + + List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); + + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); + expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).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(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); + expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); + expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); + expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of(shardId1, getSequenceNumber(res, shardId1, 0))); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) + .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)); + taskQueue.shutdown("id4"); + taskQueue.shutdown("id5"); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } // // @Test // public void testRequeueTaskWhenFailed() throws Exception // { -// supervisor = getSupervisor(2, 2, true, "PT1H", null, false); +// supervisor = getSupervisor(2, 2, true, "PT1H", null, null, false); // addSomeEvents(1); // // Capture captured = Capture.newInstance(CaptureType.ALL); // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); // expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); // expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.NOT_STARTED)) +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) // .anyTimes(); -// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); +// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); // expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( // new KinesisDataSourceMetadata( // null // ) // ).anyTimes(); // expect(taskQueue.add(capture(captured))).andReturn(true).times(4); +// +// TreeMap> checkpoints1 = new TreeMap<>(); +// checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); +// TreeMap> checkpoints2 = new TreeMap<>(); +// checkpoints2.put(0, ImmutableMap.of(1, 0L)); +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints1)) +// .anyTimes(); +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints2)) +// .anyTimes(); +// // taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); // replayAll(); // @@ -713,17 +1013,19 @@ // @Test // public void testRequeueAdoptedTaskWhenFailed() throws Exception // { -// supervisor = getSupervisor(2, 1, true, "PT1H", null, false); +// supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false); // addSomeEvents(1); // -// DateTime now = DateTime.now(); -// Task id1 = createKafkaIndexTask( +// DateTime now = DateTimes.nowUtc(); +// DateTime maxi = now.plusMinutes(60); +// Task id1 = createKinesisIndexTask( // "id1", // DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// now +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 2, 0L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// now, +// maxi // ); // // List existingTasks = ImmutableList.of(id1); @@ -735,7 +1037,7 @@ // expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); // expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); // expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); // expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes(); // expect(taskQueue.add(capture(captured))).andReturn(true); // expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( @@ -743,6 +1045,13 @@ // null // ) // ).anyTimes(); +// +// TreeMap> checkpoints = new TreeMap<>(); +// checkpoints.put(0, ImmutableMap.of(0, 0L, 2, 0L)); +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints)) +// .times(2); +// // taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); // replayAll(); // @@ -760,12 +1069,21 @@ // reset(taskStorage); // reset(taskQueue); // reset(taskClient); +// +// // for the newly created replica task +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints)) +// .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.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); // expect(taskStorage.getStatus(runningTaskId)).andReturn(Optional.of(TaskStatus.running(runningTaskId))).anyTimes(); // expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); // expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); -// expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); +// expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(Status.READING)); // expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes(); // expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); // replay(taskStorage); @@ -783,23 +1101,30 @@ // // // check that failed tasks are recreated with the same minimumMessageTime as the task it replaced, even if that // // task came from another supervisor -// Assert.assertEquals(now, ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getMinimumMessageTime().get()); +// Assert.assertEquals( +// now, +// ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getMinimumMessageTime().get() +// ); +// Assert.assertEquals( +// maxi, +// ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getMaximumMessageTime().get() +// ); // } // // @Test // public void testQueueNextTasksOnSuccess() throws Exception // { -// supervisor = getSupervisor(2, 2, true, "PT1H", null, false); +// supervisor = getSupervisor(2, 2, true, "PT1H", null, null, false); // addSomeEvents(1); // // Capture captured = Capture.newInstance(CaptureType.ALL); // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); // expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); // expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.NOT_STARTED)) +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) // .anyTimes(); -// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTime.now())).anyTimes(); +// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); // expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( // new KinesisDataSourceMetadata( // null @@ -807,6 +1132,7 @@ // ).anyTimes(); // expect(taskQueue.add(capture(captured))).andReturn(true).times(4); // taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// // replayAll(); // // supervisor.start(); @@ -816,12 +1142,30 @@ // List tasks = captured.getValues(); // // reset(taskStorage); +// reset(taskClient); +// +// expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) +// .anyTimes(); +// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); +// TreeMap> checkpoints1 = new TreeMap<>(); +// checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); +// TreeMap> checkpoints2 = new TreeMap<>(); +// checkpoints2.put(0, ImmutableMap.of(1, 0L)); +// // there would be 4 tasks, 2 for each task group +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints1)) +// .times(2); +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints2)) +// .times(2); +// // expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); // for (Task task : tasks) { // expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); // expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); // } // replay(taskStorage); +// replay(taskClient); // // supervisor.runInternal(); // verifyAll(); @@ -858,16 +1202,16 @@ // @Test // public void testBeginPublishAndQueueNextTasks() throws Exception // { -// final TaskLocation location = new TaskLocation("testHost", 1234); +// final TaskLocation location = new TaskLocation("testHost", 1234, -1); // -// supervisor = getSupervisor(2, 2, true, "PT1M", null, false); +// supervisor = getSupervisor(2, 2, true, "PT1M", null, null, false); // addSomeEvents(100); // // Capture captured = Capture.newInstance(CaptureType.ALL); // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); // expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); // expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); // expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( // new KinesisDataSourceMetadata( // null @@ -884,7 +1228,7 @@ // List tasks = captured.getValues(); // Collection workItems = new ArrayList<>(); // for (Task task : tasks) { -// workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); +// workItems.add(new TestTaskRunnerWorkItem(task, null, location)); // } // // reset(taskStorage, taskRunner, taskClient, taskQueue); @@ -896,17 +1240,17 @@ // } // expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); // expect(taskClient.getStatusAsync(anyString())) -// .andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)) +// .andReturn(Futures.immediateFuture(Status.READING)) // .anyTimes(); // expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) -// .andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2))) -// .andReturn(Futures.immediateFuture(DateTime.now())); +// .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) +// .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); // expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) -// .andReturn(Futures.immediateFuture(DateTime.now())) +// .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) // .times(2); // expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); // expect( // taskClient.setEndOffsetsAsync( // EasyMock.contains("sequenceName-0"), @@ -916,22 +1260,32 @@ // ).andReturn(Futures.immediateFuture(true)).times(2); // expect(taskQueue.add(capture(captured))).andReturn(true).times(2); // +// TreeMap> checkpoints1 = new TreeMap<>(); +// checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); +// TreeMap> checkpoints2 = new TreeMap<>(); +// checkpoints2.put(0, ImmutableMap.of(1, 0L)); +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints1)) +// .times(2); +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints2)) +// .times(2); +// // replay(taskStorage, taskRunner, taskClient, taskQueue); // // supervisor.runInternal(); // verifyAll(); // // for (Task task : captured.getValues()) { -// KinesisIndexTask kinesisIndexTask = (KinesisIndexTask) task; -// Assert.assertEquals(dataSchema, kinesisIndexTask.getDataSchema()); -// Assert.assertEquals(KinesisTuningConfig.copyOf(tuningConfig), kinesisIndexTask.getTuningConfig()); +// KinesisIndexTask KinesisIndexTask = (KinesisIndexTask) task; +// Assert.assertEquals(dataSchema, KinesisIndexTask.getDataSchema()); +// Assert.assertEquals(tuningConfig.copyOf(), KinesisIndexTask.getTuningConfig()); // -// KinesisIOConfig taskConfig = kinesisIndexTask.getIOConfig(); +// KinesisSupervisor taskConfig = KinesisIndexTask.getIOConfig(); // Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); // Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); -// Assert.assertFalse("pauseAfterRead", taskConfig.isPauseAfterRead()); // -// Assert.assertEquals(KAFKA_TOPIC, taskConfig.getStartPartitions().getStream()); +// Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); // Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); // Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); // Assert.assertEquals(35L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); @@ -941,22 +1295,23 @@ // @Test // public void testDiscoverExistingPublishingTask() throws Exception // { -// final TaskLocation location = new TaskLocation("testHost", 1234); +// final TaskLocation location = new TaskLocation("testHost", 1234, -1); // -// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); // addSomeEvents(1); // -// Task task = createKafkaIndexTask( +// Task task = createKinesisIndexTask( // "id1", // DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, // null // ); // // Collection workItems = new ArrayList<>(); -// workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); +// workItems.add(new TestTaskRunnerWorkItem(task, null, location)); // // Capture captured = Capture.newInstance(); // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -970,31 +1325,34 @@ // null // ) // ).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.PUBLISHING)); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); // expect(taskClient.getCurrentOffsetsAsync("id1", false)) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); -// expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); +// expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); // expect(taskQueue.add(capture(captured))).andReturn(true); // +// TreeMap> checkpoints = new TreeMap<>(); +// checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)); +// expect(taskClient.getCheckpoints(anyString(), anyBoolean())).andReturn(checkpoints).anyTimes(); +// // taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); // replayAll(); // // supervisor.start(); // supervisor.runInternal(); -// SupervisorReport report = supervisor.getStatus(); +// supervisor.updateCurrentAndLatestOffsets().run(); +// SupervisorReport report = supervisor.getStatus(); // verifyAll(); // // Assert.assertEquals(DATASOURCE, report.getId()); -// Assert.assertTrue(report.getPayload() instanceof KinesisSupervisorReport.KinesisSupervisorReportPayload); // -// KinesisSupervisorReport.KinesisSupervisorReportPayload payload = (KinesisSupervisorReport.KinesisSupervisorReportPayload) -// report.getPayload(); +// KinesisSupervisorReportPayload payload = report.getPayload(); // // Assert.assertEquals(DATASOURCE, payload.getDataSource()); // Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); // Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); // Assert.assertEquals(1, (int) payload.getReplicas()); -// Assert.assertEquals(KAFKA_TOPIC, payload.getTopic()); +// Assert.assertEquals(stream, payload.getStream()); // Assert.assertEquals(0, payload.getActiveTasks().size()); // Assert.assertEquals(1, payload.getPublishingTasks().size()); // @@ -1006,22 +1364,21 @@ // // KinesisIndexTask capturedTask = captured.getValue(); // Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); -// Assert.assertEquals(KinesisTuningConfig.copyOf(tuningConfig), capturedTask.getTuningConfig()); +// Assert.assertEquals(tuningConfig.copyOf(), capturedTask.getTuningConfig()); // -// KinesisIOConfig capturedTaskConfig = capturedTask.getIOConfig(); +// KinesisSupervisor capturedTaskConfig = capturedTask.getIOConfig(); // Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); // Assert.assertEquals("myCustomValue", capturedTaskConfig.getConsumerProperties().get("myCustomKey")); // Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); // Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); -// Assert.assertFalse("pauseAfterRead", capturedTaskConfig.isPauseAfterRead()); // // // check that the new task was created with starting offsets matching where the publishing task finished -// Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getStartPartitions().getStream()); +// Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); // Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); // Assert.assertEquals(20L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); // Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); // -// Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getEndPartitions().getStream()); +// Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream()); // Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)); // Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)); // Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)); @@ -1030,22 +1387,23 @@ // @Test // public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() throws Exception // { -// final TaskLocation location = new TaskLocation("testHost", 1234); +// final TaskLocation location = new TaskLocation("testHost", 1234, -1); // -// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); // addSomeEvents(1); // -// Task task = createKafkaIndexTask( +// Task task = createKinesisIndexTask( // "id1", // DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 2, 0L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, // null // ); // // Collection workItems = new ArrayList<>(); -// workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); +// workItems.add(new TestTaskRunnerWorkItem(task, null, location)); // // Capture captured = Capture.newInstance(); // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -1059,10 +1417,10 @@ // null // ) // ).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.PUBLISHING)); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); // expect(taskClient.getCurrentOffsetsAsync("id1", false)) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 2, 30L))); -// expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 2, 30L))); +// expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); // expect(taskQueue.add(capture(captured))).andReturn(true); // // taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); @@ -1070,20 +1428,19 @@ // // supervisor.start(); // supervisor.runInternal(); -// SupervisorReport report = supervisor.getStatus(); +// supervisor.updateCurrentAndLatestOffsets().run(); +// SupervisorReport report = supervisor.getStatus(); // verifyAll(); // // Assert.assertEquals(DATASOURCE, report.getId()); -// Assert.assertTrue(report.getPayload() instanceof KinesisSupervisorReport.KinesisSupervisorReportPayload); // -// KinesisSupervisorReport.KinesisSupervisorReportPayload payload = (KinesisSupervisorReport.KinesisSupervisorReportPayload) -// report.getPayload(); +// KinesisSupervisorReportPayload payload = report.getPayload(); // // Assert.assertEquals(DATASOURCE, payload.getDataSource()); // Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); // Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); // Assert.assertEquals(1, (int) payload.getReplicas()); -// Assert.assertEquals(KAFKA_TOPIC, payload.getTopic()); +// Assert.assertEquals(stream, payload.getStream()); // Assert.assertEquals(0, payload.getActiveTasks().size()); // Assert.assertEquals(1, payload.getPublishingTasks().size()); // @@ -1095,22 +1452,21 @@ // // KinesisIndexTask capturedTask = captured.getValue(); // Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); -// Assert.assertEquals(KinesisTuningConfig.copyOf(tuningConfig), capturedTask.getTuningConfig()); +// Assert.assertEquals(tuningConfig.copyOf(), capturedTask.getTuningConfig()); // -// KinesisIOConfig capturedTaskConfig = capturedTask.getIOConfig(); +// KinesisSupervisor capturedTaskConfig = capturedTask.getIOConfig(); // Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); // Assert.assertEquals("myCustomValue", capturedTaskConfig.getConsumerProperties().get("myCustomKey")); // Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); // Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); -// Assert.assertFalse("pauseAfterRead", capturedTaskConfig.isPauseAfterRead()); // // // check that the new task was created with starting offsets matching where the publishing task finished -// Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getStartPartitions().getStream()); +// Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); // Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); // Assert.assertEquals(0L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); // Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); // -// Assert.assertEquals(KAFKA_TOPIC, capturedTaskConfig.getEndPartitions().getStream()); +// Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream()); // Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)); // Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)); // Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)); @@ -1119,34 +1475,36 @@ // @Test // public void testDiscoverExistingPublishingAndReadingTask() throws Exception // { -// final TaskLocation location1 = new TaskLocation("testHost", 1234); -// final TaskLocation location2 = new TaskLocation("testHost2", 145); -// final DateTime startTime = new DateTime(); +// final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); +// final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); +// final DateTime startTime = DateTimes.nowUtc(); // -// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); -// addSomeEvents(1); +// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); +// addSomeEvents(6); // -// Task id1 = createKafkaIndexTask( +// Task id1 = createKinesisIndexTask( // "id1", // DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, // null // ); // -// Task id2 = createKafkaIndexTask( +// Task id2 = createKinesisIndexTask( // "id2", // DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, // null // ); // // Collection workItems = new ArrayList<>(); -// workItems.add(new TestTaskRunnerWorkItem(id1.getId(), null, location1)); -// workItems.add(new TestTaskRunnerWorkItem(id2.getId(), null, location2)); +// workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); +// workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); // // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); // expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); @@ -1161,34 +1519,41 @@ // null // ) // ).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.PUBLISHING)); -// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); +// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); // expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); // expect(taskClient.getCurrentOffsetsAsync("id1", false)) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); -// expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 1L, 1, 2L, 2, 3L))); +// expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); // expect(taskClient.getCurrentOffsetsAsync("id2", false)) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 40L, 1, 50L, 2, 60L))); +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 4L, 1, 5L, 2, 6L))); // // taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// +// // since id1 is publishing, so getCheckpoints wouldn't be called for it +// TreeMap> checkpoints = new TreeMap<>(); +// checkpoints.put(0, ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints)) +// .times(1); +// // replayAll(); // // supervisor.start(); // supervisor.runInternal(); -// SupervisorReport report = supervisor.getStatus(); +// supervisor.updateCurrentAndLatestOffsets().run(); +// SupervisorReport report = supervisor.getStatus(); // verifyAll(); // // Assert.assertEquals(DATASOURCE, report.getId()); -// Assert.assertTrue(report.getPayload() instanceof KinesisSupervisorReport.KinesisSupervisorReportPayload); // -// KinesisSupervisorReport.KinesisSupervisorReportPayload payload = (KinesisSupervisorReport.KinesisSupervisorReportPayload) -// report.getPayload(); +// KinesisSupervisorReportPayload payload = report.getPayload(); // // Assert.assertEquals(DATASOURCE, payload.getDataSource()); // Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); // Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); // Assert.assertEquals(1, (int) payload.getReplicas()); -// Assert.assertEquals(KAFKA_TOPIC, payload.getTopic()); +// Assert.assertEquals(stream, payload.getStream()); // Assert.assertEquals(1, payload.getActiveTasks().size()); // Assert.assertEquals(1, payload.getPublishingTasks().size()); // @@ -1197,25 +1562,32 @@ // // Assert.assertEquals("id2", activeReport.getId()); // Assert.assertEquals(startTime, activeReport.getStartTime()); -// Assert.assertEquals(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), activeReport.getStartingOffsets()); -// Assert.assertEquals(ImmutableMap.of(0, 40L, 1, 50L, 2, 60L), activeReport.getCurrentOffsets()); +// Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L), activeReport.getStartingOffsets()); +// Assert.assertEquals(ImmutableMap.of(0, 4L, 1, 5L, 2, 6L), activeReport.getCurrentOffsets()); +// Assert.assertEquals(ImmutableMap.of(0, 2L, 1, 1L, 2, 0L), activeReport.getLag()); // // Assert.assertEquals("id1", publishingReport.getId()); // Assert.assertEquals(ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), publishingReport.getStartingOffsets()); -// Assert.assertEquals(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), publishingReport.getCurrentOffsets()); +// Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L), publishingReport.getCurrentOffsets()); +// Assert.assertEquals(null, publishingReport.getLag()); +// +// Assert.assertEquals(ImmutableMap.of(0, 6L, 1, 6L, 2, 6L), payload.getLatestOffsets()); +// Assert.assertEquals(ImmutableMap.of(0, 2L, 1, 1L, 2, 0L), payload.getMinimumLag()); +// Assert.assertEquals(3L, (long) payload.getAggregateLag()); +// Assert.assertTrue(payload.getOffsetsLastUpdated().plusMinutes(1).isAfterNow()); // } // // @Test // public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception // { -// supervisor = getSupervisor(2, 2, true, "PT1H", null, false); +// supervisor = getSupervisor(2, 2, true, "PT1H", null, null, false); // addSomeEvents(1); // // Capture captured = Capture.newInstance(CaptureType.ALL); // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); // expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); // expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); // expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( // new KinesisDataSourceMetadata( // null @@ -1231,15 +1603,27 @@ // // List tasks = captured.getValues(); // -// reset(taskStorage, taskClient, taskQueue); +// reset(taskStorage, taskClient, taskQueue); +// +// TreeMap> checkpoints1 = new TreeMap<>(); +// checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); +// TreeMap> checkpoints2 = new TreeMap<>(); +// checkpoints2.put(0, ImmutableMap.of(1, 0L)); +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints1)) +// .times(2); +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints2)) +// .times(2); +// // expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); // for (Task task : tasks) { // expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); // expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); // expect(taskClient.getStatusAsync(task.getId())) -// .andReturn(Futures.immediateFuture(KinesisIndexTask.Status.NOT_STARTED)); +// .andReturn(Futures.immediateFuture(Status.NOT_STARTED)); // expect(taskClient.getStartTimeAsync(task.getId())) -// .andReturn(Futures.immediateFailedFuture(new RuntimeException())); +// .andReturn(Futures.immediateFailedFuture(new RuntimeException())); // taskQueue.shutdown(task.getId()); // } // replay(taskStorage, taskClient, taskQueue); @@ -1251,16 +1635,16 @@ // @Test // public void testKillUnresponsiveTasksWhilePausing() throws Exception // { -// final TaskLocation location = new TaskLocation("testHost", 1234); +// final TaskLocation location = new TaskLocation("testHost", 1234, -1); // -// supervisor = getSupervisor(2, 2, true, "PT1M", null, false); +// supervisor = getSupervisor(2, 2, true, "PT1M", null, null, false); // addSomeEvents(100); // // Capture captured = Capture.newInstance(CaptureType.ALL); // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); // expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); // expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); // expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( // new KinesisDataSourceMetadata( // null @@ -1277,10 +1661,22 @@ // List tasks = captured.getValues(); // Collection workItems = new ArrayList<>(); // for (Task task : tasks) { -// workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); +// workItems.add(new TestTaskRunnerWorkItem(task, null, location)); // } // // reset(taskStorage, taskRunner, taskClient, taskQueue); +// +// TreeMap> checkpoints1 = new TreeMap<>(); +// checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); +// TreeMap> checkpoints2 = new TreeMap<>(); +// checkpoints2.put(0, ImmutableMap.of(1, 0L)); +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints1)) +// .times(2); +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints2)) +// .times(2); +// // captured = Capture.newInstance(CaptureType.ALL); // expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); // for (Task task : tasks) { @@ -1289,16 +1685,16 @@ // } // expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); // expect(taskClient.getStatusAsync(anyString())) -// .andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)) +// .andReturn(Futures.immediateFuture(Status.READING)) // .anyTimes(); // expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) -// .andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2))) -// .andReturn(Futures.immediateFuture(DateTime.now())); +// .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) +// .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); // expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) -// .andReturn(Futures.immediateFuture(DateTime.now())) +// .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) // .times(2); // expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) -// .andReturn(Futures.>immediateFailedFuture(new RuntimeException())).times(2); +// .andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); // taskQueue.shutdown(EasyMock.contains("sequenceName-0")); // expectLastCall().times(2); // expect(taskQueue.add(capture(captured))).andReturn(true).times(2); @@ -1309,7 +1705,7 @@ // verifyAll(); // // for (Task task : captured.getValues()) { -// KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); +// KinesisSupervisor taskConfig = ((KinesisIndexTask) task).getIOConfig(); // Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); // Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); // } @@ -1318,16 +1714,16 @@ // @Test // public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception // { -// final TaskLocation location = new TaskLocation("testHost", 1234); +// final TaskLocation location = new TaskLocation("testHost", 1234, -1); // -// supervisor = getSupervisor(2, 2, true, "PT1M", null, false); +// supervisor = getSupervisor(2, 2, true, "PT1M", null, null, false); // addSomeEvents(100); // // Capture captured = Capture.newInstance(CaptureType.ALL); // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); // expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); // expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); // expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( // new KinesisDataSourceMetadata( // null @@ -1344,10 +1740,22 @@ // List tasks = captured.getValues(); // Collection workItems = new ArrayList<>(); // for (Task task : tasks) { -// workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); +// workItems.add(new TestTaskRunnerWorkItem(task, null, location)); // } // // reset(taskStorage, taskRunner, taskClient, taskQueue); +// +// TreeMap> checkpoints1 = new TreeMap<>(); +// checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); +// TreeMap> checkpoints2 = new TreeMap<>(); +// checkpoints2.put(0, ImmutableMap.of(1, 0L)); +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints1)) +// .times(2); +// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints2)) +// .times(2); +// // captured = Capture.newInstance(CaptureType.ALL); // expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); // for (Task task : tasks) { @@ -1356,24 +1764,24 @@ // } // expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); // expect(taskClient.getStatusAsync(anyString())) -// .andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)) +// .andReturn(Futures.immediateFuture(Status.READING)) // .anyTimes(); // expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) -// .andReturn(Futures.immediateFuture(DateTime.now().minusMinutes(2))) -// .andReturn(Futures.immediateFuture(DateTime.now())); +// .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) +// .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); // expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) -// .andReturn(Futures.immediateFuture(DateTime.now())) +// .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) // .times(2); // expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); // expect( // taskClient.setEndOffsetsAsync( // EasyMock.contains("sequenceName-0"), // EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), // EasyMock.eq(true) // ) -// ).andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); +// ).andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); // taskQueue.shutdown(EasyMock.contains("sequenceName-0")); // expectLastCall().times(2); // expect(taskQueue.add(capture(captured))).andReturn(true).times(2); @@ -1384,78 +1792,465 @@ // verifyAll(); // // for (Task task : captured.getValues()) { -// KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); +// KinesisSupervisor taskConfig = ((KinesisIndexTask) task).getIOConfig(); // Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); // Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); // } // } // // @Test(expected = IllegalStateException.class) -// public void testStopNotStarted() throws Exception +// public void testStopNotStarted() +// { +// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); +// supervisor.stop(false); +// } +// +// @Test +// public void testStop() // { -// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// taskClient.close(); +// taskRunner.unregisterListener(StringUtils.format("KinesisSupervisor-%s", DATASOURCE)); +// replayAll(); +// +// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); +// supervisor.start(); // supervisor.stop(false); +// +// verifyAll(); +// } +// +// @Test +// public void testStopGracefully() throws Exception +// { +// final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); +// final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); +// final DateTime startTime = DateTimes.nowUtc(); +// +// supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false); +// addSomeEvents(1); +// +// Task id1 = createKinesisIndexTask( +// "id1", +// DATASOURCE, +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, +// null +// ); +// +// Task id2 = createKinesisIndexTask( +// "id2", +// DATASOURCE, +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, +// null +// ); +// +// Task id3 = createKinesisIndexTask( +// "id3", +// DATASOURCE, +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, +// null +// ); +// +// Collection workItems = new ArrayList<>(); +// workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); +// workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); +// +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(workItems).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 KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); +// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); +// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); +// expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); +// expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); +// expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); +// +// // getCheckpoints will not be called for id1 as it is in publishing state +// TreeMap> checkpoints = new TreeMap<>(); +// 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("id3"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints)) +// .times(1); +// +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// reset(taskRunner, taskClient, taskQueue); +// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); +// expect(taskClient.pauseAsync("id2")) +// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); +// expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) +// .andReturn(Futures.immediateFuture(true)); +// taskQueue.shutdown("id3"); +// expectLastCall().times(2); +// +// replay(taskRunner, taskClient, taskQueue); +// +// supervisor.gracefulShutdownInternal(); +// verifyAll(); +// } +// +// @Test +// public void testResetNoTasks() throws Exception +// { +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// reset(indexerMetadataStorageCoordinator); +// expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); +// replay(indexerMetadataStorageCoordinator); +// +// supervisor.resetInternal(null); +// verifyAll(); +// +// } +// +// @Test +// public void testResetDataSourceMetadata() throws Exception +// { +// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// Capture captureDataSource = EasyMock.newCapture(); +// Capture captureDataSourceMetadata = EasyMock.newCapture(); +// +// KinesisDataSourceMetadata KinesisDataSourceMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( +// stream, +// ImmutableMap.of(0, 1000L, 1, 1000L, 2, 1000L) +// )); +// +// KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( +// stream, +// ImmutableMap.of(1, 1000L, 2, 1000L) +// )); +// +// KinesisDataSourceMetadata expectedMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( +// stream, +// ImmutableMap.of(0, 1000L) +// )); +// +// reset(indexerMetadataStorageCoordinator); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(KinesisDataSourceMetadata); +// expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata( +// EasyMock.capture(captureDataSource), +// EasyMock.capture(captureDataSourceMetadata) +// )).andReturn(true); +// replay(indexerMetadataStorageCoordinator); +// +// try { +// supervisor.resetInternal(resetMetadata); +// } +// catch (NullPointerException npe) { +// // Expected as there will be an attempt to reset partitionGroups offsets to NOT_SET +// // however there would be no entries in the map as we have not put nay data in kafka +// Assert.assertTrue(npe.getCause() == null); +// } +// verifyAll(); +// +// Assert.assertEquals(captureDataSource.getValue(), DATASOURCE); +// Assert.assertEquals(captureDataSourceMetadata.getValue(), expectedMetadata); +// } +// +// @Test +// public void testResetNoDataSourceMetadata() throws Exception +// { +// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( +// stream, +// ImmutableMap.of(1, 1000L, 2, 1000L) +// )); +// +// reset(indexerMetadataStorageCoordinator); +// // no DataSourceMetadata in metadata store +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null); +// replay(indexerMetadataStorageCoordinator); +// +// supervisor.resetInternal(resetMetadata); +// verifyAll(); +// } +// +// @Test +// public void testResetRunningTasks() throws Exception +// { +// final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); +// final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); +// final DateTime startTime = DateTimes.nowUtc(); +// +// supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false); +// addSomeEvents(1); +// +// Task id1 = createKinesisIndexTask( +// "id1", +// DATASOURCE, +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, +// null +// ); +// +// Task id2 = createKinesisIndexTask( +// "id2", +// DATASOURCE, +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, +// null +// ); +// +// Task id3 = createKinesisIndexTask( +// "id3", +// DATASOURCE, +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, +// null +// ); +// +// Collection workItems = new ArrayList<>(); +// workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); +// workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); +// +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(workItems).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 KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); +// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); +// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); +// expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); +// expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); +// expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); +// +// TreeMap> checkpoints = new TreeMap<>(); +// 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("id3"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints)) +// .times(1); +// +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); +// +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); +// +// reset(taskQueue, indexerMetadataStorageCoordinator); +// expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); +// taskQueue.shutdown("id2"); +// taskQueue.shutdown("id3"); +// replay(taskQueue, indexerMetadataStorageCoordinator); +// +// supervisor.resetInternal(null); +// verifyAll(); // } // -// @Test -// public void testStop() throws Exception -// { +// @Test +// public void testNoDataIngestionTasks() throws Exception +// { +// final DateTime startTime = DateTimes.nowUtc(); +// supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); +// //not adding any events +// Task id1 = createKinesisIndexTask( +// "id1", +// DATASOURCE, +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, +// null +// ); +// +// Task id2 = createKinesisIndexTask( +// "id2", +// DATASOURCE, +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, +// null +// ); +// +// Task id3 = createKinesisIndexTask( +// "id3", +// DATASOURCE, +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions("stream", 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(); -// taskClient.close(); -// taskRunner.unregisterListener(String.format("KinesisSupervisor-%s", DATASOURCE)); +// 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 KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); +// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); +// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); +// expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); +// expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); +// expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); +// +// TreeMap> 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 = getSupervisor(1, 1, true, "PT1H", null, false); // supervisor.start(); -// supervisor.stop(false); +// supervisor.runInternal(); +// verifyAll(); +// +// reset(taskQueue, indexerMetadataStorageCoordinator); +// expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); +// taskQueue.shutdown("id1"); +// taskQueue.shutdown("id2"); +// taskQueue.shutdown("id3"); +// replay(taskQueue, indexerMetadataStorageCoordinator); // +// supervisor.resetInternal(null); // verifyAll(); // } // -// @Test -// public void testStopGracefully() throws Exception +// @Test(timeout = 60_000L) +// public void testCheckpointForInactiveTaskGroup() +// throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException // { -// final TaskLocation location1 = new TaskLocation("testHost", 1234); -// final TaskLocation location2 = new TaskLocation("testHost2", 145); -// final DateTime startTime = new DateTime(); -// -// supervisor = getSupervisor(2, 1, true, "PT1H", null, false); -// addSomeEvents(1); -// -// Task id1 = createKafkaIndexTask( +// supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); +// //not adding any events +// final Task id1 = createKinesisIndexTask( // "id1", // DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// 0, +// new KinesisPartitions(stream, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, // null // ); // -// Task id2 = createKafkaIndexTask( +// final Task id2 = createKinesisIndexTask( // "id2", // DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// 0, +// new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, // null // ); // -// Task id3 = createKafkaIndexTask( +// final Task id3 = createKinesisIndexTask( // "id3", // DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// 0, +// new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, // null // ); // +// final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); +// final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); // Collection workItems = new ArrayList<>(); -// workItems.add(new TestTaskRunnerWorkItem(id1.getId(), null, location1)); -// workItems.add(new TestTaskRunnerWorkItem(id2.getId(), null, location2)); +// workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); +// workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); +// workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); // +// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); // expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(workItems).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(); @@ -1463,181 +2258,296 @@ // 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 KinesisDataSourceMetadata( -// null -// ) +// expect( +// indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KinesisDataSourceMetadata( +// null) // ).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.PUBLISHING)); -// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); -// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); +// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); +// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(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)); -// expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); +// +// final TreeMap> 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(); -// verifyAll(); // -// reset(taskRunner, taskClient, taskQueue); -// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); -// expect(taskClient.pauseAsync("id2")) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); -// expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) -// .andReturn(Futures.immediateFuture(true)); -// taskQueue.shutdown("id3"); -// expectLastCall().times(2); +// final Map fakeCheckpoints = Collections.emptyMap(); +// supervisor.moveTaskGroupToPendingCompletion(0); +// supervisor.checkpoint( +// 0, +// ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, checkpoints.get(0))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, fakeCheckpoints)) +// ); // -// replay(taskRunner, taskClient, taskQueue); +// while (supervisor.getNoticesQueueSize() > 0) { +// Thread.sleep(100); +// } // -// supervisor.gracefulShutdownInternal(); // verifyAll(); +// +// Assert.assertNull(serviceEmitter.getStackTrace(), serviceEmitter.getStackTrace()); +// Assert.assertNull(serviceEmitter.getExceptionMessage(), serviceEmitter.getExceptionMessage()); +// Assert.assertNull(serviceEmitter.getExceptionClass()); // } // -// @Test -// public void testResetNoTasks() throws Exception +// @Test(timeout = 60_000L) +// public void testCheckpointForUnknownTaskGroup() throws InterruptedException // { -// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); +// //not adding any events +// final Task id1 = createKinesisIndexTask( +// "id1", +// DATASOURCE, +// 0, +// new KinesisPartitions(stream, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, +// null +// ); +// +// final Task id2 = createKinesisIndexTask( +// "id2", +// DATASOURCE, +// 0, +// new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, +// null +// ); +// +// final Task id3 = createKinesisIndexTask( +// "id3", +// DATASOURCE, +// 0, +// new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions(stream, 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(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// 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 KinesisDataSourceMetadata( +// null) +// ).anyTimes(); +// // replayAll(); // // supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); // -// reset(indexerMetadataStorageCoordinator); -// expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); -// replay(indexerMetadataStorageCoordinator); +// supervisor.checkpoint( +// 0, +// ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, Collections.emptyMap())), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, Collections.emptyMap())) +// ); +// +// while (supervisor.getNoticesQueueSize() > 0) { +// Thread.sleep(100); +// } // -// supervisor.resetInternal(null); // verifyAll(); // +// while (serviceEmitter.getStackTrace() == null) { +// Thread.sleep(100); +// } +// +// Assert.assertTrue(serviceEmitter.getStackTrace() +// .startsWith("org.apache.druid.java.util.common.ISE: WTH?! cannot find")); +// Assert.assertEquals( +// "WTH?! cannot find taskGroup [0] among all taskGroups [{}]", +// serviceEmitter.getExceptionMessage() +// ); +// Assert.assertEquals(ISE.class, serviceEmitter.getExceptionClass()); // } // -// @Test -// public void testResetDataSourceMetadata() throws Exception +// @Test(timeout = 60_000L) +// public void testCheckpointWithNullTaskGroupId() +// throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException // { -// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// supervisor = getSupervisor(1, 3, true, "PT1S", null, null, false); +// //not adding any events +// final Task id1 = createKinesisIndexTask( +// "id1", +// DATASOURCE, +// 0, +// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE)), +// null, +// null +// ); +// +// final Task id2 = createKinesisIndexTask( +// "id2", +// DATASOURCE, +// 0, +// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE)), +// null, +// null +// ); +// +// final Task id3 = createKinesisIndexTask( +// "id3", +// DATASOURCE, +// 0, +// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), +// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE)), +// null, +// null +// ); +// // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); // expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).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 KinesisDataSourceMetadata( +// null) +// ).anyTimes(); // taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// expect(taskClient.getStatusAsync(anyString())) +// .andReturn(Futures.immediateFuture(Status.READING)) +// .anyTimes(); +// final TreeMap> checkpoints = new TreeMap<>(); +// checkpoints.put(0, ImmutableMap.of(0, 0L)); +// expect(taskClient.getCheckpointsAsync(anyString(), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints)) +// .times(3); +// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); +// expect(taskClient.pauseAsync(anyString())) +// .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L))) +// .anyTimes(); +// expect(taskClient.setEndOffsetsAsync(anyString(), EasyMock.eq(ImmutableMap.of(0, 10L)), anyBoolean())) +// .andReturn(Futures.immediateFuture(true)) +// .anyTimes(); +// // replayAll(); // // supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// Capture captureDataSource = EasyMock.newCapture(); -// Capture captureDataSourceMetadata = EasyMock.newCapture(); // -// KinesisDataSourceMetadata kinesisDataSourceMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( -// KAFKA_TOPIC, -// ImmutableMap.of(0, 1000L, 1, 1000L, 2, 1000L) -// )); -// -// KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( -// KAFKA_TOPIC, -// ImmutableMap.of(1, 1000L, 2, 1000L) -// )); +// supervisor.runInternal(); // -// KinesisDataSourceMetadata expectedMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( -// KAFKA_TOPIC, -// ImmutableMap.of(0, 1000L) -// )); +// final TreeMap> newCheckpoints = new TreeMap<>(); +// newCheckpoints.put(0, ImmutableMap.of(0, 10L)); +// supervisor.checkpoint( +// null, +// ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, checkpoints.get(0))), +// new KinesisDataSourceMetadata(new KinesisPartitions(stream, newCheckpoints.get(0))) +// ); // -// reset(indexerMetadataStorageCoordinator); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(kinesisDataSourceMetadata); -// expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata( -// EasyMock.capture(captureDataSource), -// EasyMock.capture(captureDataSourceMetadata) -// )).andReturn(true); -// replay(indexerMetadataStorageCoordinator); +// while (supervisor.getNoticesQueueSize() > 0) { +// Thread.sleep(100); +// } // -// supervisor.resetInternal(resetMetadata); // verifyAll(); -// -// Assert.assertEquals(captureDataSource.getValue(), DATASOURCE); -// Assert.assertEquals(captureDataSourceMetadata.getValue(), expectedMetadata); // } // // @Test -// public void testResetNoDataSourceMetadata() throws Exception +// public void testSuspendedNoRunningTasks() throws Exception // { -// supervisor = getSupervisor(1, 1, true, "PT1H", null, false); +// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false, true); +// addSomeEvents(1); +// // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); // expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( +// new KinesisDataSourceMetadata( +// null +// ) +// ).anyTimes(); +// // this asserts that taskQueue.add does not in fact get called because supervisor should be suspended +// expect(taskQueue.add(anyObject())).andAnswer((IAnswer) () -> { +// Assert.fail(); +// return null; +// }).anyTimes(); // taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); // replayAll(); // // supervisor.start(); // supervisor.runInternal(); // verifyAll(); -// -// KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( -// KAFKA_TOPIC, -// ImmutableMap.of(1, 1000L, 2, 1000L) -// )); -// -// reset(indexerMetadataStorageCoordinator); -// // no DataSourceMetadata in metadata store -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null); -// replay(indexerMetadataStorageCoordinator); -// -// supervisor.resetInternal(resetMetadata); -// verifyAll(); // } // // @Test -// public void testResetRunningTasks() throws Exception +// public void testSuspendedRunningTasks() throws Exception // { -// final TaskLocation location1 = new TaskLocation("testHost", 1234); -// final TaskLocation location2 = new TaskLocation("testHost2", 145); -// final DateTime startTime = new DateTime(); +// // graceful shutdown is expected to be called on running tasks since state is suspended +// +// final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); +// final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); +// final DateTime startTime = DateTimes.nowUtc(); // -// supervisor = getSupervisor(2, 1, true, "PT1H", null, false); +// supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false, true); // addSomeEvents(1); // -// Task id1 = createKafkaIndexTask( +// Task id1 = createKinesisIndexTask( // "id1", // DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, // null // ); // -// Task id2 = createKafkaIndexTask( +// Task id2 = createKinesisIndexTask( // "id2", // DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, // null // ); // -// Task id3 = createKafkaIndexTask( +// Task id3 = createKinesisIndexTask( // "id3", // DATASOURCE, -// "sequenceName-0", -// new KinesisPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// 0, +// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), +// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), +// null, // null // ); // // Collection workItems = new ArrayList<>(); -// workItems.add(new TestTaskRunnerWorkItem(id1.getId(), null, location1)); -// workItems.add(new TestTaskRunnerWorkItem(id2.getId(), null, location2)); +// workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); +// workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); // // expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); // expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); @@ -1654,210 +2564,380 @@ // null // ) // ).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.PUBLISHING)); -// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); -// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KinesisIndexTask.Status.READING)); +// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); +// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); +// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); // expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); // expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); -// expect(taskClient.getCurrentOffsets("id1", true)).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); +// expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); +// +// // getCheckpoints will not be called for id1 as it is in publishing state +// TreeMap> checkpoints = new TreeMap<>(); +// 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("id3"), anyBoolean())) +// .andReturn(Futures.immediateFuture(checkpoints)) +// .times(1); // // taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); // -// reset(taskQueue, indexerMetadataStorageCoordinator); -// expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); -// taskQueue.shutdown("id2"); +// expect(taskClient.pauseAsync("id2")) +// .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); +// expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) +// .andReturn(Futures.immediateFuture(true)); // taskQueue.shutdown("id3"); -// replay(taskQueue, indexerMetadataStorageCoordinator); +// expectLastCall().times(2); // -// supervisor.resetInternal(null); +// replayAll(); +// supervisor.start(); +// supervisor.runInternal(); // verifyAll(); // } // -// private void addSomeEvents(int numEventsPerPartition) throws Exception -// { -// try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { -// for (int i = 0; i < NUM_PARTITIONS; i++) { -// for (int j = 0; j < numEventsPerPartition; j++) { -// kafkaProducer.send( -// new ProducerRecord( -// KAFKA_TOPIC, -// i, -// null, -// String.format("event-%d", j).getBytes() -// ) -// ).get(); -// } -// } -// } -// } -// -// private KinesisSupervisor getSupervisor( -// int replicas, -// int taskCount, -// boolean useEarliestOffset, -// String duration, -// Period lateMessageRejectionPeriod, -// boolean skipOffsetGaps -// ) -// { -// KinesisSupervisorIOConfig kafkaSupervisorIOConfig = new KinesisSupervisorIOConfig( -// KAFKA_TOPIC, -// replicas, -// taskCount, -// new Period(duration), -// ImmutableMap.of("myCustomKey", "myCustomValue", "bootstrap.servers", kafkaHost), -// new Period("P1D"), -// new Period("PT30S"), -// useEarliestOffset, -// new Period("PT30M"), -// lateMessageRejectionPeriod, -// skipOffsetGaps -// ); -// -// KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory(null, null) -// { -// @Override -// public KinesisIndexTaskClient build( -// TaskInfoProvider taskInfoProvider, -// String dataSource, -// int numThreads, -// Duration httpTimeout, -// long numRetries -// ) -// { -// Assert.assertEquals(TEST_CHAT_THREADS, numThreads); -// Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout); -// Assert.assertEquals(TEST_CHAT_RETRIES, numRetries); -// return taskClient; -// } -// }; -// -// return new TestableKinesisSupervisor( -// taskStorage, -// taskMaster, -// indexerMetadataStorageCoordinator, -// taskClientFactory, -// objectMapper, -// new KinesisSupervisorSpec( -// dataSchema, -// tuningConfig, -// kafkaSupervisorIOConfig, -// null, -// taskStorage, -// taskMaster, -// indexerMetadataStorageCoordinator, -// taskClientFactory, -// objectMapper, -// new NoopServiceEmitter(), -// new DruidMonitorSchedulerConfig() -// ) -// ); -// } -// -// private DataSchema getDataSchema(String dataSource) -// { -// List dimensions = new ArrayList<>(); -// dimensions.add(StringDimensionSchema.create("dim1")); -// dimensions.add(StringDimensionSchema.create("dim2")); -// -// return new DataSchema( -// dataSource, -// objectMapper.convertValue( -// new StringInputRowParser( -// new JSONParseSpec( -// new TimestampSpec("timestamp", "iso", null), -// new DimensionsSpec( -// dimensions, -// null, -// null -// ), -// new JSONPathSpec(true, ImmutableList.of()), -// ImmutableMap.of() -// ), -// Charsets.UTF_8.name() -// ), -// Map.class -// ), -// new AggregatorFactory[]{new CountAggregatorFactory("rows")}, -// new UniformGranularitySpec( -// Granularities.HOUR, -// Granularities.NONE, -// ImmutableList.of() -// ), -// objectMapper -// ); -// } -// -// private KinesisIndexTask createKafkaIndexTask( -// String id, -// String dataSource, -// String sequenceName, -// KinesisPartitions startPartitions, -// KinesisPartitions endPartitions, -// DateTime minimumMessageTime -// ) -// { -// return new KinesisIndexTask( -// id, -// null, -// getDataSchema(dataSource), -// tuningConfig, -// new KinesisIOConfig( -// sequenceName, -// startPartitions, -// endPartitions, -// ImmutableMap.of(), -// true, -// false, -// minimumMessageTime, -// false -// ), -// ImmutableMap.of(), -// null -// ); -// } -// -// private static class TestTaskRunnerWorkItem extends TaskRunnerWorkItem +// @Test +// public void testResetSuspended() throws Exception // { +// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); +// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); +// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); +// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); +// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); +// replayAll(); // -// private TaskLocation location; +// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false, true); +// supervisor.start(); +// supervisor.runInternal(); +// verifyAll(); // -// public TestTaskRunnerWorkItem(String taskId, ListenableFuture result, TaskLocation location) -// { -// super(taskId, result); -// this.location = location; -// } +// reset(indexerMetadataStorageCoordinator); +// expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); +// replay(indexerMetadataStorageCoordinator); // -// @Override -// public TaskLocation getLocation() -// { -// return location; -// } +// supervisor.resetInternal(null); +// verifyAll(); // } // -// private static class TestableKinesisSupervisor extends KinesisSupervisor -// { -// public TestableKinesisSupervisor( -// TaskStorage taskStorage, -// TaskMaster taskMaster, -// IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, -// KinesisIndexTaskClientFactory taskClientFactory, -// ObjectMapper mapper, -// KinesisSupervisorSpec spec -// ) -// { -// super(taskStorage, taskMaster, indexerMetadataStorageCoordinator, taskClientFactory, mapper, spec); -// } -// -// @Override -// protected String generateSequenceName(int groupId) -// { -// return String.format("sequenceName-%d", groupId); -// } -// } -//} + + private static List insertData( + AmazonKinesis kinesis, + PutRecordsRequest req + ) + { + PutRecordsResult res = kinesis.putRecords(req); + Assert.assertTrue(isResponseOk(res.getSdkHttpMetadata())); + Assert.assertEquals((int) res.getFailedRecordCount(), 0); + return res.getRecords(); + } + + private AmazonKinesis getKinesisClientInstance() throws InterruptedException + { + AmazonKinesis kinesis = cloud.localstack.TestUtils.getClientKinesis(); + SdkHttpMetadata createRes = kinesis.createStream(stream, 2).getSdkHttpMetadata(); + // sleep required because of kinesalite + Thread.sleep(500); + Assert.assertTrue(isResponseOk(createRes)); + return kinesis; + } + + private static boolean isResponseOk(SdkHttpMetadata sdkHttpMetadata) + { + return sdkHttpMetadata.getHttpStatusCode() == 200; + } + + private KinesisSupervisor getSupervisor( + int replicas, + int taskCount, + boolean useEarliestOffset, + String duration, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod + ) + { + return getSupervisor( + replicas, + taskCount, + useEarliestOffset, + duration, + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod, + false, + null, + null + ); + } + + private KinesisSupervisor getSupervisor( + int replicas, + int taskCount, + boolean useEarliestOffset, + String duration, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod, + boolean suspended, + Integer recordsPerFetch, + Integer fetchDelayMillis + + ) + { + KinesisSupervisorIOConfig KinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( + stream, + Localstack.getEndpointKinesis(), + null, + replicas, + taskCount, + new Period(duration), + new Period("P1D"), + new Period("PT30S"), + useEarliestOffset, + new Period("PT30M"), + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod, + recordsPerFetch, + fetchDelayMillis, + cloud.localstack.TestUtils.TEST_ACCESS_KEY, + cloud.localstack.TestUtils.TEST_SECRET_KEY, + null, + null, + false + ); + + KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( + null, + null + ) + { + @Override + public KinesisIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ) + { + Assert.assertEquals(TEST_CHAT_THREADS, numThreads); + Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout); + Assert.assertEquals(TEST_CHAT_RETRIES, numRetries); + return taskClient; + } + }; + + return new TestableKinesisSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new KinesisSupervisorSpec( + dataSchema, + tuningConfig, + KinesisSupervisorIOConfig, + null, + suspended, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory + ), + rowIngestionMetersFactory + ); + } + + private static DataSchema getDataSchema(String dataSource) + { + List dimensions = new ArrayList<>(); + dimensions.add(StringDimensionSchema.create("dim1")); + dimensions.add(StringDimensionSchema.create("dim2")); + + return new DataSchema( + dataSource, + objectMapper.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + dimensions, + null, + null + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of() + ), + StandardCharsets.UTF_8.name() + ), + Map.class + ), + new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of() + ), + null, + objectMapper + ); + } + + private static PutRecordsRequestEntry generateRequestEntry(String partition, byte[] data) + { + return new PutRecordsRequestEntry().withPartitionKey(partition) + .withData(ByteBuffer.wrap(data)); + } + + private static PutRecordsRequest generateRecordsRequests(String stream, int first, int last) + { + return new PutRecordsRequest() + .withStreamName(stream) + .withRecords(records.subList(first, last)); + } + + private static PutRecordsRequest generateRecordsRequests(String stream) + { + return new PutRecordsRequest() + .withStreamName(stream) + .withRecords(records); + } + + private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) + { + try { + return new ObjectMapper().writeValueAsBytes( + ImmutableMap.builder() + .put("timestamp", timestamp) + .put("dim1", dim1) + .put("dim2", dim2) + .put("dimLong", dimLong) + .put("dimFloat", dimFloat) + .put("met1", met1) + .build() + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + private static String getSequenceNumber(List entries, String shardId, int offset) + { + List sortedEntries = entries.stream() + .filter(e -> e.getShardId().equals(shardId)) + .sorted(Comparator.comparing(e -> KinesisSequenceNumber.of(e.getSequenceNumber()))) + .collect(Collectors.toList()); + return sortedEntries.get(offset).getSequenceNumber(); + } + + private KinesisIndexTask createKinesisIndexTask( + String id, + String dataSource, + int taskGroupId, + KinesisPartitions startPartitions, + KinesisPartitions endPartitions, + DateTime minimumMessageTime, + DateTime maximumMessageTime + ) + { + return new KinesisIndexTask( + id, + null, + getDataSchema(dataSource), + tuningConfig, + new KinesisIOConfig( + "sequenceName-" + taskGroupId, + startPartitions, + endPartitions, + true, + null, + minimumMessageTime, + maximumMessageTime, + Localstack.getEndpointKinesis(), + null, + null, + cloud.localstack.TestUtils.TEST_ACCESS_KEY, + cloud.localstack.TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ), + Collections.emptyMap(), + null, + null, + rowIngestionMetersFactory + ); + } + + private static class TestTaskRunnerWorkItem extends TaskRunnerWorkItem + { + private final String taskType; + private final TaskLocation location; + private final String dataSource; + + public TestTaskRunnerWorkItem(Task task, ListenableFuture result, TaskLocation location) + { + super(task.getId(), result); + this.taskType = task.getType(); + this.location = location; + this.dataSource = task.getDataSource(); + } + + @Override + public TaskLocation getLocation() + { + return location; + } + + @Override + public String getTaskType() + { + return taskType; + } + + @Override + public String getDataSource() + { + return dataSource; + } + + } + + private static class TestableKinesisSupervisor extends KinesisSupervisor + { + public TestableKinesisSupervisor( + TaskStorage taskStorage, + TaskMaster taskMaster, + IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + KinesisIndexTaskClientFactory taskClientFactory, + ObjectMapper mapper, + KinesisSupervisorSpec spec, + RowIngestionMetersFactory rowIngestionMetersFactory + ) + { + super( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + mapper, + spec, + rowIngestionMetersFactory + ); + } + + @Override + protected String generateSequenceName( + Map startPartitions, + Optional minimumMessageTime, + Optional maximumMessageTime + ) + { + final int groupId = getTaskGroupIdForPartition(startPartitions.keySet().iterator().next()); + return StringUtils.format("sequenceName-%d", groupId); + } + } +} diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java index 36a02f4135ed..df1d79e49202 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java @@ -58,7 +58,7 @@ public void testSerdeWithDefaults() throws Exception ); Assert.assertNotNull(config.getBasePersistDirectory()); - Assert.assertEquals(75000, config.getMaxRowsInMemory()); + Assert.assertEquals(1000000, config.getMaxRowsInMemory()); Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment()); Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); Assert.assertEquals(0, config.getMaxPendingPersists()); @@ -107,7 +107,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(100, config.getMaxRowsInMemory()); Assert.assertEquals(100, config.getMaxRowsPerSegment()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); - Assert.assertEquals(100, config.getMaxPendingPersists()); + Assert.assertEquals(0, config.getMaxPendingPersists()); Assert.assertEquals(false, config.getBuildV9Directly()); Assert.assertEquals(true, config.isReportParseExceptions()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java index b91b1e76bc18..5a01aece4560 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java @@ -48,11 +48,11 @@ public interface RecordSupplier extends Closeable T2 getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException; + // TODO: maybe remove this, not needed in kinesis, only in kafka T2 position(StreamPartition partition); Set getPartitionIds(String streamName); - @Override void close(); } From 0ef3dd75bc09aacd89d8c6bc365584d37e720d2f Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 3 Oct 2018 14:37:55 -0700 Subject: [PATCH 15/87] more bug fixes for kinesis-indexing-service --- .../apache/druid/indexing/kinesis/KinesisSequenceNumber.java | 5 +++-- .../supervisor/SeekableStreamSupervisorReportPayload.java | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java index 0071edf954d9..02b91f7cedab 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java @@ -22,6 +22,7 @@ import org.apache.druid.indexing.seekablestream.common.SequenceNumber; +import javax.validation.constraints.NotNull; import java.math.BigInteger; public class KinesisSequenceNumber extends SequenceNumber @@ -29,10 +30,10 @@ public class KinesisSequenceNumber extends SequenceNumber private final BigInteger intSequence; - private KinesisSequenceNumber(String sequenceNumber, boolean useExclusive, boolean isExclusive) + private KinesisSequenceNumber(@NotNull String sequenceNumber, boolean useExclusive, boolean isExclusive) { super(sequenceNumber, useExclusive, isExclusive); - this.intSequence = new BigInteger(sequenceNumber); + this.intSequence = sequenceNumber.equals("") ? new BigInteger("-1") : new BigInteger(sequenceNumber); } public BigInteger getBigInteger() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java index 991509829707..98083ae53663 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -88,7 +88,7 @@ public String getDataSource() return dataSource; } - protected String getId() + public String getId() { return id; } From be9ebb64293b65b57fbcadfe2b17c51bcc07b8cb Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 3 Oct 2018 14:39:16 -0700 Subject: [PATCH 16/87] finsihed refactoring kinesis unit tests --- .../supervisor/KinesisSupervisorTest.java | 4271 ++++++++++------- .../KinesisSupervisorTuningConfigTest.java | 2 +- 2 files changed, 2433 insertions(+), 1840 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index d6abd3649d17..9b6d7c09e0dc 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -55,6 +55,7 @@ import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; import org.apache.druid.indexing.kinesis.KinesisPartitions; import org.apache.druid.indexing.kinesis.KinesisSequenceNumber; +import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueue; @@ -62,7 +63,9 @@ import org.apache.druid.indexing.overlord.TaskRunnerListener; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -83,6 +86,7 @@ import org.easymock.CaptureType; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; +import org.easymock.IAnswer; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; @@ -97,6 +101,7 @@ import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -109,7 +114,11 @@ import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.anyString; import static org.easymock.EasyMock.capture; +import static org.easymock.EasyMock.eq; import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.expectLastCall; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.reset; // TODO: improve helper methods like insertData(...) @RunWith(LocalstackDockerTestRunner.class) @@ -527,55 +536,7 @@ public void testEarlyMessageRejectionPeriod() throws Exception ); } - // @Test -// /** -// * Test generating the starting offsets from the partition high water marks in Kinesis. -// */ -// public void testLatestOffset() throws Exception -// { -// final ExecutorService exec = Execs.singleThreaded("workerexec"); -// supervisor = getSupervisor(1, 1, false, "PT1H", null, null); -// AmazonKinesis kinesis = getKinesisClientInstance(); -// -// -// Capture captured = Capture.newInstance(); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true); -// replayAll(); -// -// supervisor.start(); -// try { -// supervisor.runInternal(); -// } -// catch (TimeoutException e) { -// supervisor. -// List res = insertData(kinesis, generateRecordsRequests(stream)); -// supervisor.runInternal(); -// } -// -// -// verifyAll(); -// -// KinesisIndexTask task = captured.getValue(); -// KinesisIOConfig taskConfig = task.getIOConfig(); -// Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); -// Assert.assertEquals( -// getSequenceNumber(res, shardId1, 12), -// taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) -// ); -// Assert.assertEquals( -// getSequenceNumber(res, shardId0, 1), -// taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) -// ); -// } -// + @Test /** * Test generating the starting offsets from the partition data stored in druid_dataSource which contains the @@ -739,64 +700,2282 @@ public void testKillIncompatibleTasks() throws Exception Task id5 = new RealtimeIndexTask( "id5", null, - new FireDepartment( - dataSchema, - new RealtimeIOConfig(null, null, null), - null - ), + new FireDepartment( + dataSchema, + new RealtimeIOConfig(null, null, null), + null + ), + null + ); + + List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); + + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); + expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(false)); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskQueue.shutdown("id3"); + + expect(taskQueue.add(anyObject(Task.class))).andReturn(true); + + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0), + shardId1, + getSequenceNumber(res, shardId1, 0) + )); + + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(2); + + replayAll(); + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + + // TODO: delete redundant tasks + @Test + public void testKillBadPartitionAssignment() throws Exception + { + supervisor = getSupervisor(1, 2, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Task id1 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 12) + )), + null, + null + ); + Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 1, + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 1) + )), + null, + null + ); + Task id3 = createKinesisIndexTask( + "id3", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0), + shardId1, + getSequenceNumber(res, shardId1, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 1), + shardId1, + getSequenceNumber(res, shardId1, 12) + )), + null, + null + ); + Task id4 = createKinesisIndexTask( + "id4", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 1) + )), + null, + null + ); + Task id5 = createKinesisIndexTask( + "id5", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 1) + )), + null, + null + ); + + List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); + + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); + expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).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(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); + expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); + expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); + expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of(shardId1, getSequenceNumber(res, shardId1, 0))); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) + .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)); + taskQueue.shutdown("id4"); + taskQueue.shutdown("id5"); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + + @Test + public void testRequeueTaskWhenFailed() throws Exception + { + supervisor = getSupervisor(2, 2, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true).times(4); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put( + 0, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0) + ) + ); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0) + )); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .anyTimes(); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .anyTimes(); + + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + List tasks = captured.getValues(); + + // test that running the main loop again checks the status of the tasks that were created and does nothing if they + // are all still running + reset(taskStorage); + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + for (Task task : tasks) { + expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + } + replay(taskStorage); + + supervisor.runInternal(); + verifyAll(); + + // test that a task failing causes a new task to be re-queued with the same parameters + Capture aNewTaskCapture = Capture.newInstance(); + List imStillAlive = tasks.subList(0, 3); + KinesisIndexTask iHaveFailed = (KinesisIndexTask) tasks.get(3); + reset(taskStorage); + reset(taskQueue); + expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes(); + for (Task task : imStillAlive) { + expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + } + expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); + expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); + expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); + replay(taskStorage); + replay(taskQueue); + + supervisor.runInternal(); + verifyAll(); + + Assert.assertNotEquals(iHaveFailed.getId(), aNewTaskCapture.getValue().getId()); + Assert.assertEquals( + iHaveFailed.getIOConfig().getBaseSequenceName(), + ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getBaseSequenceName() + ); + } + + @Test + public void testRequeueAdoptedTaskWhenFailed() throws Exception + { + supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + DateTime now = DateTimes.nowUtc(); + DateTime maxi = now.plusMinutes(60); + Task id1 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0), + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + now, + maxi + ); + + List existingTasks = ImmutableList.of(id1); + + Capture captured = Capture.newInstance(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); + expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0), + shardId0, + getSequenceNumber(res, shardId0, 0) + )); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(2); + + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + // check that replica tasks are created with the same minimumMessageTime as tasks inherited from another supervisor + Assert.assertEquals(now, ((KinesisIndexTask) captured.getValue()).getIOConfig().getMinimumMessageTime().get()); + + // test that a task failing causes a new task to be re-queued with the same parameters + String runningTaskId = captured.getValue().getId(); + Capture aNewTaskCapture = Capture.newInstance(); + KinesisIndexTask iHaveFailed = (KinesisIndexTask) existingTasks.get(0); + reset(taskStorage); + reset(taskQueue); + reset(taskClient); + + // for the newly created replica task + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .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.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); + expect(taskStorage.getStatus(runningTaskId)).andReturn(Optional.of(TaskStatus.running(runningTaskId))).anyTimes(); + expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); + expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); + expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes(); + expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); + replay(taskStorage); + replay(taskQueue); + replay(taskClient); + + supervisor.runInternal(); + verifyAll(); + + Assert.assertNotEquals(iHaveFailed.getId(), aNewTaskCapture.getValue().getId()); + Assert.assertEquals( + iHaveFailed.getIOConfig().getBaseSequenceName(), + ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getBaseSequenceName() + ); + + // check that failed tasks are recreated with the same minimumMessageTime as the task it replaced, even if that + // task came from another supervisor + Assert.assertEquals( + now, + ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getMinimumMessageTime().get() + ); + Assert.assertEquals( + maxi, + ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getMaximumMessageTime().get() + ); + } + + @Test + public void testQueueNextTasksOnSuccess() throws Exception + { + supervisor = getSupervisor(2, 2, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + List tasks = captured.getValues(); + + reset(taskStorage); + reset(taskClient); + + expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0), + shardId0, + getSequenceNumber(res, shardId0, 0) + )); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0) + )); + // there would be 4 tasks, 2 for each task group + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); + + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + for (Task task : tasks) { + expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + } + replay(taskStorage); + replay(taskClient); + + supervisor.runInternal(); + verifyAll(); + + // test that a task succeeding causes a new task to be re-queued with the next offset range and causes any replica + // tasks to be shutdown + Capture newTasksCapture = Capture.newInstance(CaptureType.ALL); + Capture shutdownTaskIdCapture = Capture.newInstance(); + List imStillRunning = tasks.subList(1, 4); + KinesisIndexTask iAmSuccess = (KinesisIndexTask) tasks.get(0); + reset(taskStorage); + reset(taskQueue); + reset(taskClient); + expect(taskStorage.getActiveTasks()).andReturn(imStillRunning).anyTimes(); + for (Task task : imStillRunning) { + expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + } + expect(taskStorage.getStatus(iAmSuccess.getId())).andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId()))); + expect(taskStorage.getTask(iAmSuccess.getId())).andReturn(Optional.of((Task) iAmSuccess)).anyTimes(); + expect(taskQueue.add(capture(newTasksCapture))).andReturn(true).times(2); + expect(taskClient.stopAsync(capture(shutdownTaskIdCapture), eq(false))).andReturn(Futures.immediateFuture(true)); + replay(taskStorage); + replay(taskQueue); + replay(taskClient); + + supervisor.runInternal(); + verifyAll(); + + // make sure we killed the right task (sequenceName for replicas are the same) + Assert.assertTrue(shutdownTaskIdCapture.getValue().contains(iAmSuccess.getIOConfig().getBaseSequenceName())); + } + + @Test + public void testBeginPublishAndQueueNextTasks() throws Exception + { + final TaskLocation location = new TaskLocation("testHost", 1234, -1); + + supervisor = getSupervisor(2, 2, true, "PT1M", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + List tasks = captured.getValues(); + Collection workItems = new ArrayList<>(); + for (Task task : tasks) { + workItems.add(new TestTaskRunnerWorkItem(task, null, location)); + } + + reset(taskStorage, taskRunner, taskClient, taskQueue); + captured = Capture.newInstance(CaptureType.ALL); + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + for (Task task : tasks) { + expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + } + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskClient.getStatusAsync(anyString())) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .times(2); + expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 1), + shardId0, + getSequenceNumber(res, shardId0, 0) + ))) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); + expect( + taskClient.setEndOffsetsAsync( + EasyMock.contains("sequenceName-0"), + EasyMock.eq(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3), + shardId0, + getSequenceNumber(res, shardId0, 1) + )), + EasyMock.eq(true) + ) + ).andReturn(Futures.immediateFuture(true)).times(2); + expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0) + )); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of( + shardId0, + getSequenceNumber(res, shardId0, 0) + )); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); + + replay(taskStorage, taskRunner, taskClient, taskQueue); + + supervisor.runInternal(); + verifyAll(); + + for (Task task : captured.getValues()) { + KinesisIndexTask KinesisIndexTask = (KinesisIndexTask) task; + Assert.assertEquals(dataSchema, KinesisIndexTask.getDataSchema()); + Assert.assertEquals(tuningConfig.copyOf(), KinesisIndexTask.getTuningConfig()); + + KinesisIOConfig taskConfig = KinesisIndexTask.getIOConfig(); + Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); + Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); + + Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); + Assert.assertEquals( + getSequenceNumber(res, shardId1, 3), + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + Assert.assertEquals( + getSequenceNumber(res, shardId0, 1), + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + } + } + + @Test + public void testDiscoverExistingPublishingTask() throws Exception + { + final TaskLocation location = new TaskLocation("testHost", 1234, -1); + + supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Task task = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0), + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + Collection workItems = new ArrayList<>(); + workItems.add(new TestTaskRunnerWorkItem(task, null, location)); + + Capture captured = Capture.newInstance(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); + expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + expect(taskClient.getCurrentOffsetsAsync("id1", false)) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + )); + expect(taskQueue.add(capture(captured))).andReturn(true); + + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0), + shardId0, + getSequenceNumber(res, shardId0, 0) + )); + expect(taskClient.getCheckpoints(anyString(), anyBoolean())).andReturn(checkpoints).anyTimes(); + + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + supervisor.updateCurrentAndLatestOffsets().run(); + SupervisorReport report = supervisor.getStatus(); + verifyAll(); + + Assert.assertEquals(DATASOURCE, report.getId()); + + KinesisSupervisorReportPayload payload = report.getPayload(); + + Assert.assertEquals(DATASOURCE, payload.getDataSource()); + Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); + Assert.assertEquals(2, (int) payload.getPartitions()); + Assert.assertEquals(1, (int) payload.getReplicas()); + Assert.assertEquals(stream, payload.getId()); + Assert.assertEquals(0, payload.getActiveTasks().size()); + Assert.assertEquals(1, payload.getPublishingTasks().size()); + + TaskReportData publishingReport = payload.getPublishingTasks().get(0); + + Assert.assertEquals("id1", publishingReport.getId()); + Assert.assertEquals(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0), + shardId0, + getSequenceNumber(res, shardId0, 0) + ), publishingReport.getStartingOffsets()); + Assert.assertEquals(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + ), publishingReport.getCurrentOffsets()); + + KinesisIndexTask capturedTask = captured.getValue(); + Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); + Assert.assertEquals(tuningConfig.copyOf(), capturedTask.getTuningConfig()); + + KinesisIOConfig capturedTaskConfig = capturedTask.getIOConfig(); + Assert.assertEquals(cloud.localstack.TestUtils.TEST_ACCESS_KEY, capturedTaskConfig.getAwsAccessKeyId()); + Assert.assertEquals(cloud.localstack.TestUtils.TEST_SECRET_KEY, capturedTaskConfig.getAwsSecretAccessKey()); + Assert.assertEquals(Localstack.getEndpointKinesis(), capturedTaskConfig.getEndpoint()); + Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); + Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); + + // check that the new task was created with starting offsets matching where the publishing task finished + Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); + Assert.assertEquals( + getSequenceNumber(res, shardId1, 2), + capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + Assert.assertEquals( + getSequenceNumber(res, shardId0, 1), + capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + + Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream()); + Assert.assertEquals( + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + Assert.assertEquals( + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + } + + @Test + public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() throws Exception + { + final TaskLocation location = new TaskLocation("testHost", 1234, -1); + + supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Task task = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0), + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + Collection workItems = new ArrayList<>(); + workItems.add(new TestTaskRunnerWorkItem(task, null, location)); + + Capture captured = Capture.newInstance(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); + expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + expect(taskClient.getCurrentOffsetsAsync("id1", false)) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + )); + expect(taskQueue.add(capture(captured))).andReturn(true); + + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + supervisor.updateCurrentAndLatestOffsets().run(); + SupervisorReport report = supervisor.getStatus(); + verifyAll(); + + Assert.assertEquals(DATASOURCE, report.getId()); + + KinesisSupervisorReportPayload payload = report.getPayload(); + + Assert.assertEquals(DATASOURCE, payload.getDataSource()); + Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); + Assert.assertEquals(2, (int) payload.getPartitions()); + Assert.assertEquals(1, (int) payload.getReplicas()); + Assert.assertEquals(stream, payload.getId()); + Assert.assertEquals(0, payload.getActiveTasks().size()); + Assert.assertEquals(1, payload.getPublishingTasks().size()); + + TaskReportData publishingReport = payload.getPublishingTasks().get(0); + + Assert.assertEquals("id1", publishingReport.getId()); + Assert.assertEquals(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0), + shardId0, + getSequenceNumber(res, shardId0, 0) + ), publishingReport.getStartingOffsets()); + Assert.assertEquals(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + ), publishingReport.getCurrentOffsets()); + + KinesisIndexTask capturedTask = captured.getValue(); + Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); + Assert.assertEquals(tuningConfig.copyOf(), capturedTask.getTuningConfig()); + + KinesisIOConfig capturedTaskConfig = capturedTask.getIOConfig(); + Assert.assertEquals(cloud.localstack.TestUtils.TEST_ACCESS_KEY, capturedTaskConfig.getAwsAccessKeyId()); + Assert.assertEquals(cloud.localstack.TestUtils.TEST_SECRET_KEY, capturedTaskConfig.getAwsSecretAccessKey()); + Assert.assertEquals(Localstack.getEndpointKinesis(), capturedTaskConfig.getEndpoint()); + Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); + Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); + + // check that the new task was created with starting offsets matching where the publishing task finished + Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); + Assert.assertEquals( + getSequenceNumber(res, shardId1, 2), + capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + Assert.assertEquals( + getSequenceNumber(res, shardId0, 1), + capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + + Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream()); + Assert.assertEquals( + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + Assert.assertEquals( + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + } + + @Test + public void testDiscoverExistingPublishingAndReadingTask() throws Exception + { + final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); + final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); + final DateTime startTime = DateTimes.nowUtc(); + + supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Task id1 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new KinesisPartitions( + "stream", + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0), + shardId0, + getSequenceNumber(res, shardId0, 0) + ) + ), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + )), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + Collection workItems = new ArrayList<>(); + workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); + workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); + + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2)).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.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getCurrentOffsetsAsync("id1", false)) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + )); + expect(taskClient.getCurrentOffsetsAsync("id2", false)) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 12), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); + + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + + // since id1 is publishing, so getCheckpoints wouldn't be called for it + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + )); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + supervisor.updateCurrentAndLatestOffsets().run(); + SupervisorReport report = supervisor.getStatus(); + verifyAll(); + + Assert.assertEquals(DATASOURCE, report.getId()); + + KinesisSupervisorReportPayload payload = report.getPayload(); + + Assert.assertEquals(DATASOURCE, payload.getDataSource()); + Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); + Assert.assertEquals(2, (int) payload.getPartitions()); + Assert.assertEquals(1, (int) payload.getReplicas()); + Assert.assertEquals(stream, payload.getId()); + Assert.assertEquals(1, payload.getActiveTasks().size()); + Assert.assertEquals(1, payload.getPublishingTasks().size()); + + TaskReportData activeReport = payload.getActiveTasks().get(0); + TaskReportData publishingReport = payload.getPublishingTasks().get(0); + + Assert.assertEquals("id2", activeReport.getId()); + Assert.assertEquals(startTime, activeReport.getStartTime()); + Assert.assertEquals(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + ), activeReport.getStartingOffsets()); + Assert.assertEquals(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 12), + shardId0, + getSequenceNumber(res, shardId0, 1) + ), activeReport.getCurrentOffsets()); + + Assert.assertEquals("id1", publishingReport.getId()); + Assert.assertEquals(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0), + shardId0, + getSequenceNumber(res, shardId0, 0) + ), publishingReport.getStartingOffsets()); + Assert.assertEquals(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + ), publishingReport.getCurrentOffsets()); + } + + @Test + public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception + { + supervisor = getSupervisor(2, 2, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + List tasks = captured.getValues(); + + reset(taskStorage, taskClient, taskQueue); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 0), + + shardId0, + + getSequenceNumber(res, shardId0, 0) + )); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); + + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + for (Task task : tasks) { + expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + expect(taskClient.getStatusAsync(task.getId())) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)); + expect(taskClient.getStartTimeAsync(task.getId())) + .andReturn(Futures.immediateFailedFuture(new RuntimeException())); + taskQueue.shutdown(task.getId()); + } + replay(taskStorage, taskClient, taskQueue); + + supervisor.runInternal(); + verifyAll(); + } + + @Test + public void testKillUnresponsiveTasksWhilePausing() throws Exception + { + final TaskLocation location = new TaskLocation("testHost", 1234, -1); + + supervisor = getSupervisor(2, 2, true, "PT1M", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + List tasks = captured.getValues(); + Collection workItems = new ArrayList<>(); + for (Task task : tasks) { + workItems.add(new TestTaskRunnerWorkItem(task, null, location)); + } + + reset(taskStorage, taskRunner, taskClient, taskQueue); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0) + )); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); + + captured = Capture.newInstance(CaptureType.ALL); + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + for (Task task : tasks) { + expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + } + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskClient.getStatusAsync(anyString())) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .times(2); + expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); + taskQueue.shutdown(EasyMock.contains("sequenceName-0")); + expectLastCall().times(2); + expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + + replay(taskStorage, taskRunner, taskClient, taskQueue); + + supervisor.runInternal(); + verifyAll(); + + for (Task task : captured.getValues()) { + KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); + Assert.assertEquals( + getSequenceNumber(res, shardId1, 0), + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + Assert.assertNull( + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + ); + } + } + + @Test + public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception + { + final TaskLocation location = new TaskLocation("testHost", 1234, -1); + + supervisor = getSupervisor(2, 2, true, "PT1M", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Capture captured = Capture.newInstance(CaptureType.ALL); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + new KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskQueue.add(capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + List tasks = captured.getValues(); + Collection workItems = new ArrayList<>(); + for (Task task : tasks) { + workItems.add(new TestTaskRunnerWorkItem(task, null, location)); + } + + reset(taskStorage, taskRunner, taskClient, taskQueue); + + TreeMap> checkpoints1 = new TreeMap<>(); + checkpoints1.put(0, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0) + )); + TreeMap> checkpoints2 = new TreeMap<>(); + checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); + + captured = Capture.newInstance(CaptureType.ALL); + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + for (Task task : tasks) { + expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + } + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskClient.getStatusAsync(anyString())) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .times(2); + expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 1) + ))) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3) + ))); + expect( + taskClient.setEndOffsetsAsync( + EasyMock.contains("sequenceName-0"), + EasyMock.eq(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3) + )), + EasyMock.eq(true) + ) + ).andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); + taskQueue.shutdown(EasyMock.contains("sequenceName-0")); + expectLastCall().times(2); + expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + + replay(taskStorage, taskRunner, taskClient, taskQueue); + + supervisor.runInternal(); + verifyAll(); + + for (Task task : captured.getValues()) { + KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); + Assert.assertEquals( + getSequenceNumber(res, shardId1, 0), + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + ); + } + } + + @Test(expected = IllegalStateException.class) + public void testStopNotStarted() + { + supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor.stop(false); + } + + @Test + public void testStop() + { + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + taskClient.close(); + taskRunner.unregisterListener(StringUtils.format("KinesisSupervisor-%s", DATASOURCE)); + replayAll(); + + supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor.start(); + supervisor.stop(false); + + verifyAll(); + } + + @Test + public void testStopGracefully() throws Exception + { + final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); + final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); + final DateTime startTime = DateTimes.nowUtc(); + + supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Task id1 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0), + shardId0, + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3), + shardId0, + getSequenceNumber(res, shardId0, 1) + )), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + Task id3 = createKinesisIndexTask( + "id3", + DATASOURCE, + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3), + shardId0, + getSequenceNumber(res, shardId0, 1) + )), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + Collection workItems = new ArrayList<>(); + workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); + workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); + + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).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 KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3), + shardId0, + getSequenceNumber(res, shardId0, 1) + )); + + // getCheckpoints will not be called for id1 as it is in publishing state + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3), + shardId0, + getSequenceNumber(res, shardId0, 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(); + verifyAll(); + + reset(taskRunner, taskClient, taskQueue); + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskClient.pauseAsync("id2")) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 12), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); + expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 12), + shardId0, + getSequenceNumber(res, shardId0, 1) + ), true)) + .andReturn(Futures.immediateFuture(true)); + taskQueue.shutdown("id3"); + expectLastCall().times(2); + + replay(taskRunner, taskClient, taskQueue); + + supervisor.gracefulShutdownInternal(); + verifyAll(); + } + + @Test + public void testResetNoTasks() throws Exception + { + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + reset(indexerMetadataStorageCoordinator); + expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + replay(indexerMetadataStorageCoordinator); + + supervisor.resetInternal(null); + verifyAll(); + + } + + @Test + public void testResetDataSourceMetadata() throws Exception + { + supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + Capture captureDataSource = EasyMock.newCapture(); + Capture captureDataSourceMetadata = EasyMock.newCapture(); + + KinesisDataSourceMetadata KinesisDataSourceMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( + stream, + ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + ) + )); + + KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( + stream, + ImmutableMap.of( + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + ) + )); + + KinesisDataSourceMetadata expectedMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( + stream, + ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + ) + )); + + reset(indexerMetadataStorageCoordinator); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(KinesisDataSourceMetadata); + expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata( + EasyMock.capture(captureDataSource), + EasyMock.capture(captureDataSourceMetadata) + )).andReturn(true); + replay(indexerMetadataStorageCoordinator); + + try { + supervisor.resetInternal(resetMetadata); + } + catch (NullPointerException npe) { + // Expected as there will be an attempt to reset partitionGroups offsets to NOT_SET + // however there would be no entries in the map as we have not put nay data in kafka + Assert.assertTrue(npe.getCause() == null); + } + verifyAll(); + + Assert.assertEquals(captureDataSource.getValue(), DATASOURCE); + Assert.assertEquals(captureDataSourceMetadata.getValue(), expectedMetadata); + } + + @Test + public void testResetNoDataSourceMetadata() throws Exception + { + supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( + stream, + ImmutableMap.of( + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + ) + )); + + reset(indexerMetadataStorageCoordinator); + // no DataSourceMetadata in metadata store + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null); + replay(indexerMetadataStorageCoordinator); + + supervisor.resetInternal(resetMetadata); + verifyAll(); + } + + @Test + public void testResetRunningTasks() throws Exception + { + final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); + final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); + final DateTime startTime = DateTimes.nowUtc(); + + supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + Task id1 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 0), + + shardId0, + + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 3), + + shardId0, + + getSequenceNumber(res, shardId0, 1) + )), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + Task id3 = createKinesisIndexTask( + "id3", + DATASOURCE, + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 3), + + shardId0, + + getSequenceNumber(res, shardId0, 1) + )), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + Collection workItems = new ArrayList<>(); + workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); + workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); + + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).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 KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 3), + + shardId0, + + getSequenceNumber(res, shardId0, 1) + )); + + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 3), + + shardId0, + + getSequenceNumber(res, shardId0, 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(); + verifyAll(); + + reset(taskQueue, indexerMetadataStorageCoordinator); + expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + taskQueue.shutdown("id2"); + taskQueue.shutdown("id3"); + replay(taskQueue, indexerMetadataStorageCoordinator); + + supervisor.resetInternal(null); + verifyAll(); + } + + @Test + public void testNoDataIngestionTasks() throws Exception + { + final DateTime startTime = DateTimes.nowUtc(); + supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + //not adding any events + Task id1 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 0), + + shardId0, + + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 3), + + shardId0, + + getSequenceNumber(res, shardId0, 1) + )), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + Task id3 = createKinesisIndexTask( + "id3", + DATASOURCE, + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 3), + + shardId0, + + getSequenceNumber(res, shardId0, 1) + )), + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + 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 KinesisDataSourceMetadata( + null + ) + ).anyTimes(); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 3), + + shardId0, + + getSequenceNumber(res, shardId0, 1) + )); + 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(); + verifyAll(); + + reset(taskQueue, indexerMetadataStorageCoordinator); + expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + taskQueue.shutdown("id1"); + taskQueue.shutdown("id2"); + taskQueue.shutdown("id3"); + replay(taskQueue, indexerMetadataStorageCoordinator); + + supervisor.resetInternal(null); + verifyAll(); + } + + /* + @Test(timeout = 60_000L) + public void testCheckpointForInactiveTaskGroup() + throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException + { + supervisor = getSupervisor(2, 1, true, "PT1H", null, null); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + //not adding any events + final Task id1 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 0), + + shardId0, + + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + final Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 3), + + shardId0, + + getSequenceNumber(res, shardId0, 1) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + final Task id3 = createKinesisIndexTask( + "id3", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 3), + + shardId0, + + getSequenceNumber(res, shardId0, 1) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); + final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); + Collection workItems = new ArrayList<>(); + workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); + workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); + workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); + + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + 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 KinesisDataSourceMetadata( + null) + ).anyTimes(); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.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> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 3), + + shardId0, + + getSequenceNumber(res, shardId0, 1) + )); + 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 fakeCheckpoints = Collections.emptyMap(); + supervisor.moveTaskGroupToPendingCompletion(0); + supervisor.checkpoint( + 0, + ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), + new KinesisDataSourceMetadata(new KinesisPartitions(stream, checkpoints.get(0))), + new KinesisDataSourceMetadata(new KinesisPartitions(stream, fakeCheckpoints)) + ); + + while (supervisor.getNoticesQueueSize() > 0) { + Thread.sleep(100); + } + + verifyAll(); + + Assert.assertNull(serviceEmitter.getStackTrace(), serviceEmitter.getStackTrace()); + Assert.assertNull(serviceEmitter.getExceptionMessage(), 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 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 0), + + shardId0, + + getSequenceNumber(res, shardId0, 0) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + final Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 3), + + shardId0, + + getSequenceNumber(res, shardId0, 1) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + final Task id3 = createKinesisIndexTask( + "id3", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + + getSequenceNumber(res, shardId1, 3), + + shardId0, + + getSequenceNumber(res, shardId0, 1) + )), + new KinesisPartitions(stream, ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER + )), + 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 KinesisDataSourceMetadata( + null) + ).anyTimes(); + + replayAll(); + + supervisor.start(); + + supervisor.checkpoint( + 0, + ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), + new KinesisDataSourceMetadata(new KinesisPartitions(stream, Collections.emptyMap())), + new KinesisDataSourceMetadata(new KinesisPartitions(stream, Collections.emptyMap())) + ); + + while (supervisor.getNoticesQueueSize() > 0) { + Thread.sleep(100); + } + + verifyAll(); + + while (serviceEmitter.getStackTrace() == null) { + Thread.sleep(100); + } + + Assert.assertTrue(serviceEmitter.getStackTrace() + .startsWith("org.apache.druid.java.util.common.ISE: WTH?! cannot find")); + Assert.assertEquals( + "WTH?! cannot find taskGroup [0] among all taskGroups [{}]", + serviceEmitter.getExceptionMessage() + ); + Assert.assertEquals(ISE.class, serviceEmitter.getExceptionClass()); + } + + @Test(timeout = 60_000L) + public void testCheckpointWithNullTaskGroupId() + throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException + { + supervisor = getSupervisor(1, 3, true, "PT1S", null, null, false); + //not adding any events + final Task id1 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), + new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE)), + null, + null + ); + + final Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), + new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE)), + null, null ); - List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); + final Task id3 = createKinesisIndexTask( + "id3", + DATASOURCE, + 0, + new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), + new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE)), + null, + null + ); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) - .anyTimes(); + expect( + indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KinesisDataSourceMetadata( + null) + ).anyTimes(); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + expect(taskClient.getStatusAsync(anyString())) + .andReturn(Futures.immediateFuture(Status.READING)) + .anyTimes(); + final TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of(0, 0L)); + expect(taskClient.getCheckpointsAsync(anyString(), anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(3); expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + expect(taskClient.pauseAsync(anyString())) + .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L))) + .anyTimes(); + expect(taskClient.setEndOffsetsAsync(anyString(), EasyMock.eq(ImmutableMap.of(0, 10L)), anyBoolean())) + .andReturn(Futures.immediateFuture(true)) + .anyTimes(); + + replayAll(); + + supervisor.start(); + + supervisor.runInternal(); + + final TreeMap> newCheckpoints = new TreeMap<>(); + newCheckpoints.put(0, ImmutableMap.of(0, 10L)); + supervisor.checkpoint( + null, + ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), + new KinesisDataSourceMetadata(new KinesisPartitions(stream, checkpoints.get(0))), + new KinesisDataSourceMetadata(new KinesisPartitions(stream, newCheckpoints.get(0))) + ); + + while (supervisor.getNoticesQueueSize() > 0) { + Thread.sleep(100); + } + + verifyAll(); + } + */ + + @Test + public void testSuspendedNoRunningTasks() throws Exception + { + supervisor = getSupervisor(1, 1, true, "PT1H", null, null, true); + + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); - expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(false)); + // this asserts that taskQueue.add does not in fact get called because supervisor should be suspended + expect(taskQueue.add(anyObject())).andAnswer((IAnswer) () -> { + Assert.fail(); + return null; + }).anyTimes(); taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); - taskQueue.shutdown("id3"); - - expect(taskQueue.add(anyObject(Task.class))).andReturn(true); - - TreeMap> checkpoints = new TreeMap<>(); - checkpoints.put(0, ImmutableMap.of( - shardId0, - getSequenceNumber(res, shardId0, 0), - shardId1, - getSequenceNumber(res, shardId1, 0) - )); - - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(2); - replayAll(); + supervisor.start(); supervisor.runInternal(); verifyAll(); } - // TODO: delete redundant tasks @Test - public void testKillBadPartitionAssignment() throws Exception + public void testSuspendedRunningTasks() throws Exception { - supervisor = getSupervisor(1, 2, true, "PT1H", null, null); + // graceful shutdown is expected to be called on running tasks since state is suspended + + final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); + final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); + final DateTime startTime = DateTimes.nowUtc(); + + supervisor = getSupervisor(2, 1, true, "PT1H", null, null, true); AmazonKinesis kinesis = getKinesisClientInstance(); List res = insertData(kinesis, generateRecordsRequests(stream)); @@ -804,1821 +2983,156 @@ public void testKillBadPartitionAssignment() throws Exception "id1", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of( + new KinesisPartitions("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0) + getSequenceNumber(res, shardId1, 0), + shardId0, + getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new KinesisPartitions("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 12) + KinesisPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + KinesisPartitions.NO_END_SEQUENCE_NUMBER )), null, null ); + Task id2 = createKinesisIndexTask( "id2", DATASOURCE, - 1, - new KinesisPartitions(stream, ImmutableMap.of( + 0, + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3), shardId0, - getSequenceNumber(res, shardId0, 0) + getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions(stream, ImmutableMap.of( + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - getSequenceNumber(res, shardId0, 1) + KinesisPartitions.NO_END_SEQUENCE_NUMBER )), null, null ); + Task id3 = createKinesisIndexTask( "id3", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of( - shardId0, - getSequenceNumber(res, shardId0, 0), - shardId1, - getSequenceNumber(res, shardId1, 0) - )), - new KinesisPartitions(stream, ImmutableMap.of( - shardId0, - getSequenceNumber(res, shardId0, 1), + new KinesisPartitions("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 12) - )), - null, - null - ); - Task id4 = createKinesisIndexTask( - "id4", - DATASOURCE, - 0, - new KinesisPartitions(stream, ImmutableMap.of( - shardId0, - getSequenceNumber(res, shardId0, 0) - )), - new KinesisPartitions(stream, ImmutableMap.of( + getSequenceNumber(res, shardId1, 3), shardId0, getSequenceNumber(res, shardId0, 1) )), - null, - null - ); - Task id5 = createKinesisIndexTask( - "id5", - DATASOURCE, - 0, - new KinesisPartitions(stream, ImmutableMap.of( - shardId0, - getSequenceNumber(res, shardId0, 0) - )), - new KinesisPartitions(stream, ImmutableMap.of( + new KinesisPartitions("stream", ImmutableMap.of( + shardId1, + KinesisPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - getSequenceNumber(res, shardId0, 1) + KinesisPartitions.NO_END_SEQUENCE_NUMBER )), null, null ); - List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); + Collection workItems = new ArrayList<>(); + workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); + workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).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.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); - expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).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(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); - expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); - expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) - .anyTimes(); - expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); - expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); - expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3), + shardId0, + getSequenceNumber(res, shardId0, 1) + )); - TreeMap> checkpoints1 = new TreeMap<>(); - checkpoints1.put(0, ImmutableMap.of(shardId1, getSequenceNumber(res, shardId1, 0))); - TreeMap> checkpoints2 = new TreeMap<>(); - checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(1); + // getCheckpoints will not be called for id1 as it is in publishing state + TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3), + shardId0, + getSequenceNumber(res, shardId0, 1) + )); expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) + .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)); - taskQueue.shutdown("id4"); - taskQueue.shutdown("id5"); + + expect(taskClient.pauseAsync("id2")) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 12), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); + expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 12), + shardId0, + getSequenceNumber(res, shardId0, 1) + ), true)) + .andReturn(Futures.immediateFuture(true)); + taskQueue.shutdown("id3"); + expectLastCall().times(2); + + replayAll(); + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + } + + @Test + public void testResetSuspended() throws Exception + { + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); + supervisor = getSupervisor(1, 1, true, "PT1H", null, null, true); supervisor.start(); supervisor.runInternal(); verifyAll(); + + reset(indexerMetadataStorageCoordinator); + expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + replay(indexerMetadataStorageCoordinator); + + supervisor.resetInternal(null); + verifyAll(); } -// -// @Test -// public void testRequeueTaskWhenFailed() throws Exception -// { -// supervisor = getSupervisor(2, 2, true, "PT1H", null, null, false); -// addSomeEvents(1); -// -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) -// .anyTimes(); -// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true).times(4); -// -// TreeMap> checkpoints1 = new TreeMap<>(); -// checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); -// TreeMap> checkpoints2 = new TreeMap<>(); -// checkpoints2.put(0, ImmutableMap.of(1, 0L)); -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints1)) -// .anyTimes(); -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints2)) -// .anyTimes(); -// -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// List tasks = captured.getValues(); -// -// // test that running the main loop again checks the status of the tasks that were created and does nothing if they -// // are all still running -// reset(taskStorage); -// expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); -// for (Task task : tasks) { -// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); -// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); -// } -// replay(taskStorage); -// -// supervisor.runInternal(); -// verifyAll(); -// -// // test that a task failing causes a new task to be re-queued with the same parameters -// Capture aNewTaskCapture = Capture.newInstance(); -// List imStillAlive = tasks.subList(0, 3); -// KinesisIndexTask iHaveFailed = (KinesisIndexTask) tasks.get(3); -// reset(taskStorage); -// reset(taskQueue); -// expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes(); -// for (Task task : imStillAlive) { -// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); -// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); -// } -// expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); -// expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); -// expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); -// replay(taskStorage); -// replay(taskQueue); -// -// supervisor.runInternal(); -// verifyAll(); -// -// Assert.assertNotEquals(iHaveFailed.getId(), aNewTaskCapture.getValue().getId()); -// Assert.assertEquals( -// iHaveFailed.getIOConfig().getBaseSequenceName(), -// ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getBaseSequenceName() -// ); -// } -// -// @Test -// public void testRequeueAdoptedTaskWhenFailed() throws Exception -// { -// supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false); -// addSomeEvents(1); -// -// DateTime now = DateTimes.nowUtc(); -// DateTime maxi = now.plusMinutes(60); -// Task id1 = createKinesisIndexTask( -// "id1", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 2, 0L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// now, -// maxi -// ); -// -// List existingTasks = ImmutableList.of(id1); -// -// Capture captured = Capture.newInstance(); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); -// expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); -// expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); -// expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// -// TreeMap> checkpoints = new TreeMap<>(); -// checkpoints.put(0, ImmutableMap.of(0, 0L, 2, 0L)); -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints)) -// .times(2); -// -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// // check that replica tasks are created with the same minimumMessageTime as tasks inherited from another supervisor -// Assert.assertEquals(now, ((KinesisIndexTask) captured.getValue()).getIOConfig().getMinimumMessageTime().get()); -// -// // test that a task failing causes a new task to be re-queued with the same parameters -// String runningTaskId = captured.getValue().getId(); -// Capture aNewTaskCapture = Capture.newInstance(); -// KinesisIndexTask iHaveFailed = (KinesisIndexTask) existingTasks.get(0); -// reset(taskStorage); -// reset(taskQueue); -// reset(taskClient); -// -// // for the newly created replica task -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints)) -// .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.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); -// expect(taskStorage.getStatus(runningTaskId)).andReturn(Optional.of(TaskStatus.running(runningTaskId))).anyTimes(); -// expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); -// expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); -// expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(Status.READING)); -// expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes(); -// expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); -// replay(taskStorage); -// replay(taskQueue); -// replay(taskClient); -// -// supervisor.runInternal(); -// verifyAll(); -// -// Assert.assertNotEquals(iHaveFailed.getId(), aNewTaskCapture.getValue().getId()); -// Assert.assertEquals( -// iHaveFailed.getIOConfig().getBaseSequenceName(), -// ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getBaseSequenceName() -// ); -// -// // check that failed tasks are recreated with the same minimumMessageTime as the task it replaced, even if that -// // task came from another supervisor -// Assert.assertEquals( -// now, -// ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getMinimumMessageTime().get() -// ); -// Assert.assertEquals( -// maxi, -// ((KinesisIndexTask) aNewTaskCapture.getValue()).getIOConfig().getMaximumMessageTime().get() -// ); -// } -// -// @Test -// public void testQueueNextTasksOnSuccess() throws Exception -// { -// supervisor = getSupervisor(2, 2, true, "PT1H", null, null, false); -// addSomeEvents(1); -// -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) -// .anyTimes(); -// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true).times(4); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// List tasks = captured.getValues(); -// -// reset(taskStorage); -// reset(taskClient); -// -// expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) -// .anyTimes(); -// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); -// TreeMap> checkpoints1 = new TreeMap<>(); -// checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); -// TreeMap> checkpoints2 = new TreeMap<>(); -// checkpoints2.put(0, ImmutableMap.of(1, 0L)); -// // there would be 4 tasks, 2 for each task group -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints1)) -// .times(2); -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints2)) -// .times(2); -// -// expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); -// for (Task task : tasks) { -// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); -// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); -// } -// replay(taskStorage); -// replay(taskClient); -// -// supervisor.runInternal(); -// verifyAll(); -// -// // test that a task succeeding causes a new task to be re-queued with the next offset range and causes any replica -// // tasks to be shutdown -// Capture newTasksCapture = Capture.newInstance(CaptureType.ALL); -// Capture shutdownTaskIdCapture = Capture.newInstance(); -// List imStillRunning = tasks.subList(1, 4); -// KinesisIndexTask iAmSuccess = (KinesisIndexTask) tasks.get(0); -// reset(taskStorage); -// reset(taskQueue); -// reset(taskClient); -// expect(taskStorage.getActiveTasks()).andReturn(imStillRunning).anyTimes(); -// for (Task task : imStillRunning) { -// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); -// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); -// } -// expect(taskStorage.getStatus(iAmSuccess.getId())).andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId()))); -// expect(taskStorage.getTask(iAmSuccess.getId())).andReturn(Optional.of((Task) iAmSuccess)).anyTimes(); -// expect(taskQueue.add(capture(newTasksCapture))).andReturn(true).times(2); -// expect(taskClient.stopAsync(capture(shutdownTaskIdCapture), eq(false))).andReturn(Futures.immediateFuture(true)); -// replay(taskStorage); -// replay(taskQueue); -// replay(taskClient); -// -// supervisor.runInternal(); -// verifyAll(); -// -// // make sure we killed the right task (sequenceName for replicas are the same) -// Assert.assertTrue(shutdownTaskIdCapture.getValue().contains(iAmSuccess.getIOConfig().getBaseSequenceName())); -// } -// -// @Test -// public void testBeginPublishAndQueueNextTasks() throws Exception -// { -// final TaskLocation location = new TaskLocation("testHost", 1234, -1); -// -// supervisor = getSupervisor(2, 2, true, "PT1M", null, null, false); -// addSomeEvents(100); -// -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true).times(4); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// List tasks = captured.getValues(); -// Collection workItems = new ArrayList<>(); -// for (Task task : tasks) { -// workItems.add(new TestTaskRunnerWorkItem(task, null, location)); -// } -// -// reset(taskStorage, taskRunner, taskClient, taskQueue); -// captured = Capture.newInstance(CaptureType.ALL); -// expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); -// for (Task task : tasks) { -// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); -// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); -// } -// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); -// expect(taskClient.getStatusAsync(anyString())) -// .andReturn(Futures.immediateFuture(Status.READING)) -// .anyTimes(); -// expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) -// .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) -// .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); -// expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) -// .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) -// .times(2); -// expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); -// expect( -// taskClient.setEndOffsetsAsync( -// EasyMock.contains("sequenceName-0"), -// EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), -// EasyMock.eq(true) -// ) -// ).andReturn(Futures.immediateFuture(true)).times(2); -// expect(taskQueue.add(capture(captured))).andReturn(true).times(2); -// -// TreeMap> checkpoints1 = new TreeMap<>(); -// checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); -// TreeMap> checkpoints2 = new TreeMap<>(); -// checkpoints2.put(0, ImmutableMap.of(1, 0L)); -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints1)) -// .times(2); -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints2)) -// .times(2); -// -// replay(taskStorage, taskRunner, taskClient, taskQueue); -// -// supervisor.runInternal(); -// verifyAll(); -// -// for (Task task : captured.getValues()) { -// KinesisIndexTask KinesisIndexTask = (KinesisIndexTask) task; -// Assert.assertEquals(dataSchema, KinesisIndexTask.getDataSchema()); -// Assert.assertEquals(tuningConfig.copyOf(), KinesisIndexTask.getTuningConfig()); -// -// KinesisSupervisor taskConfig = KinesisIndexTask.getIOConfig(); -// Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); -// Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); -// -// Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); -// Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); -// Assert.assertEquals(35L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); -// } -// } -// -// @Test -// public void testDiscoverExistingPublishingTask() throws Exception -// { -// final TaskLocation location = new TaskLocation("testHost", 1234, -1); -// -// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); -// addSomeEvents(1); -// -// Task task = createKinesisIndexTask( -// "id1", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Collection workItems = new ArrayList<>(); -// workItems.add(new TestTaskRunnerWorkItem(task, null, location)); -// -// Capture captured = Capture.newInstance(); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); -// expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); -// expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); -// expect(taskClient.getCurrentOffsetsAsync("id1", false)) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); -// expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); -// expect(taskQueue.add(capture(captured))).andReturn(true); -// -// TreeMap> checkpoints = new TreeMap<>(); -// checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)); -// expect(taskClient.getCheckpoints(anyString(), anyBoolean())).andReturn(checkpoints).anyTimes(); -// -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// supervisor.updateCurrentAndLatestOffsets().run(); -// SupervisorReport report = supervisor.getStatus(); -// verifyAll(); -// -// Assert.assertEquals(DATASOURCE, report.getId()); -// -// KinesisSupervisorReportPayload payload = report.getPayload(); -// -// Assert.assertEquals(DATASOURCE, payload.getDataSource()); -// Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); -// Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); -// Assert.assertEquals(1, (int) payload.getReplicas()); -// Assert.assertEquals(stream, payload.getStream()); -// Assert.assertEquals(0, payload.getActiveTasks().size()); -// Assert.assertEquals(1, payload.getPublishingTasks().size()); -// -// TaskReportData publishingReport = payload.getPublishingTasks().get(0); -// -// Assert.assertEquals("id1", publishingReport.getId()); -// Assert.assertEquals(ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), publishingReport.getStartingOffsets()); -// Assert.assertEquals(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), publishingReport.getCurrentOffsets()); -// -// KinesisIndexTask capturedTask = captured.getValue(); -// Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); -// Assert.assertEquals(tuningConfig.copyOf(), capturedTask.getTuningConfig()); -// -// KinesisSupervisor capturedTaskConfig = capturedTask.getIOConfig(); -// Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); -// Assert.assertEquals("myCustomValue", capturedTaskConfig.getConsumerProperties().get("myCustomKey")); -// Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); -// Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); -// -// // check that the new task was created with starting offsets matching where the publishing task finished -// Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); -// Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(20L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); -// Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); -// -// Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream()); -// Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)); -// Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)); -// } -// -// @Test -// public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() throws Exception -// { -// final TaskLocation location = new TaskLocation("testHost", 1234, -1); -// -// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); -// addSomeEvents(1); -// -// Task task = createKinesisIndexTask( -// "id1", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 2, 0L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Collection workItems = new ArrayList<>(); -// workItems.add(new TestTaskRunnerWorkItem(task, null, location)); -// -// Capture captured = Capture.newInstance(); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); -// expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); -// expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); -// expect(taskClient.getCurrentOffsetsAsync("id1", false)) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 2, 30L))); -// expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); -// expect(taskQueue.add(capture(captured))).andReturn(true); -// -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// supervisor.updateCurrentAndLatestOffsets().run(); -// SupervisorReport report = supervisor.getStatus(); -// verifyAll(); -// -// Assert.assertEquals(DATASOURCE, report.getId()); -// -// KinesisSupervisorReportPayload payload = report.getPayload(); -// -// Assert.assertEquals(DATASOURCE, payload.getDataSource()); -// Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); -// Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); -// Assert.assertEquals(1, (int) payload.getReplicas()); -// Assert.assertEquals(stream, payload.getStream()); -// Assert.assertEquals(0, payload.getActiveTasks().size()); -// Assert.assertEquals(1, payload.getPublishingTasks().size()); -// -// TaskReportData publishingReport = payload.getPublishingTasks().get(0); -// -// Assert.assertEquals("id1", publishingReport.getId()); -// Assert.assertEquals(ImmutableMap.of(0, 0L, 2, 0L), publishingReport.getStartingOffsets()); -// Assert.assertEquals(ImmutableMap.of(0, 10L, 2, 30L), publishingReport.getCurrentOffsets()); -// -// KinesisIndexTask capturedTask = captured.getValue(); -// Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); -// Assert.assertEquals(tuningConfig.copyOf(), capturedTask.getTuningConfig()); -// -// KinesisSupervisor capturedTaskConfig = capturedTask.getIOConfig(); -// Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); -// Assert.assertEquals("myCustomValue", capturedTaskConfig.getConsumerProperties().get("myCustomKey")); -// Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); -// Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); -// -// // check that the new task was created with starting offsets matching where the publishing task finished -// Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); -// Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(0L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); -// Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); -// -// Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream()); -// Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)); -// Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)); -// } -// -// @Test -// public void testDiscoverExistingPublishingAndReadingTask() throws Exception -// { -// final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); -// final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); -// final DateTime startTime = DateTimes.nowUtc(); -// -// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); -// addSomeEvents(6); -// -// Task id1 = createKinesisIndexTask( -// "id1", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Task id2 = createKinesisIndexTask( -// "id2", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Collection workItems = new ArrayList<>(); -// workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); -// workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); -// -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2)).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.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); -// expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); -// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); -// expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); -// expect(taskClient.getCurrentOffsetsAsync("id1", false)) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 1L, 1, 2L, 2, 3L))); -// expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); -// expect(taskClient.getCurrentOffsetsAsync("id2", false)) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 4L, 1, 5L, 2, 6L))); -// -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// -// // since id1 is publishing, so getCheckpoints wouldn't be called for it -// TreeMap> checkpoints = new TreeMap<>(); -// checkpoints.put(0, ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints)) -// .times(1); -// -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// supervisor.updateCurrentAndLatestOffsets().run(); -// SupervisorReport report = supervisor.getStatus(); -// verifyAll(); -// -// Assert.assertEquals(DATASOURCE, report.getId()); -// -// KinesisSupervisorReportPayload payload = report.getPayload(); -// -// Assert.assertEquals(DATASOURCE, payload.getDataSource()); -// Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); -// Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); -// Assert.assertEquals(1, (int) payload.getReplicas()); -// Assert.assertEquals(stream, payload.getStream()); -// Assert.assertEquals(1, payload.getActiveTasks().size()); -// Assert.assertEquals(1, payload.getPublishingTasks().size()); -// -// TaskReportData activeReport = payload.getActiveTasks().get(0); -// TaskReportData publishingReport = payload.getPublishingTasks().get(0); -// -// Assert.assertEquals("id2", activeReport.getId()); -// Assert.assertEquals(startTime, activeReport.getStartTime()); -// Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L), activeReport.getStartingOffsets()); -// Assert.assertEquals(ImmutableMap.of(0, 4L, 1, 5L, 2, 6L), activeReport.getCurrentOffsets()); -// Assert.assertEquals(ImmutableMap.of(0, 2L, 1, 1L, 2, 0L), activeReport.getLag()); -// -// Assert.assertEquals("id1", publishingReport.getId()); -// Assert.assertEquals(ImmutableMap.of(0, 0L, 1, 0L, 2, 0L), publishingReport.getStartingOffsets()); -// Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L), publishingReport.getCurrentOffsets()); -// Assert.assertEquals(null, publishingReport.getLag()); -// -// Assert.assertEquals(ImmutableMap.of(0, 6L, 1, 6L, 2, 6L), payload.getLatestOffsets()); -// Assert.assertEquals(ImmutableMap.of(0, 2L, 1, 1L, 2, 0L), payload.getMinimumLag()); -// Assert.assertEquals(3L, (long) payload.getAggregateLag()); -// Assert.assertTrue(payload.getOffsetsLastUpdated().plusMinutes(1).isAfterNow()); -// } -// -// @Test -// public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception -// { -// supervisor = getSupervisor(2, 2, true, "PT1H", null, null, false); -// addSomeEvents(1); -// -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true).times(4); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// List tasks = captured.getValues(); -// -// reset(taskStorage, taskClient, taskQueue); -// -// TreeMap> checkpoints1 = new TreeMap<>(); -// checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); -// TreeMap> checkpoints2 = new TreeMap<>(); -// checkpoints2.put(0, ImmutableMap.of(1, 0L)); -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints1)) -// .times(2); -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints2)) -// .times(2); -// -// expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); -// for (Task task : tasks) { -// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); -// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); -// expect(taskClient.getStatusAsync(task.getId())) -// .andReturn(Futures.immediateFuture(Status.NOT_STARTED)); -// expect(taskClient.getStartTimeAsync(task.getId())) -// .andReturn(Futures.immediateFailedFuture(new RuntimeException())); -// taskQueue.shutdown(task.getId()); -// } -// replay(taskStorage, taskClient, taskQueue); -// -// supervisor.runInternal(); -// verifyAll(); -// } -// -// @Test -// public void testKillUnresponsiveTasksWhilePausing() throws Exception -// { -// final TaskLocation location = new TaskLocation("testHost", 1234, -1); -// -// supervisor = getSupervisor(2, 2, true, "PT1M", null, null, false); -// addSomeEvents(100); -// -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true).times(4); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// List tasks = captured.getValues(); -// Collection workItems = new ArrayList<>(); -// for (Task task : tasks) { -// workItems.add(new TestTaskRunnerWorkItem(task, null, location)); -// } -// -// reset(taskStorage, taskRunner, taskClient, taskQueue); -// -// TreeMap> checkpoints1 = new TreeMap<>(); -// checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); -// TreeMap> checkpoints2 = new TreeMap<>(); -// checkpoints2.put(0, ImmutableMap.of(1, 0L)); -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints1)) -// .times(2); -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints2)) -// .times(2); -// -// captured = Capture.newInstance(CaptureType.ALL); -// expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); -// for (Task task : tasks) { -// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); -// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); -// } -// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); -// expect(taskClient.getStatusAsync(anyString())) -// .andReturn(Futures.immediateFuture(Status.READING)) -// .anyTimes(); -// expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) -// .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) -// .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); -// expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) -// .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) -// .times(2); -// expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) -// .andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); -// taskQueue.shutdown(EasyMock.contains("sequenceName-0")); -// expectLastCall().times(2); -// expect(taskQueue.add(capture(captured))).andReturn(true).times(2); -// -// replay(taskStorage, taskRunner, taskClient, taskQueue); -// -// supervisor.runInternal(); -// verifyAll(); -// -// for (Task task : captured.getValues()) { -// KinesisSupervisor taskConfig = ((KinesisIndexTask) task).getIOConfig(); -// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); -// } -// } -// -// @Test -// public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception -// { -// final TaskLocation location = new TaskLocation("testHost", 1234, -1); -// -// supervisor = getSupervisor(2, 2, true, "PT1M", null, null, false); -// addSomeEvents(100); -// -// Capture captured = Capture.newInstance(CaptureType.ALL); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskQueue.add(capture(captured))).andReturn(true).times(4); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// List tasks = captured.getValues(); -// Collection workItems = new ArrayList<>(); -// for (Task task : tasks) { -// workItems.add(new TestTaskRunnerWorkItem(task, null, location)); -// } -// -// reset(taskStorage, taskRunner, taskClient, taskQueue); -// -// TreeMap> checkpoints1 = new TreeMap<>(); -// checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); -// TreeMap> checkpoints2 = new TreeMap<>(); -// checkpoints2.put(0, ImmutableMap.of(1, 0L)); -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints1)) -// .times(2); -// expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints2)) -// .times(2); -// -// captured = Capture.newInstance(CaptureType.ALL); -// expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); -// for (Task task : tasks) { -// expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); -// expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); -// } -// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); -// expect(taskClient.getStatusAsync(anyString())) -// .andReturn(Futures.immediateFuture(Status.READING)) -// .anyTimes(); -// expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) -// .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) -// .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); -// expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) -// .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) -// .times(2); -// expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); -// expect( -// taskClient.setEndOffsetsAsync( -// EasyMock.contains("sequenceName-0"), -// EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), -// EasyMock.eq(true) -// ) -// ).andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); -// taskQueue.shutdown(EasyMock.contains("sequenceName-0")); -// expectLastCall().times(2); -// expect(taskQueue.add(capture(captured))).andReturn(true).times(2); -// -// replay(taskStorage, taskRunner, taskClient, taskQueue); -// -// supervisor.runInternal(); -// verifyAll(); -// -// for (Task task : captured.getValues()) { -// KinesisSupervisor taskConfig = ((KinesisIndexTask) task).getIOConfig(); -// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); -// Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); -// } -// } -// -// @Test(expected = IllegalStateException.class) -// public void testStopNotStarted() -// { -// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); -// supervisor.stop(false); -// } -// -// @Test -// public void testStop() -// { -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// taskClient.close(); -// taskRunner.unregisterListener(StringUtils.format("KinesisSupervisor-%s", DATASOURCE)); -// replayAll(); -// -// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); -// supervisor.start(); -// supervisor.stop(false); -// -// verifyAll(); -// } -// -// @Test -// public void testStopGracefully() throws Exception -// { -// final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); -// final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); -// final DateTime startTime = DateTimes.nowUtc(); -// -// supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false); -// addSomeEvents(1); -// -// Task id1 = createKinesisIndexTask( -// "id1", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Task id2 = createKinesisIndexTask( -// "id2", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Task id3 = createKinesisIndexTask( -// "id3", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Collection workItems = new ArrayList<>(); -// workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); -// workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); -// -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(workItems).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 KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); -// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); -// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); -// expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); -// expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); -// expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); -// -// // getCheckpoints will not be called for id1 as it is in publishing state -// TreeMap> checkpoints = new TreeMap<>(); -// 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("id3"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints)) -// .times(1); -// -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// reset(taskRunner, taskClient, taskQueue); -// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); -// expect(taskClient.pauseAsync("id2")) -// .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); -// expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) -// .andReturn(Futures.immediateFuture(true)); -// taskQueue.shutdown("id3"); -// expectLastCall().times(2); -// -// replay(taskRunner, taskClient, taskQueue); -// -// supervisor.gracefulShutdownInternal(); -// verifyAll(); -// } -// -// @Test -// public void testResetNoTasks() throws Exception -// { -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// reset(indexerMetadataStorageCoordinator); -// expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); -// replay(indexerMetadataStorageCoordinator); -// -// supervisor.resetInternal(null); -// verifyAll(); -// -// } -// -// @Test -// public void testResetDataSourceMetadata() throws Exception -// { -// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// Capture captureDataSource = EasyMock.newCapture(); -// Capture captureDataSourceMetadata = EasyMock.newCapture(); -// -// KinesisDataSourceMetadata KinesisDataSourceMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( -// stream, -// ImmutableMap.of(0, 1000L, 1, 1000L, 2, 1000L) -// )); -// -// KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( -// stream, -// ImmutableMap.of(1, 1000L, 2, 1000L) -// )); -// -// KinesisDataSourceMetadata expectedMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( -// stream, -// ImmutableMap.of(0, 1000L) -// )); -// -// reset(indexerMetadataStorageCoordinator); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(KinesisDataSourceMetadata); -// expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata( -// EasyMock.capture(captureDataSource), -// EasyMock.capture(captureDataSourceMetadata) -// )).andReturn(true); -// replay(indexerMetadataStorageCoordinator); -// -// try { -// supervisor.resetInternal(resetMetadata); -// } -// catch (NullPointerException npe) { -// // Expected as there will be an attempt to reset partitionGroups offsets to NOT_SET -// // however there would be no entries in the map as we have not put nay data in kafka -// Assert.assertTrue(npe.getCause() == null); -// } -// verifyAll(); -// -// Assert.assertEquals(captureDataSource.getValue(), DATASOURCE); -// Assert.assertEquals(captureDataSourceMetadata.getValue(), expectedMetadata); -// } -// -// @Test -// public void testResetNoDataSourceMetadata() throws Exception -// { -// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( -// stream, -// ImmutableMap.of(1, 1000L, 2, 1000L) -// )); -// -// reset(indexerMetadataStorageCoordinator); -// // no DataSourceMetadata in metadata store -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null); -// replay(indexerMetadataStorageCoordinator); -// -// supervisor.resetInternal(resetMetadata); -// verifyAll(); -// } -// -// @Test -// public void testResetRunningTasks() throws Exception -// { -// final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); -// final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); -// final DateTime startTime = DateTimes.nowUtc(); -// -// supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false); -// addSomeEvents(1); -// -// Task id1 = createKinesisIndexTask( -// "id1", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Task id2 = createKinesisIndexTask( -// "id2", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Task id3 = createKinesisIndexTask( -// "id3", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Collection workItems = new ArrayList<>(); -// workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); -// workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); -// -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(workItems).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 KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); -// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); -// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); -// expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); -// expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); -// expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); -// -// TreeMap> checkpoints = new TreeMap<>(); -// 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("id3"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints)) -// .times(1); -// -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// reset(taskQueue, indexerMetadataStorageCoordinator); -// expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); -// taskQueue.shutdown("id2"); -// taskQueue.shutdown("id3"); -// replay(taskQueue, indexerMetadataStorageCoordinator); -// -// supervisor.resetInternal(null); -// verifyAll(); -// } -// -// @Test -// public void testNoDataIngestionTasks() throws Exception -// { -// final DateTime startTime = DateTimes.nowUtc(); -// supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); -// //not adding any events -// Task id1 = createKinesisIndexTask( -// "id1", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Task id2 = createKinesisIndexTask( -// "id2", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Task id3 = createKinesisIndexTask( -// "id3", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions("stream", 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 KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); -// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); -// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); -// expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); -// expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); -// expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); -// -// TreeMap> 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(); -// verifyAll(); -// -// reset(taskQueue, indexerMetadataStorageCoordinator); -// expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); -// taskQueue.shutdown("id1"); -// taskQueue.shutdown("id2"); -// taskQueue.shutdown("id3"); -// replay(taskQueue, indexerMetadataStorageCoordinator); -// -// supervisor.resetInternal(null); -// 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 = createKinesisIndexTask( -// "id1", -// DATASOURCE, -// 0, -// new KinesisPartitions(stream, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// final Task id2 = createKinesisIndexTask( -// "id2", -// DATASOURCE, -// 0, -// new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// final Task id3 = createKinesisIndexTask( -// "id3", -// DATASOURCE, -// 0, -// new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); -// final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); -// Collection workItems = new ArrayList<>(); -// workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); -// workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); -// workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); -// -// expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); -// 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 KinesisDataSourceMetadata( -// null) -// ).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); -// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); -// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(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> 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 fakeCheckpoints = Collections.emptyMap(); -// supervisor.moveTaskGroupToPendingCompletion(0); -// supervisor.checkpoint( -// 0, -// ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, checkpoints.get(0))), -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, fakeCheckpoints)) -// ); -// -// while (supervisor.getNoticesQueueSize() > 0) { -// Thread.sleep(100); -// } -// -// verifyAll(); -// -// Assert.assertNull(serviceEmitter.getStackTrace(), serviceEmitter.getStackTrace()); -// Assert.assertNull(serviceEmitter.getExceptionMessage(), 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 = createKinesisIndexTask( -// "id1", -// DATASOURCE, -// 0, -// new KinesisPartitions(stream, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// final Task id2 = createKinesisIndexTask( -// "id2", -// DATASOURCE, -// 0, -// new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// final Task id3 = createKinesisIndexTask( -// "id3", -// DATASOURCE, -// 0, -// new KinesisPartitions(stream, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions(stream, 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 KinesisDataSourceMetadata( -// null) -// ).anyTimes(); -// -// replayAll(); -// -// supervisor.start(); -// -// supervisor.checkpoint( -// 0, -// ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, Collections.emptyMap())), -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, Collections.emptyMap())) -// ); -// -// while (supervisor.getNoticesQueueSize() > 0) { -// Thread.sleep(100); -// } -// -// verifyAll(); -// -// while (serviceEmitter.getStackTrace() == null) { -// Thread.sleep(100); -// } -// -// Assert.assertTrue(serviceEmitter.getStackTrace() -// .startsWith("org.apache.druid.java.util.common.ISE: WTH?! cannot find")); -// Assert.assertEquals( -// "WTH?! cannot find taskGroup [0] among all taskGroups [{}]", -// serviceEmitter.getExceptionMessage() -// ); -// Assert.assertEquals(ISE.class, serviceEmitter.getExceptionClass()); -// } -// -// @Test(timeout = 60_000L) -// public void testCheckpointWithNullTaskGroupId() -// throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException -// { -// supervisor = getSupervisor(1, 3, true, "PT1S", null, null, false); -// //not adding any events -// final Task id1 = createKinesisIndexTask( -// "id1", -// DATASOURCE, -// 0, -// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE)), -// null, -// null -// ); -// -// final Task id2 = createKinesisIndexTask( -// "id2", -// DATASOURCE, -// 0, -// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE)), -// null, -// null -// ); -// -// final Task id3 = createKinesisIndexTask( -// "id3", -// DATASOURCE, -// 0, -// new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), -// new KinesisPartitions(stream, ImmutableMap.of(0, 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 KinesisDataSourceMetadata( -// null) -// ).anyTimes(); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// expect(taskClient.getStatusAsync(anyString())) -// .andReturn(Futures.immediateFuture(Status.READING)) -// .anyTimes(); -// final TreeMap> checkpoints = new TreeMap<>(); -// checkpoints.put(0, ImmutableMap.of(0, 0L)); -// expect(taskClient.getCheckpointsAsync(anyString(), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints)) -// .times(3); -// expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); -// expect(taskClient.pauseAsync(anyString())) -// .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L))) -// .anyTimes(); -// expect(taskClient.setEndOffsetsAsync(anyString(), EasyMock.eq(ImmutableMap.of(0, 10L)), anyBoolean())) -// .andReturn(Futures.immediateFuture(true)) -// .anyTimes(); -// -// replayAll(); -// -// supervisor.start(); -// -// supervisor.runInternal(); -// -// final TreeMap> newCheckpoints = new TreeMap<>(); -// newCheckpoints.put(0, ImmutableMap.of(0, 10L)); -// supervisor.checkpoint( -// null, -// ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, checkpoints.get(0))), -// new KinesisDataSourceMetadata(new KinesisPartitions(stream, newCheckpoints.get(0))) -// ); -// -// while (supervisor.getNoticesQueueSize() > 0) { -// Thread.sleep(100); -// } -// -// verifyAll(); -// } -// -// @Test -// public void testSuspendedNoRunningTasks() throws Exception -// { -// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false, true); -// addSomeEvents(1); -// -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( -// new KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// // this asserts that taskQueue.add does not in fact get called because supervisor should be suspended -// expect(taskQueue.add(anyObject())).andAnswer((IAnswer) () -> { -// Assert.fail(); -// return null; -// }).anyTimes(); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// } -// -// @Test -// public void testSuspendedRunningTasks() throws Exception -// { -// // graceful shutdown is expected to be called on running tasks since state is suspended -// -// final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); -// final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); -// final DateTime startTime = DateTimes.nowUtc(); -// -// supervisor = getSupervisor(2, 1, true, "PT1H", null, null, false, true); -// addSomeEvents(1); -// -// Task id1 = createKinesisIndexTask( -// "id1", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Task id2 = createKinesisIndexTask( -// "id2", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Task id3 = createKinesisIndexTask( -// "id3", -// DATASOURCE, -// 0, -// new KinesisPartitions("stream", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), -// new KinesisPartitions("stream", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), -// null, -// null -// ); -// -// Collection workItems = new ArrayList<>(); -// workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); -// workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); -// -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(workItems).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 KinesisDataSourceMetadata( -// null -// ) -// ).anyTimes(); -// expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); -// expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); -// expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); -// expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); -// expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); -// expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); -// -// // getCheckpoints will not be called for id1 as it is in publishing state -// TreeMap> checkpoints = new TreeMap<>(); -// 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("id3"), anyBoolean())) -// .andReturn(Futures.immediateFuture(checkpoints)) -// .times(1); -// -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// -// expect(taskClient.pauseAsync("id2")) -// .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); -// expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) -// .andReturn(Futures.immediateFuture(true)); -// taskQueue.shutdown("id3"); -// expectLastCall().times(2); -// -// replayAll(); -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// } -// -// @Test -// public void testResetSuspended() throws Exception -// { -// expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); -// expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); -// expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); -// expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); -// taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); -// replayAll(); -// -// supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false, true); -// supervisor.start(); -// supervisor.runInternal(); -// verifyAll(); -// -// reset(indexerMetadataStorageCoordinator); -// expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); -// replay(indexerMetadataStorageCoordinator); -// -// supervisor.resetInternal(null); -// verifyAll(); -// } -// + private static List insertData( AmazonKinesis kinesis, @@ -2872,6 +3386,84 @@ private KinesisIndexTask createKinesisIndexTask( ); } + private KinesisSupervisor getSupervisor( + int replicas, + int taskCount, + boolean useEarliestOffset, + String duration, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod, + boolean suspended + ) + { + KinesisSupervisorIOConfig KinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( + stream, + Localstack.getEndpointKinesis(), + null, + replicas, + taskCount, + new Period(duration), + new Period("P1D"), + new Period("PT30S"), + useEarliestOffset, + new Period("PT30M"), + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod, + null, + null, + cloud.localstack.TestUtils.TEST_ACCESS_KEY, + cloud.localstack.TestUtils.TEST_SECRET_KEY, + null, + null, + false + ); + + KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory( + null, + null + ) + { + @Override + public KinesisIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String dataSource, + int numThreads, + Duration httpTimeout, + long numRetries + ) + { + Assert.assertEquals(TEST_CHAT_THREADS, numThreads); + Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), httpTimeout); + Assert.assertEquals(TEST_CHAT_RETRIES, numRetries); + return taskClient; + } + }; + + return new TestableKinesisSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new KinesisSupervisorSpec( + dataSchema, + tuningConfig, + KinesisSupervisorIOConfig, + null, + suspended, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + objectMapper, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory + ), + rowIngestionMetersFactory + ); + } + private static class TestTaskRunnerWorkItem extends TaskRunnerWorkItem { private final String taskType; @@ -2940,4 +3532,5 @@ protected String generateSequenceName( return StringUtils.format("sequenceName-%d", groupId); } } + } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java index df1d79e49202..46600426732b 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java @@ -108,7 +108,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(100, config.getMaxRowsPerSegment()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); Assert.assertEquals(0, config.getMaxPendingPersists()); - Assert.assertEquals(false, config.getBuildV9Directly()); + Assert.assertEquals(true, config.getBuildV9Directly()); Assert.assertEquals(true, config.isReportParseExceptions()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); Assert.assertEquals(12, (int) config.getWorkerThreads()); From 67edb8b425177d81d42a3909525571f34111b76a Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 3 Oct 2018 17:16:21 -0700 Subject: [PATCH 17/87] removed KinesisParititons and KafkaPartitions to use SeekableStreamPartitions --- ...ementalPublishingKafkaIndexTaskRunner.java | 44 ++- .../kafka/KafkaDataSourceMetadata.java | 9 +- .../druid/indexing/kafka/KafkaIOConfig.java | 19 +- .../druid/indexing/kafka/KafkaPartitions.java | 64 --- .../indexing/kafka/KafkaRecordSupplier.java | 31 +- .../kafka/LegacyKafkaIndexTaskRunner.java | 18 +- .../kafka/supervisor/KafkaSupervisor.java | 10 +- .../kafka/KafkaDataSourceMetadataTest.java | 3 +- .../indexing/kafka/KafkaIOConfigTest.java | 2 +- .../indexing/kafka/KafkaIndexTaskTest.java | 173 ++++---- .../kafka/supervisor/KafkaSupervisorTest.java | 218 ++++++---- .../kinesis/KinesisDataSourceMetadata.java | 9 +- .../indexing/kinesis/KinesisIOConfig.java | 13 +- .../indexing/kinesis/KinesisIndexTask.java | 33 +- .../kinesis/KinesisIndexTaskClient.java | 243 +----------- .../indexing/kinesis/KinesisPartitions.java | 62 --- .../kinesis/supervisor/KinesisSupervisor.java | 10 +- .../KinesisDataSourceMetadataTest.java | 3 +- .../indexing/kinesis/KinesisIOConfigTest.java | 8 +- .../kinesis/KinesisIndexTaskTest.java | 103 ++--- .../supervisor/KinesisSupervisorTest.java | 374 +++++++++--------- .../SeekableStreamDataSourceMetadata.java | 8 +- .../SeekableStreamIOConfig.java | 8 +- .../SeekableStreamPartitions.java | 68 +++- .../supervisor/SeekableStreamSupervisor.java | 26 +- 25 files changed, 669 insertions(+), 890 deletions(-) delete mode 100644 extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java delete mode 100644 extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 0c65620ce812..2e4d06068e92 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -58,6 +58,7 @@ import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -344,10 +345,18 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception nextOffsets.putAll(sequences.get(0).startOffsets); } else { final Map restoredMetadataMap = (Map) restoredMetadata; - final KafkaPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( + final SeekableStreamPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( restoredMetadataMap.get(METADATA_NEXT_PARTITIONS), - KafkaPartitions.class + toolbox.getObjectMapper() + .getTypeFactory() + .constructParametrizedType( + SeekableStreamPartitions.class, + SeekableStreamPartitions.class, + Integer.class, + Long.class + ) ); + nextOffsets.putAll(restoredNextPartitions.getPartitionOffsetMap()); // Sanity checks. @@ -388,7 +397,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception public Object getMetadata() { return ImmutableMap.of( - METADATA_NEXT_PARTITIONS, new KafkaPartitions( + METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>( ioConfig.getStartPartitions().getTopic(), snapshot ) @@ -602,8 +611,11 @@ public void onFailure(Throwable t) task.getDataSource(), ioConfig.getTaskGroupId(), task.getIOConfig().getBaseSequenceName(), - new KafkaDataSourceMetadata(new KafkaPartitions(topic, sequenceToCheckpoint.getStartOffsets())), - new KafkaDataSourceMetadata(new KafkaPartitions(topic, nextOffsets)) + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( + topic, + sequenceToCheckpoint.getStartOffsets() + )), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, nextOffsets)) ); if (!toolbox.getTaskActionClient().submit(checkpointAction)) { throw new ISE("Checkpoint request with offsets [%s] failed, dying", nextOffsets); @@ -1129,7 +1141,7 @@ private void sendResetRequestAndWait(Map outOfRangePartiti boolean result = taskToolbox.getTaskActionClient() .submit(new ResetDataSourceMetadataAction( task.getDataSource(), - new KafkaDataSourceMetadata(new KafkaPartitions( + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( ioConfig.getStartPartitions() .getTopic(), partitionOffsetMap @@ -1736,8 +1748,8 @@ public Object getMetadata() // Publish metadata can be different from persist metadata as we are going to publish only // subset of segments return ImmutableMap.of( - METADATA_NEXT_PARTITIONS, new KafkaPartitions(topic, lastPersistedOffsets), - METADATA_PUBLISH_PARTITIONS, new KafkaPartitions(topic, endOffsets) + METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>(topic, lastPersistedOffsets), + METADATA_PUBLISH_PARTITIONS, new SeekableStreamPartitions<>(topic, endOffsets) ); } finally { @@ -1756,9 +1768,16 @@ public void run() TransactionalSegmentPublisher createPublisher(TaskToolbox toolbox, boolean useTransaction) { return (segments, commitMetadata) -> { - final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue( + final SeekableStreamPartitions finalPartitions = toolbox.getObjectMapper().convertValue( ((Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_PUBLISH_PARTITIONS), - KafkaPartitions.class + toolbox.getObjectMapper() + .getTypeFactory() + .constructParametrizedType( + SeekableStreamPartitions.class, + SeekableStreamPartitions.class, + Integer.class, + Long.class + ) ); // Sanity check, we should only be publishing things that match our desired end state. @@ -1775,7 +1794,10 @@ TransactionalSegmentPublisher createPublisher(TaskToolbox toolbox, boolean useTr if (useTransaction) { action = new SegmentTransactionalInsertAction( segments, - new KafkaDataSourceMetadata(new KafkaPartitions(finalPartitions.getTopic(), getStartOffsets())), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( + finalPartitions.getTopic(), + getStartOffsets() + )), new KafkaDataSourceMetadata(finalPartitions) ); } else { diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java index 3d98a6021e98..2ad9c443f66a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import java.util.Map; @@ -30,16 +31,16 @@ public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata kafkaPartitions ) { super(kafkaPartitions); } @JsonProperty("partitions") - public KafkaPartitions getKafkaPartitions() + public SeekableStreamPartitions getKafkaPartitions() { - return (KafkaPartitions) super.getSeekableStreamPartitions(); + return super.getSeekableStreamPartitions(); } @Override @@ -47,6 +48,6 @@ protected SeekableStreamDataSourceMetadata createConcretDataSourc String streamId, Map newMap ) { - return new KafkaDataSourceMetadata(new KafkaPartitions(streamId, newMap)); + return new KafkaDataSourceMetadata(new SeekableStreamPartitions(streamId, newMap)); } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java index a4f1f195c56e..d4d7b3d23cd3 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -41,8 +42,8 @@ public class KafkaIOConfig extends SeekableStreamIOConfig public KafkaIOConfig( @JsonProperty("taskGroupId") @Nullable Integer taskGroupId, // can be null for backward compabitility @JsonProperty("baseSequenceName") String baseSequenceName, - @JsonProperty("startPartitions") KafkaPartitions startPartitions, - @JsonProperty("endPartitions") KafkaPartitions endPartitions, + @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, + @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, @JsonProperty("consumerProperties") Map consumerProperties, @JsonProperty("useTransaction") Boolean useTransaction, @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, @@ -63,11 +64,11 @@ public KafkaIOConfig( this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); this.skipOffsetGaps = skipOffsetGaps != null ? skipOffsetGaps : DEFAULT_SKIP_OFFSET_GAPS; - for (int partition : endPartitions.getPartitionSequenceMap().keySet()) { + for (int partition : endPartitions.getMap().keySet()) { Preconditions.checkArgument( - endPartitions.getPartitionSequenceMap() + endPartitions.getMap() .get(partition) - .compareTo(startPartitions.getPartitionSequenceMap().get(partition)) >= 0, + .compareTo(startPartitions.getMap().get(partition)) >= 0, "end offset must be >= start offset for partition[%s]", partition ); @@ -76,16 +77,16 @@ public KafkaIOConfig( @Override @JsonProperty - public KafkaPartitions getStartPartitions() + public SeekableStreamPartitions getStartPartitions() { - return (KafkaPartitions) super.getStartPartitions(); + return super.getStartPartitions(); } @Override @JsonProperty - public KafkaPartitions getEndPartitions() + public SeekableStreamPartitions getEndPartitions() { - return (KafkaPartitions) super.getEndPartitions(); + return super.getEndPartitions(); } @Override diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java deleted file mode 100644 index e92bc89973a7..000000000000 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaPartitions.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.kafka; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; - -import java.util.Map; - -public class KafkaPartitions extends SeekableStreamPartitions -{ - - public static final long NO_END_SEQUENCE_NUMBER = Long.MAX_VALUE; - - @JsonCreator - public KafkaPartitions( - @JsonProperty("topic") final String topic, - @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap - ) - { - super( - topic, - partitionOffsetMap - ); - - } - - @Override - public Long getNoEndSequenceNumber() - { - return Long.MAX_VALUE; - } - - @JsonProperty - public String getTopic() - { - return getId(); - } - - @JsonProperty - public Map getPartitionOffsetMap() - { - return getPartitionSequenceMap(); - } - -} diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index af18b2338e4c..540c3eb00d46 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -26,27 +26,35 @@ import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Random; import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -// TODO: in the future, should refactor to do smt similar to KinesisRecordSupplier public class KafkaRecordSupplier implements RecordSupplier { + private static final EmittingLogger log = new EmittingLogger(KafkaRecordSupplier.class); private static final Random RANDOM = new Random(); private final KafkaConsumer consumer; private final KafkaSupervisorIOConfig ioConfig; private boolean closed; + private final BlockingQueue> records; public KafkaRecordSupplier( @@ -56,6 +64,7 @@ public KafkaRecordSupplier( this.ioConfig = ioConfig; this.consumer = getKafkaConsumer(); this.closed = false; + this.records = new LinkedBlockingQueue<>(); } @Override @@ -110,7 +119,25 @@ public Set> getAssignment() @Override public Record poll(long timeout) { - throw new UnsupportedOperationException(); + if (records.isEmpty()) { + ConsumerRecords polledRecords = consumer.poll(timeout); + for (ConsumerRecord record : polledRecords) { + records.offer(new Record<>( + record.topic(), + record.partition(), + record.offset(), + Arrays.asList(record.value()) + )); + } + } + + try { + return records.poll(timeout, TimeUnit.MILLISECONDS); + } + catch (InterruptedException e) { + log.warn(e, "InterruptedException"); + return null; + } } @Override diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 3b23a478b79e..f2fc35f8db0b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -48,6 +48,7 @@ import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -283,9 +284,14 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception nextOffsets.putAll(ioConfig.getStartPartitions().getPartitionOffsetMap()); } else { final Map restoredMetadataMap = (Map) restoredMetadata; - final KafkaPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( + final SeekableStreamPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( restoredMetadataMap.get(METADATA_NEXT_PARTITIONS), - KafkaPartitions.class + toolbox.getObjectMapper().getTypeFactory().constructParametrizedType( + SeekableStreamPartitions.class, + SeekableStreamPartitions.class, + Integer.class, + Long.class + ) ); nextOffsets.putAll(restoredNextPartitions.getPartitionOffsetMap()); @@ -327,7 +333,7 @@ public Committer get() public Object getMetadata() { return ImmutableMap.of( - METADATA_NEXT_PARTITIONS, new KafkaPartitions( + METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>( ioConfig.getStartPartitions().getTopic(), snapshot ) @@ -495,9 +501,9 @@ public void run() } final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { - final KafkaPartitions finalPartitions = toolbox.getObjectMapper().convertValue( + final SeekableStreamPartitions finalPartitions = toolbox.getObjectMapper().convertValue( ((Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_NEXT_PARTITIONS), - KafkaPartitions.class + SeekableStreamPartitions.class ); // Sanity check, we should only be publishing things that match our desired end state. @@ -716,7 +722,7 @@ private void sendResetRequestAndWait(Map outOfRangePartiti boolean result = taskToolbox.getTaskActionClient() .submit(new ResetDataSourceMetadataAction( task.getDataSource(), - new KafkaDataSourceMetadata(new KafkaPartitions( + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( ioConfig.getStartPartitions() .getTopic(), partitionOffsetMap diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index b092daa9cfec..337bb85f9bb2 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -33,7 +33,6 @@ import org.apache.druid.indexing.kafka.KafkaIOConfig; import org.apache.druid.indexing.kafka.KafkaIndexTask; import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; -import org.apache.druid.indexing.kafka.KafkaPartitions; import org.apache.druid.indexing.kafka.KafkaRecordSupplier; import org.apache.druid.indexing.kafka.KafkaSequenceNumber; import org.apache.druid.indexing.kafka.KafkaTuningConfig; @@ -43,6 +42,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.SequenceNumber; @@ -241,8 +241,8 @@ protected SeekableStreamIOConfig createIoConfig( return new KafkaIOConfig( groupId, baseSequenceName, - new KafkaPartitions(kafkaIoConfig.getTopic(), startPartitions), - new KafkaPartitions(kafkaIoConfig.getTopic(), endPartitions), + new SeekableStreamPartitions<>(kafkaIoConfig.getTopic(), startPartitions), + new SeekableStreamPartitions<>(kafkaIoConfig.getTopic(), endPartitions), kafkaIoConfig.getConsumerProperties(), true, minimumMessageTime, @@ -319,7 +319,7 @@ protected KafkaDataSourceMetadata createDataSourceMetaData( String topic, Map map ) { - return new KafkaDataSourceMetadata(new KafkaPartitions(topic, map)); + return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, map)); } @Override @@ -327,7 +327,7 @@ protected Map createNewTaskEndPartitions(Set startPartit { Map endPartitions = new HashMap<>(); for (int partition : startPartitions) { - endPartitions.put(partition, KafkaPartitions.NO_END_SEQUENCE_NUMBER); + endPartitions.put(partition, Long.MAX_VALUE); } return endPartitions; } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java index 51afdeb8ea40..89f5ce8b0e76 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.kafka; import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.junit.Assert; import org.junit.Test; @@ -125,6 +126,6 @@ public void testMinus() private static KafkaDataSourceMetadata KM(String topic, Map offsets) { - return new KafkaDataSourceMetadata(new KafkaPartitions(topic, offsets)); + return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, offsets)); } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java index 2321974ff383..fab3789d71a3 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java @@ -212,7 +212,7 @@ public void testStartAndEndTopicMatch() throws Exception exception.expect(JsonMappingException.class); exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class)); - exception.expectMessage(CoreMatchers.containsString("start topic and end topic must match")); + exception.expectMessage(CoreMatchers.containsString("start topic/stream and end topic/stream must match")); mapper.readValue(jsonStr, IOConfig.class); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index c231980b4bb6..9cc2e0e43211 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -80,6 +80,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; import org.apache.druid.indexing.test.TestDataSegmentKiller; import org.apache.druid.java.util.common.CompressionUtils; @@ -383,8 +384,8 @@ public void testRunAfterDataInserted() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -408,7 +409,7 @@ public void testRunAfterDataInserted() throws Exception SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -425,8 +426,8 @@ public void testRunBeforeDataInserted() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -462,7 +463,7 @@ public void testRunBeforeDataInserted() throws Exception SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -490,12 +491,12 @@ public void testIncrementalHandOff() throws Exception Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); - final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L)); + final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L, 1, 0L)); // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering // of events fetched across two partitions from Kafka - final KafkaPartitions checkpoint1 = new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 0L)); - final KafkaPartitions checkpoint2 = new KafkaPartitions(topic, ImmutableMap.of(0, 4L, 1, 2L)); - final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L)); + final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 0L)); + final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 4L, 1, 2L)); + final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L)); final KafkaIndexTask task = createTask( null, new KafkaIOConfig( @@ -527,7 +528,7 @@ public void testIncrementalHandOff() throws Exception DATA_SCHEMA.getDataSource(), 0, new KafkaDataSourceMetadata(startPartitions), - new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets)) + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, currentOffsets)) ) ) ); @@ -547,7 +548,7 @@ public void testIncrementalHandOff() throws Exception SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -585,11 +586,11 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); - final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L)); - final KafkaPartitions checkpoint1 = new KafkaPartitions(topic, ImmutableMap.of(0, 3L, 1, 0L)); - final KafkaPartitions checkpoint2 = new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 0L)); + final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L, 1, 0L)); + final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 3L, 1, 0L)); + final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 0L)); - final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L)); + final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L)); final KafkaIndexTask task = createTask( null, new KafkaIOConfig( @@ -635,7 +636,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception DATA_SCHEMA.getDataSource(), 0, new KafkaDataSourceMetadata(startPartitions), - new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets)) + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, currentOffsets)) ) ) ); @@ -644,8 +645,8 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Objects.hash( DATA_SCHEMA.getDataSource(), 0, - new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets)), - new KafkaDataSourceMetadata(new KafkaPartitions(topic, nextOffsets)) + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, currentOffsets)), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, nextOffsets)) ) ) ); @@ -665,7 +666,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 2L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -702,10 +703,10 @@ public void testTimeBasedIncrementalHandOff() throws Exception Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); - final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L)); + final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L, 1, 0L)); // Checkpointing will happen at checkpoint - final KafkaPartitions checkpoint = new KafkaPartitions(topic, ImmutableMap.of(0, 1L, 1, 0L)); - final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L)); + final SeekableStreamPartitions checkpoint = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 1L, 1, 0L)); + final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L, 1, 0L)); final KafkaIndexTask task = createTask( null, new KafkaIOConfig( @@ -738,7 +739,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception DATA_SCHEMA.getDataSource(), 0, new KafkaDataSourceMetadata(startPartitions), - new KafkaDataSourceMetadata(new KafkaPartitions(topic, checkpoint.getPartitionOffsetMap())) + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, checkpoint.getPartitionOffsetMap())) ) ) ); @@ -753,7 +754,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L, 1, 0L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -770,8 +771,8 @@ public void testRunWithMinimumMessageTime() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, DateTimes.of("2010"), @@ -807,7 +808,7 @@ public void testRunWithMinimumMessageTime() throws Exception SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -824,8 +825,8 @@ public void testRunWithMaximumMessageTime() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -862,7 +863,7 @@ public void testRunWithMaximumMessageTime() throws Exception SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -888,8 +889,8 @@ public void testRunWithTransformSpec() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -924,7 +925,7 @@ public void testRunWithTransformSpec() throws Exception SegmentDescriptor desc1 = SD(task, "2009/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -948,8 +949,8 @@ public void testRunOnNothing() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), kafkaServer.consumerProperties(), true, null, @@ -989,8 +990,8 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -1014,7 +1015,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1041,8 +1042,8 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -1066,7 +1067,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1096,8 +1097,8 @@ public void testReportParseExceptions() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 7L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 7L)), kafkaServer.consumerProperties(), true, null, @@ -1140,8 +1141,8 @@ public void testMultipleParseExceptionsSuccess() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 13L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L)), kafkaServer.consumerProperties(), true, null, @@ -1171,7 +1172,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception SegmentDescriptor desc4 = SD(task, "2049/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 13L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 13L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1222,8 +1223,8 @@ public void testMultipleParseExceptionsFailure() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 10L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L)), kafkaServer.consumerProperties(), true, null, @@ -1282,8 +1283,8 @@ public void testRunReplicas() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -1296,8 +1297,8 @@ public void testRunReplicas() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -1333,7 +1334,7 @@ public void testRunReplicas() throws Exception SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1350,8 +1351,8 @@ public void testRunConflicting() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -1364,8 +1365,8 @@ public void testRunConflicting() throws Exception new KafkaIOConfig( 1, "sequence1", - new KafkaPartitions(topic, ImmutableMap.of(0, 3L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 10L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 3L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L)), kafkaServer.consumerProperties(), true, null, @@ -1402,7 +1403,7 @@ public void testRunConflicting() throws Exception SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1419,8 +1420,8 @@ public void testRunConflictingWithoutTransactions() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), false, null, @@ -1433,8 +1434,8 @@ public void testRunConflictingWithoutTransactions() throws Exception new KafkaIOConfig( 1, "sequence1", - new KafkaPartitions(topic, ImmutableMap.of(0, 3L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 10L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 3L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L)), kafkaServer.consumerProperties(), false, null, @@ -1493,8 +1494,8 @@ public void testRunOneTaskTwoPartitions() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L, 1, 0L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 2L)), kafkaServer.consumerProperties(), true, null, @@ -1531,7 +1532,7 @@ public void testRunOneTaskTwoPartitions() throws Exception ? ImmutableSet.of(desc1, desc2, desc4) : ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 2L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 2L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1558,8 +1559,8 @@ public void testRunTwoTasksTwoPartitions() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -1572,8 +1573,8 @@ public void testRunTwoTasksTwoPartitions() throws Exception new KafkaIOConfig( 1, "sequence1", - new KafkaPartitions(topic, ImmutableMap.of(1, 0L)), - new KafkaPartitions(topic, ImmutableMap.of(1, 1L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(1, 0L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(1, 1L)), kafkaServer.consumerProperties(), true, null, @@ -1610,7 +1611,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception SegmentDescriptor desc3 = SD(task2, "2012/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 1L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 1L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1628,8 +1629,8 @@ public void testRestore() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -1665,8 +1666,8 @@ public void testRestore() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -1700,7 +1701,7 @@ public void testRestore() throws Exception SegmentDescriptor desc2 = SD(task1, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1717,8 +1718,8 @@ public void testRunWithPauseAndResume() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -1784,7 +1785,7 @@ public void testRunWithPauseAndResume() throws Exception SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1801,8 +1802,8 @@ public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 2L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -1840,8 +1841,8 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva new KafkaIOConfig( 0, "sequence0", - new KafkaPartitions(topic, ImmutableMap.of(0, 200L)), - new KafkaPartitions(topic, ImmutableMap.of(0, 500L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 200L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 500L)), kafkaServer.consumerProperties(), true, null, @@ -1894,8 +1895,8 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception 0, "sequence0", // 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, 5L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L)), kafkaServer.consumerProperties(), true, null, @@ -1920,7 +1921,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 259201fd17e9..8c7820dcf9c0 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -49,7 +49,6 @@ import org.apache.druid.indexing.kafka.KafkaIndexTask; import org.apache.druid.indexing.kafka.KafkaIndexTaskClient; import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; -import org.apache.druid.indexing.kafka.KafkaPartitions; import org.apache.druid.indexing.kafka.test.TestBroker; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -62,6 +61,7 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -534,7 +534,7 @@ public void testDatasourceMetadata() throws Exception expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( - new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) ) ).anyTimes(); expect(taskQueue.add(capture(captured))).andReturn(true); @@ -562,7 +562,7 @@ public void testBadMetadataOffsets() throws Exception expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( - new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) ) ).anyTimes(); replayAll(); @@ -582,8 +582,8 @@ public void testKillIncompatibleTasks() throws Exception "id1", DATASOURCE, 1, - new KafkaPartitions("topic", ImmutableMap.of(0, 0L)), - new KafkaPartitions("topic", ImmutableMap.of(0, 10L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L)), null, null ); @@ -593,8 +593,8 @@ public void testKillIncompatibleTasks() throws Exception "id2", DATASOURCE, 0, - new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), - new KafkaPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 333L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 333L)), null, null ); @@ -604,8 +604,8 @@ public void testKillIncompatibleTasks() throws Exception "id3", DATASOURCE, 1, - new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 1L)), - new KafkaPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 330L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 1L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 330L)), null, null ); @@ -615,8 +615,8 @@ public void testKillIncompatibleTasks() throws Exception "id4", "other-datasource", 2, - new KafkaPartitions("topic", ImmutableMap.of(0, 0L)), - new KafkaPartitions("topic", ImmutableMap.of(0, 10L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L)), null, null ); @@ -683,8 +683,8 @@ public void testKillBadPartitionAssignment() throws Exception "id1", DATASOURCE, 0, - new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)), - new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), null, null ); @@ -692,8 +692,8 @@ public void testKillBadPartitionAssignment() throws Exception "id2", DATASOURCE, 1, - new KafkaPartitions("topic", ImmutableMap.of(1, 0L)), - new KafkaPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, Long.MAX_VALUE)), null, null ); @@ -701,8 +701,11 @@ public void testKillBadPartitionAssignment() throws Exception "id3", 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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -710,8 +713,8 @@ public void testKillBadPartitionAssignment() throws Exception "id4", DATASOURCE, 0, - new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L)), - new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE)), null, null ); @@ -719,8 +722,8 @@ public void testKillBadPartitionAssignment() throws Exception "id5", DATASOURCE, 0, - new KafkaPartitions("topic", ImmutableMap.of(1, 0L, 2, 0L)), - new KafkaPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, 0L, 2, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), null, null ); @@ -867,8 +870,8 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception "id1", DATASOURCE, 0, - new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)), - new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), now, maxi ); @@ -1146,8 +1149,11 @@ public void testDiscoverExistingPublishingTask() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -1238,8 +1244,8 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() "id1", DATASOURCE, 0, - new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)), - new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 2, 0L)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)), null, null ); @@ -1328,8 +1334,11 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -1338,8 +1347,11 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception "id2", DATASOURCE, 0, - 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 SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -1676,8 +1688,11 @@ public void testStopGracefully() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -1686,8 +1701,11 @@ public void testStopGracefully() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -1696,8 +1714,11 @@ public void testStopGracefully() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -1802,17 +1823,17 @@ public void testResetDataSourceMetadata() throws Exception Capture captureDataSource = EasyMock.newCapture(); Capture captureDataSourceMetadata = EasyMock.newCapture(); - KafkaDataSourceMetadata kafkaDataSourceMetadata = new KafkaDataSourceMetadata(new KafkaPartitions( + KafkaDataSourceMetadata kafkaDataSourceMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( topic, ImmutableMap.of(0, 1000L, 1, 1000L, 2, 1000L) )); - KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(new KafkaPartitions( + KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( topic, ImmutableMap.of(1, 1000L, 2, 1000L) )); - KafkaDataSourceMetadata expectedMetadata = new KafkaDataSourceMetadata(new KafkaPartitions( + KafkaDataSourceMetadata expectedMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( topic, ImmutableMap.of(0, 1000L) )); @@ -1854,7 +1875,7 @@ public void testResetNoDataSourceMetadata() throws Exception supervisor.runInternal(); verifyAll(); - KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(new KafkaPartitions( + KafkaDataSourceMetadata resetMetadata = new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( topic, ImmutableMap.of(1, 1000L, 2, 1000L) )); @@ -1882,8 +1903,11 @@ public void testResetRunningTasks() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -1892,8 +1916,11 @@ public void testResetRunningTasks() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -1902,8 +1929,11 @@ public void testResetRunningTasks() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -1970,8 +2000,11 @@ public void testNoDataIngestionTasks() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -1980,8 +2013,11 @@ public void testNoDataIngestionTasks() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -1990,8 +2026,11 @@ public void testNoDataIngestionTasks() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -2057,8 +2096,8 @@ public void testCheckpointForInactiveTaskGroup() "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)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), null, null ); @@ -2067,8 +2106,8 @@ public void testCheckpointForInactiveTaskGroup() "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)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), null, null ); @@ -2077,8 +2116,8 @@ public void testCheckpointForInactiveTaskGroup() "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)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), null, null ); @@ -2135,8 +2174,8 @@ public void testCheckpointForInactiveTaskGroup() supervisor.checkpoint( 0, ((KafkaIndexTask) id1).getIOConfig().getBaseSequenceName(), - new KafkaDataSourceMetadata(new KafkaPartitions(topic, checkpoints.get(0))), - new KafkaDataSourceMetadata(new KafkaPartitions(topic, fakeCheckpoints)) + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, checkpoints.get(0))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, fakeCheckpoints)) ); while (supervisor.getNoticesQueueSize() > 0) { @@ -2159,8 +2198,8 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException "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)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), null, null ); @@ -2169,8 +2208,8 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException "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)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), null, null ); @@ -2179,8 +2218,8 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException "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)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)), null, null ); @@ -2205,8 +2244,8 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException supervisor.checkpoint( 0, ((KafkaIndexTask) id1).getIOConfig().getBaseSequenceName(), - new KafkaDataSourceMetadata(new KafkaPartitions(topic, Collections.emptyMap())), - new KafkaDataSourceMetadata(new KafkaPartitions(topic, Collections.emptyMap())) + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, Collections.emptyMap())), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, Collections.emptyMap())) ); while (supervisor.getNoticesQueueSize() > 0) { @@ -2238,8 +2277,8 @@ public void testCheckpointWithNullTaskGroupId() "id1", DATASOURCE, 0, - new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), - new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)), null, null ); @@ -2248,8 +2287,8 @@ public void testCheckpointWithNullTaskGroupId() "id2", DATASOURCE, 0, - new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), - new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)), null, null ); @@ -2258,8 +2297,8 @@ public void testCheckpointWithNullTaskGroupId() "id3", DATASOURCE, 0, - new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), - new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)), + new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, Long.MAX_VALUE)), null, null ); @@ -2304,8 +2343,8 @@ public void testCheckpointWithNullTaskGroupId() supervisor.checkpoint( null, ((KafkaIndexTask) id1).getIOConfig().getBaseSequenceName(), - new KafkaDataSourceMetadata(new KafkaPartitions(topic, checkpoints.get(0))), - new KafkaDataSourceMetadata(new KafkaPartitions(topic, newCheckpoints.get(0))) + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, checkpoints.get(0))), + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, newCheckpoints.get(0))) ); while (supervisor.getNoticesQueueSize() > 0) { @@ -2358,8 +2397,11 @@ public void testSuspendedRunningTasks() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -2368,8 +2410,11 @@ public void testSuspendedRunningTasks() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -2378,8 +2423,11 @@ public void testSuspendedRunningTasks() throws Exception "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)), + new SeekableStreamPartitions<>("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)), + new SeekableStreamPartitions<>( + "topic", + ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE) + ), null, null ); @@ -2751,8 +2799,8 @@ private KafkaIndexTask createKafkaIndexTask( String id, String dataSource, int taskGroupId, - KafkaPartitions startPartitions, - KafkaPartitions endPartitions, + SeekableStreamPartitions startPartitions, + SeekableStreamPartitions endPartitions, DateTime minimumMessageTime, DateTime maximumMessageTime ) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java index d5b884ac5b28..d972fa314cdd 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import java.util.Map; @@ -29,21 +30,21 @@ public class KinesisDataSourceMetadata extends SeekableStreamDataSourceMetadata< { @JsonCreator public KinesisDataSourceMetadata( - @JsonProperty("partitions") KinesisPartitions kinesisPartitions + @JsonProperty("partitions") SeekableStreamPartitions kinesisPartitions ) { super(kinesisPartitions); } @JsonProperty("partitions") - public KinesisPartitions getKinesisPartitions() + public SeekableStreamPartitions getKinesisPartitions() { - return (KinesisPartitions) super.getSeekableStreamPartitions(); + return getSeekableStreamPartitions(); } @Override protected KinesisDataSourceMetadata createConcretDataSourceMetaData(String streamName, Map newMap) { - return new KinesisDataSourceMetadata(new KinesisPartitions(streamName, newMap)); + return new KinesisDataSourceMetadata(new SeekableStreamPartitions(streamName, newMap)); } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java index af8895d17242..ee76bad55d5a 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.joda.time.DateTime; import java.util.Set; @@ -47,8 +48,8 @@ public class KinesisIOConfig extends SeekableStreamIOConfig @JsonCreator public KinesisIOConfig( @JsonProperty("baseSequenceName") String baseSequenceName, - @JsonProperty("startPartitions") KinesisPartitions startPartitions, - @JsonProperty("endPartitions") KinesisPartitions endPartitions, + @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, + @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, @JsonProperty("useTransaction") Boolean useTransaction, @JsonProperty("pauseAfterRead") Boolean pauseAfterRead, @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, @@ -149,16 +150,16 @@ public boolean isDeaggregate() @Override @JsonProperty - public KinesisPartitions getStartPartitions() + public SeekableStreamPartitions getStartPartitions() { - return (KinesisPartitions) super.getStartPartitions(); + return super.getStartPartitions(); } @Override @JsonProperty - public KinesisPartitions getEndPartitions() + public SeekableStreamPartitions getEndPartitions() { - return (KinesisPartitions) super.getEndPartitions(); + return super.getEndPartitions(); } @Override diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 4e48eb03b873..4b8791f10e70 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -56,6 +56,7 @@ import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.common.Record; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; @@ -222,7 +223,7 @@ public KinesisIndexTask( this.ingestionState = IngestionState.NOT_STARTED; this.ioConfig = ioConfig; this.tuningConfig = tuningConfig; - this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionSequenceMap()); + this.endOffsets.putAll(ioConfig.getEndPartitions().getMap()); } @Override @@ -281,14 +282,14 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception // Start up, set up initial offsets. final Object restoredMetadata = driver.startJob(); if (restoredMetadata == null) { - lastOffsets.putAll(ioConfig.getStartPartitions().getPartitionSequenceMap()); + lastOffsets.putAll(ioConfig.getStartPartitions().getMap()); } else { final Map restoredMetadataMap = (Map) restoredMetadata; - final KinesisPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( + final SeekableStreamPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( restoredMetadataMap.get(METADATA_NEXT_PARTITIONS), - KinesisPartitions.class + SeekableStreamPartitions.class ); - lastOffsets.putAll(restoredNextPartitions.getPartitionSequenceMap()); + lastOffsets.putAll(restoredNextPartitions.getMap()); // Sanity checks. if (!restoredNextPartitions.getId().equals(ioConfig.getStartPartitions().getId())) { @@ -299,11 +300,11 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ); } - if (!lastOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceMap().keySet())) { + if (!lastOffsets.keySet().equals(ioConfig.getStartPartitions().getMap().keySet())) { throw new ISE( "WTF?! Restored partitions[%s] but expected partitions[%s]", lastOffsets.keySet(), - ioConfig.getStartPartitions().getPartitionSequenceMap().keySet() + ioConfig.getStartPartitions().getMap().keySet() ); } } @@ -338,7 +339,7 @@ public Committer get() public Object getMetadata() { return ImmutableMap.of( - METADATA_NEXT_PARTITIONS, new KinesisPartitions( + METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>( ioConfig.getStartPartitions().getId(), snapshot ) @@ -429,7 +430,7 @@ public void run() if (Record.END_OF_SHARD_MARKER.equals(record.getSequenceNumber())) { lastOffsets.put(record.getPartitionId(), record.getSequenceNumber()); - } else if (KinesisPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffsets.get(record.getPartitionId())) + } else if (SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffsets.get(record.getPartitionId())) || record.getSequenceNumber().compareTo(endOffsets.get(record.getPartitionId())) <= 0) { try { @@ -533,13 +534,13 @@ public void run() } final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { - final KinesisPartitions finalPartitions = toolbox.getObjectMapper().convertValue( + final SeekableStreamPartitions finalPartitions = toolbox.getObjectMapper().convertValue( ((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS), - KinesisPartitions.class + SeekableStreamPartitions.class ); // Sanity check, we should only be publishing things that match our desired end state. - if (!endOffsets.equals(finalPartitions.getPartitionSequenceMap())) { + if (!endOffsets.equals(finalPartitions.getMap())) { throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata); } @@ -1067,7 +1068,7 @@ private boolean isPaused() private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) { final int maxRowsInMemoryPerPartition = (tuningConfig.getMaxRowsInMemory() / - ioConfig.getStartPartitions().getPartitionSequenceMap().size()); + ioConfig.getStartPartitions().getMap().size()); return Appenderators.createRealtime( dataSchema, tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), @@ -1119,7 +1120,7 @@ private RecordSupplier getRecordSupplier() { int fetchThreads = tuningConfig.getFetchThreads() != null ? tuningConfig.getFetchThreads() - : Math.max(1, ioConfig.getStartPartitions().getPartitionSequenceMap().size()); + : Math.max(1, ioConfig.getStartPartitions().getMap().size()); return new KinesisRecordSupplier( ioConfig.getEndpoint(), @@ -1154,7 +1155,7 @@ private Set assignPartitions(RecordSupplier recordSupplier, String topic for (Map.Entry entry : lastOffsets.entrySet()) { final String endOffset = endOffsets.get(entry.getKey()); if (Record.END_OF_SHARD_MARKER.equals(endOffset) - || KinesisPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) + || SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) || KinesisSequenceNumber.of(entry.getValue()).compareTo(KinesisSequenceNumber.of(endOffset)) < 0) { assignment.add(entry.getKey()); } else if (entry.getValue().equals(endOffset)) { @@ -1302,7 +1303,7 @@ private void sendResetRequestAndWait( new ResetDataSourceMetadataAction( getDataSource(), new KinesisDataSourceMetadata( - new KinesisPartitions(ioConfig.getStartPartitions().getId(), partitionOffsetMap) + new SeekableStreamPartitions(ioConfig.getStartPartitions().getId(), partitionOffsetMap) ) ) ); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java index 8b0e3b5bcd80..f73a77aef2de 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java @@ -21,72 +21,16 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Charsets; -import com.google.common.base.Optional; -import com.google.common.base.Throwables; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.RetryPolicy; -import org.apache.druid.indexing.common.RetryPolicyConfig; -import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.java.util.http.client.Request; -import org.apache.druid.java.util.http.client.response.FullResponseHandler; -import org.apache.druid.java.util.http.client.response.FullResponseHolder; -import org.apache.druid.segment.realtime.firehose.ChatHandlerResource; -import org.jboss.netty.channel.ChannelException; -import org.jboss.netty.handler.codec.http.HttpMethod; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.Duration; -import org.joda.time.Period; -import javax.ws.rs.core.MediaType; -import java.io.IOException; -import java.net.Socket; -import java.net.URI; import java.util.Map; public class KinesisIndexTaskClient extends SeekableStreamIndexTaskClient { - public static class NoTaskLocationException extends RuntimeException - { - public NoTaskLocationException(String message) - { - super(message); - } - } - - public static class TaskNotRunnableException extends RuntimeException - { - public TaskNotRunnableException(String message) - { - super(message); - } - } - private static ObjectMapper mapper = new ObjectMapper(); - public static final int MAX_RETRY_WAIT_SECONDS = 10; - private static final int MIN_RETRY_WAIT_SECONDS = 2; - private static final EmittingLogger log = new EmittingLogger(KinesisIndexTaskClient.class); - private static final String BASE_PATH = "/druid/worker/v1/chat"; - private static final int TASK_MISMATCH_RETRY_DELAY_SECONDS = 5; - - private final HttpClient httpClient; - private final ObjectMapper jsonMapper; - private final TaskInfoProvider taskInfoProvider; - private final Duration httpTimeout; - private final RetryPolicyFactory retryPolicyFactory; - private final ListeningExecutorService executorService; - private final long numRetries; public KinesisIndexTaskClient( HttpClient httpClient, @@ -107,22 +51,6 @@ public KinesisIndexTaskClient( httpTimeout, numRetries ); - this.httpClient = httpClient; - this.jsonMapper = jsonMapper; - this.taskInfoProvider = taskInfoProvider; - this.httpTimeout = httpTimeout; - this.numRetries = numRetries; - this.retryPolicyFactory = createRetryPolicyFactory(); - - this.executorService = MoreExecutors.listeningDecorator( - Execs.multiThreaded( - numThreads, - String.format( - "KinesisIndexTaskClient-%s-%%d", - dataSource - ) - ) - ); } @Override @@ -130,174 +58,5 @@ protected JavaType constructMapType(Class mapType) { return mapper.getTypeFactory().constructMapType(mapType, String.class, String.class); } - - @Override - public void close() - { - executorService.shutdownNow(); - } - - - @VisibleForTesting - RetryPolicyFactory createRetryPolicyFactory() - { - // Retries [numRetries] times before giving up; this should be set long enough to handle any temporary - // unresponsiveness such as network issues, if a task is still in the process of starting up, or if the task is in - // the middle of persisting to disk and doesn't respond immediately. - return new RetryPolicyFactory( - new RetryPolicyConfig() - .setMinWait(Period.seconds(MIN_RETRY_WAIT_SECONDS)) - .setMaxWait(Period.seconds(MAX_RETRY_WAIT_SECONDS)) - .setMaxRetryCount(numRetries) - ); - } - - @Override - @VisibleForTesting - protected void checkConnection(String host, int port) throws IOException - { - new Socket(host, port).close(); - } - - - private FullResponseHolder submitRequest(String id, HttpMethod method, String pathSuffix, String query, boolean retry) - { - return submitRequest(id, method, pathSuffix, query, new byte[0], retry); - } - - private FullResponseHolder submitRequest( - String id, - HttpMethod method, - String pathSuffix, - String query, - byte[] content, - boolean retry - ) - { - final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy(); - while (true) { - FullResponseHolder response = null; - Request request = null; - TaskLocation location = TaskLocation.unknown(); - String path = String.format("%s/%s/%s", BASE_PATH, id, pathSuffix); - - Optional status = taskInfoProvider.getTaskStatus(id); - if (!status.isPresent() || !status.get().isRunnable()) { - throw new TaskNotRunnableException(String.format("Aborting request because task [%s] is not runnable", id)); - } - - String host = location.getHost(); - String scheme = ""; - int port = -1; - - try { - location = taskInfoProvider.getTaskLocation(id); - if (location.equals(TaskLocation.unknown())) { - throw new NoTaskLocationException(String.format("No TaskLocation available for task [%s]", id)); - } - - host = location.getHost(); - scheme = location.getTlsPort() >= 0 ? "https" : "http"; - port = location.getTlsPort() >= 0 ? location.getTlsPort() : location.getPort(); - - // Netty throws some annoying exceptions if a connection can't be opened, which happens relatively frequently - // for tasks that happen to still be starting up, so test the connection first to keep the logs clean. - checkConnection(host, port); - - try { - URI serviceUri = new URI(scheme, null, host, port, path, query, null); - request = new Request(method, serviceUri.toURL()); - - // used to validate that we are talking to the correct worker - request.addHeader(ChatHandlerResource.TASK_ID_HEADER, id); - - if (content.length > 0) { - request.setContent(MediaType.APPLICATION_JSON, content); - } - - log.debug("HTTP %s: %s", method.getName(), serviceUri.toString()); - response = httpClient.go(request, new FullResponseHandler(Charsets.UTF_8), httpTimeout).get(); - } - catch (Exception e) { - Throwables.propagateIfInstanceOf(e.getCause(), IOException.class); - Throwables.propagateIfInstanceOf(e.getCause(), ChannelException.class); - throw Throwables.propagate(e); - } - - int responseCode = response.getStatus().getCode(); - if (responseCode / 100 == 2) { - return response; - } else if (responseCode == 400) { // don't bother retrying if it's a bad request - throw new IAE("Received 400 Bad Request with body: %s", response.getContent()); - } else { - throw new IOException(String.format("Received status [%d]", responseCode)); - } - } - catch (IOException | ChannelException e) { - - // Since workers are free to move tasks around to different ports, there is a chance that a task may have been - // moved but our view of its location has not been updated yet from ZK. To detect this case, we send a header - // identifying our expected recipient in the request; if this doesn't correspond to the worker we messaged, the - // worker will return an HTTP 404 with its ID in the response header. If we get a mismatching task ID, then - // we will wait for a short period then retry the request indefinitely, expecting the task's location to - // eventually be updated. - - final Duration delay; - if (response != null && response.getStatus().equals(HttpResponseStatus.NOT_FOUND)) { - String headerId = response.getResponse().headers().get(ChatHandlerResource.TASK_ID_HEADER); - if (headerId != null && !headerId.equals(id)) { - log.warn( - "Expected worker to have taskId [%s] but has taskId [%s], will retry in [%d]s", - id, headerId, TASK_MISMATCH_RETRY_DELAY_SECONDS - ); - delay = Duration.standardSeconds(TASK_MISMATCH_RETRY_DELAY_SECONDS); - } else { - delay = retryPolicy.getAndIncrementRetryDelay(); - } - } else { - delay = retryPolicy.getAndIncrementRetryDelay(); - } - - String urlForLog = (request != null - ? request.getUrl().toString() - : StringUtils.format("%s://%s:%d%s", scheme, host, port, path)); - - if (!retry) { - // if retry=false, we probably aren't too concerned if the operation doesn't succeed (i.e. the request was - // for informational purposes only) so don't log a scary stack trace - log.info("submitRequest failed for [%s], with message [%s]", urlForLog, e.getMessage()); - Throwables.propagate(e); - } else if (delay == null) { - log.warn(e, "Retries exhausted for [%s], last exception:", urlForLog); - Throwables.propagate(e); - } else { - try { - final long sleepTime = delay.getMillis(); - log.debug( - "Bad response HTTP [%s] from [%s]; will try again in [%s] (body/exception: [%s])", - (response != null ? response.getStatus().getCode() : "no response"), - urlForLog, - new Duration(sleepTime).toString(), - (response != null ? response.getContent() : e.getMessage()) - ); - Thread.sleep(sleepTime); - } - catch (InterruptedException e2) { - Throwables.propagate(e2); - } - } - } - catch (NoTaskLocationException e) { - log.info( - "No TaskLocation available for task [%s], this task may not have been assigned to a worker yet or " - + "may have already completed", id - ); - throw e; - } - catch (Exception e) { - log.warn(e, "Exception while sending request"); - throw e; - } - } - } } + diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java deleted file mode 100644 index 9f91edb394ea..000000000000 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisPartitions.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.kinesis; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; - -import java.util.Map; - -public class KinesisPartitions extends SeekableStreamPartitions -{ - public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER"; - - @JsonCreator - public KinesisPartitions( - @JsonProperty("stream") final String stream, - @JsonProperty("partitionSequenceNumberMap") final Map partitionSequenceNumberMap - ) - { - super( - stream, - partitionSequenceNumberMap - ); - } - - @Override - public String getNoEndSequenceNumber() - { - return NO_END_SEQUENCE_NUMBER; - } - - @JsonProperty - public String getStream() - { - return getId(); - } - - @JsonProperty - public Map getPartitionSequenceNumberMap() - { - return getPartitionSequenceMap(); - } - -} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 902fff206299..870258a820d1 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -31,7 +31,6 @@ import org.apache.druid.indexing.kinesis.KinesisIOConfig; import org.apache.druid.indexing.kinesis.KinesisIndexTask; import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; -import org.apache.druid.indexing.kinesis.KinesisPartitions; import org.apache.druid.indexing.kinesis.KinesisRecordSupplier; import org.apache.druid.indexing.kinesis.KinesisSequenceNumber; import org.apache.druid.indexing.kinesis.KinesisTuningConfig; @@ -42,6 +41,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.SequenceNumber; @@ -132,8 +132,8 @@ protected SeekableStreamIOConfig createIoConfig( KinesisSupervisorIOConfig ioConfig = (KinesisSupervisorIOConfig) ioConfigg; return new KinesisIOConfig( baseSequenceName, - new KinesisPartitions(ioConfig.getStream(), startPartitions), - new KinesisPartitions(ioConfig.getStream(), endPartitions), + new SeekableStreamPartitions(ioConfig.getStream(), startPartitions), + new SeekableStreamPartitions(ioConfig.getStream(), endPartitions), true, true, // should pause after reading otherwise the task may complete early which will confuse the supervisor minimumMessageTime, @@ -267,7 +267,7 @@ protected SeekableStreamDataSourceMetadata createDataSourceMetaD ) { return new KinesisDataSourceMetadata( - new KinesisPartitions(stream, map) + new SeekableStreamPartitions<>(stream, map) ); } @@ -276,7 +276,7 @@ protected Map createNewTaskEndPartitions(Set startPartit { Map endPartitions = new HashMap<>(); for (String partition : startPartitions) { - endPartitions.put(partition, KinesisPartitions.NO_END_SEQUENCE_NUMBER); + endPartitions.put(partition, SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER); } return endPartitions; } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java index 506889b59428..70862cb7fae3 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java @@ -21,6 +21,7 @@ package org.apache.druid.indexing.kinesis; import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.junit.Assert; import org.junit.Test; @@ -126,6 +127,6 @@ public void testMinus() private static KinesisDataSourceMetadata KM(String stream, Map offsets) { - return new KinesisDataSourceMetadata(new KinesisPartitions(stream, offsets)); + return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, offsets)); } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index f12bb52ebb93..8b09a71d5c76 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -70,12 +70,12 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals("mystream", config.getStartPartitions().getStream()); Assert.assertEquals( ImmutableMap.of("0", "1", "1", "10"), - config.getStartPartitions().getPartitionSequenceNumberMap() + config.getStartPartitions().getMap() ); Assert.assertEquals("mystream", config.getEndPartitions().getStream()); Assert.assertEquals( ImmutableMap.of("0", "15", "1", "200"), - config.getEndPartitions().getPartitionSequenceNumberMap() + config.getEndPartitions().getMap() ); Assert.assertTrue(config.isUseTransaction()); Assert.assertTrue(config.isPauseAfterRead()); @@ -127,12 +127,12 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals("mystream", config.getStartPartitions().getStream()); Assert.assertEquals( ImmutableMap.of("0", "1", "1", "10"), - config.getStartPartitions().getPartitionSequenceNumberMap() + config.getStartPartitions().getMap() ); Assert.assertEquals("mystream", config.getEndPartitions().getStream()); Assert.assertEquals( ImmutableMap.of("0", "15", "1", "200"), - config.getEndPartitions().getPartitionSequenceNumberMap() + config.getEndPartitions().getMap() ); Assert.assertFalse(config.isUseTransaction()); Assert.assertFalse(config.isPauseAfterRead()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 2c4cb226867a..e656877e10e6 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -85,6 +85,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.common.Record; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; @@ -423,11 +424,11 @@ public void testRunAfterDataInserted() throws Exception null, new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 4) )), @@ -463,7 +464,7 @@ public void testRunAfterDataInserted() throws Exception SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( - new KinesisDataSourceMetadata(new KinesisPartitions( + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, @@ -501,11 +502,11 @@ public void testRunBeforeDataInserted() throws Exception null, new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, Record.END_OF_SHARD_MARKER )), @@ -554,7 +555,7 @@ public void testRunBeforeDataInserted() throws Exception Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); Assert.assertEquals( new KinesisDataSourceMetadata( - new KinesisPartitions( + new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId0, @@ -581,11 +582,11 @@ public void testRunWithMinimumMessageTime() throws Exception null, new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 4) )), @@ -626,7 +627,7 @@ public void testRunWithMinimumMessageTime() throws Exception Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( new KinesisDataSourceMetadata( - new KinesisPartitions( + new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, @@ -651,11 +652,11 @@ public void testRunWithMaximumMessageTime() throws Exception null, new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 4) )), @@ -697,7 +698,7 @@ public void testRunWithMaximumMessageTime() throws Exception Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); Assert.assertEquals( new KinesisDataSourceMetadata( - new KinesisPartitions( + new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, @@ -731,11 +732,11 @@ public void testRunWithTransformSpec() throws Exception ), new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 4) )), @@ -775,7 +776,7 @@ public void testRunWithTransformSpec() throws Exception Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors()); Assert.assertEquals( new KinesisDataSourceMetadata( - new KinesisPartitions( + new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, @@ -800,11 +801,11 @@ public void testRunOnNothing() throws Exception null, new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2) )), @@ -854,11 +855,11 @@ public void testReportParseExceptions() throws Exception null, new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 6) )), @@ -908,11 +909,11 @@ public void testMultipleParseExceptionsSuccess() throws Exception null, new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 12) )), @@ -953,7 +954,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals( new KinesisDataSourceMetadata( - new KinesisPartitions( + new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, @@ -1007,11 +1008,11 @@ public void testMultipleParseExceptionsFailure() throws Exception null, new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 9) )), @@ -1084,11 +1085,11 @@ public void testRunReplicas() throws Exception null, new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 4) )), @@ -1111,11 +1112,11 @@ public void testRunReplicas() throws Exception null, new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 4) )), @@ -1156,7 +1157,7 @@ public void testRunReplicas() throws Exception Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( new KinesisDataSourceMetadata( - new KinesisPartitions( + new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, @@ -1182,11 +1183,11 @@ public void testRunConflicting() throws Exception null, new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 4) )), @@ -1209,11 +1210,11 @@ public void testRunConflicting() throws Exception null, new KinesisIOConfig( "sequence1", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 9) )), @@ -1255,7 +1256,7 @@ public void testRunConflicting() throws Exception Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( new KinesisDataSourceMetadata( - new KinesisPartitions( + new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, @@ -1281,11 +1282,11 @@ public void testRunConflictingWithoutTransactions() throws Exception null, new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 4) )), @@ -1308,11 +1309,11 @@ public void testRunConflictingWithoutTransactions() throws Exception null, new KinesisIOConfig( "sequence1", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 9) )), @@ -1378,13 +1379,13 @@ public void testRunOneTaskTwoPartitions() throws Exception null, new KinesisIOConfig( "sequence1", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2), shardId0, getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 4), shardId0, @@ -1427,7 +1428,7 @@ public void testRunOneTaskTwoPartitions() throws Exception ? ImmutableSet.of(desc1, desc2, desc4) : ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals( - new KinesisDataSourceMetadata(new KinesisPartitions(stream, ImmutableMap.of( + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 4), shardId0, @@ -1462,11 +1463,11 @@ public void testRunTwoTasksTwoPartitions() throws Exception null, new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 4) )), @@ -1489,11 +1490,11 @@ public void testRunTwoTasksTwoPartitions() throws Exception null, new KinesisIOConfig( "sequence1", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 1) )), @@ -1537,7 +1538,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); Assert.assertEquals( new KinesisDataSourceMetadata( - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 4), shardId0, @@ -1571,11 +1572,11 @@ public void testRestore() throws Exception "task1", new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, Record.END_OF_SHARD_MARKER )), @@ -1616,11 +1617,11 @@ public void testRestore() throws Exception task1.getId(), new KinesisIOConfig( "sequence0", - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, Record.END_OF_SHARD_MARKER )), @@ -1666,7 +1667,7 @@ public void testRestore() throws Exception Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( new KinesisDataSourceMetadata( - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, Record.END_OF_SHARD_MARKER ))), diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 9b6d7c09e0dc..d13b4bf23e3a 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -53,7 +53,6 @@ import org.apache.druid.indexing.kinesis.KinesisIndexTask; import org.apache.druid.indexing.kinesis.KinesisIndexTaskClient; import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; -import org.apache.druid.indexing.kinesis.KinesisPartitions; import org.apache.druid.indexing.kinesis.KinesisSequenceNumber; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -65,6 +64,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -324,21 +324,21 @@ public void testNoInitialState() throws Exception Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); Assert.assertEquals( getSequenceNumber(res, shardId1, 0), - taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + taskConfig.getStartPartitions().getMap().get(shardId1) ); Assert.assertEquals( getSequenceNumber(res, shardId0, 0), - taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + taskConfig.getStartPartitions().getMap().get(shardId0) ); Assert.assertEquals(stream, taskConfig.getEndPartitions().getStream()); Assert.assertEquals( - KinesisPartitions.NO_END_SEQUENCE_NUMBER, - taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + taskConfig.getEndPartitions().getMap().get(shardId1) ); Assert.assertEquals( - KinesisPartitions.NO_END_SEQUENCE_NUMBER, - taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + taskConfig.getEndPartitions().getMap().get(shardId0) ); } @@ -367,27 +367,27 @@ public void testMultiTask() throws Exception verifyAll(); KinesisIndexTask task1 = captured.getValues().get(0); - Assert.assertEquals(1, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); - Assert.assertEquals(1, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(1, task1.getIOConfig().getStartPartitions().getMap().size()); + Assert.assertEquals(1, task1.getIOConfig().getEndPartitions().getMap().size()); Assert.assertEquals( getSequenceNumber(res, shardId1, 0), - task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + task1.getIOConfig().getStartPartitions().getMap().get(shardId1) ); Assert.assertEquals( - KinesisPartitions.NO_END_SEQUENCE_NUMBER, - task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + task1.getIOConfig().getEndPartitions().getMap().get(shardId1) ); KinesisIndexTask task2 = captured.getValues().get(1); - Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); - Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getMap().size()); + Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getMap().size()); Assert.assertEquals( getSequenceNumber(res, shardId0, 0), - task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + task2.getIOConfig().getStartPartitions().getMap().get(shardId0) ); Assert.assertEquals( - KinesisPartitions.NO_END_SEQUENCE_NUMBER, - task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + task2.getIOConfig().getEndPartitions().getMap().get(shardId0) ); } @@ -415,43 +415,43 @@ public void testReplicas() throws Exception verifyAll(); KinesisIndexTask task1 = captured.getValues().get(0); - Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); - Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getMap().size()); + Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getMap().size()); Assert.assertEquals( getSequenceNumber(res, shardId0, 0), - task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + task1.getIOConfig().getStartPartitions().getMap().get(shardId0) ); Assert.assertEquals( - KinesisPartitions.NO_END_SEQUENCE_NUMBER, - task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + task1.getIOConfig().getEndPartitions().getMap().get(shardId0) ); Assert.assertEquals( getSequenceNumber(res, shardId1, 0), - task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + task1.getIOConfig().getStartPartitions().getMap().get(shardId1) ); Assert.assertEquals( - KinesisPartitions.NO_END_SEQUENCE_NUMBER, - task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + task1.getIOConfig().getEndPartitions().getMap().get(shardId1) ); KinesisIndexTask task2 = captured.getValues().get(1); - Assert.assertEquals(2, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); - Assert.assertEquals(2, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(2, task2.getIOConfig().getStartPartitions().getMap().size()); + Assert.assertEquals(2, task2.getIOConfig().getEndPartitions().getMap().size()); Assert.assertEquals( getSequenceNumber(res, shardId0, 0), - task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + task2.getIOConfig().getStartPartitions().getMap().get(shardId0) ); Assert.assertEquals( - KinesisPartitions.NO_END_SEQUENCE_NUMBER, - task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + task2.getIOConfig().getEndPartitions().getMap().get(shardId0) ); Assert.assertEquals( getSequenceNumber(res, shardId1, 0), - task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + task2.getIOConfig().getStartPartitions().getMap().get(shardId1) ); Assert.assertEquals( - KinesisPartitions.NO_END_SEQUENCE_NUMBER, - task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + task2.getIOConfig().getEndPartitions().getMap().get(shardId1) ); } @@ -554,7 +554,7 @@ public void testDatasourceMetadata() throws Exception expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2), shardId0, @@ -575,11 +575,11 @@ public void testDatasourceMetadata() throws Exception Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); Assert.assertEquals( getSequenceNumber(res, shardId1, 2), - taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + taskConfig.getStartPartitions().getMap().get(shardId1) ); Assert.assertEquals( getSequenceNumber(res, shardId0, 1), - taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + taskConfig.getStartPartitions().getMap().get(shardId0) ); } @@ -595,7 +595,7 @@ public void testBadMetadataOffsets() throws Exception expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, "00000000000000000000000000000000000000000000000000000000", shardId0, @@ -621,11 +621,11 @@ public void testKillIncompatibleTasks() throws Exception "id1", DATASOURCE, 1, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 1) )), @@ -638,13 +638,13 @@ public void testKillIncompatibleTasks() throws Exception "id2", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 0), shardId1, getSequenceNumber(res, shardId1, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 1), shardId1, @@ -659,13 +659,13 @@ public void testKillIncompatibleTasks() throws Exception "id3", DATASOURCE, 1, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 0), shardId1, getSequenceNumber(res, shardId1, 1) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 1), shardId1, @@ -680,13 +680,13 @@ public void testKillIncompatibleTasks() throws Exception "id4", "other-datasource", 2, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 0), shardId1, getSequenceNumber(res, shardId1, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 1), shardId1, @@ -765,11 +765,11 @@ public void testKillBadPartitionAssignment() throws Exception "id1", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 12) )), @@ -780,11 +780,11 @@ public void testKillBadPartitionAssignment() throws Exception "id2", DATASOURCE, 1, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 1) )), @@ -795,13 +795,13 @@ public void testKillBadPartitionAssignment() throws Exception "id3", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 0), shardId1, getSequenceNumber(res, shardId1, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 1), shardId1, @@ -814,11 +814,11 @@ public void testKillBadPartitionAssignment() throws Exception "id4", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 1) )), @@ -829,11 +829,11 @@ public void testKillBadPartitionAssignment() throws Exception "id5", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, getSequenceNumber(res, shardId0, 1) )), @@ -994,17 +994,17 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception "id1", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 0), shardId0, getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), now, maxi @@ -1306,11 +1306,11 @@ public void testBeginPublishAndQueueNextTasks() throws Exception Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); Assert.assertEquals( getSequenceNumber(res, shardId1, 3), - taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + taskConfig.getStartPartitions().getMap().get(shardId1) ); Assert.assertEquals( getSequenceNumber(res, shardId0, 1), - taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + taskConfig.getStartPartitions().getMap().get(shardId0) ); } } @@ -1328,17 +1328,17 @@ public void testDiscoverExistingPublishingTask() throws Exception "id1", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 0), shardId0, getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -1436,21 +1436,21 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); Assert.assertEquals( getSequenceNumber(res, shardId1, 2), - capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + capturedTaskConfig.getStartPartitions().getMap().get(shardId1) ); Assert.assertEquals( getSequenceNumber(res, shardId0, 1), - capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + capturedTaskConfig.getStartPartitions().getMap().get(shardId0) ); Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream()); Assert.assertEquals( - KinesisPartitions.NO_END_SEQUENCE_NUMBER, - capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + capturedTaskConfig.getEndPartitions().getMap().get(shardId1) ); Assert.assertEquals( - KinesisPartitions.NO_END_SEQUENCE_NUMBER, - capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + capturedTaskConfig.getEndPartitions().getMap().get(shardId0) ); } @@ -1467,17 +1467,17 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() "id1", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 0), shardId0, getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -1566,21 +1566,21 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); Assert.assertEquals( getSequenceNumber(res, shardId1, 2), - capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + capturedTaskConfig.getStartPartitions().getMap().get(shardId1) ); Assert.assertEquals( getSequenceNumber(res, shardId0, 1), - capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + capturedTaskConfig.getStartPartitions().getMap().get(shardId0) ); Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream()); Assert.assertEquals( - KinesisPartitions.NO_END_SEQUENCE_NUMBER, - capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + capturedTaskConfig.getEndPartitions().getMap().get(shardId1) ); Assert.assertEquals( - KinesisPartitions.NO_END_SEQUENCE_NUMBER, - capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + capturedTaskConfig.getEndPartitions().getMap().get(shardId0) ); } @@ -1599,7 +1599,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception "id1", DATASOURCE, 0, - new KinesisPartitions( + new SeekableStreamPartitions<>( "stream", ImmutableMap.of( shardId1, @@ -1608,11 +1608,11 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception getSequenceNumber(res, shardId0, 0) ) ), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -1622,17 +1622,17 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception "id2", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2), shardId0, getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -1890,10 +1890,10 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); Assert.assertEquals( getSequenceNumber(res, shardId1, 0), - taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + taskConfig.getStartPartitions().getMap().get(shardId1) ); Assert.assertNull( - taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) + taskConfig.getStartPartitions().getMap().get(shardId0) ); } } @@ -1995,7 +1995,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); Assert.assertEquals( getSequenceNumber(res, shardId1, 0), - taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) + taskConfig.getStartPartitions().getMap().get(shardId1) ); } } @@ -2037,17 +2037,17 @@ public void testStopGracefully() throws Exception "id1", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 0), shardId0, getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2057,17 +2057,17 @@ public void testStopGracefully() throws Exception "id2", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), shardId0, getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2077,17 +2077,17 @@ public void testStopGracefully() throws Exception "id3", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), shardId0, getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2214,29 +2214,29 @@ public void testResetDataSourceMetadata() throws Exception Capture captureDataSource = EasyMock.newCapture(); Capture captureDataSourceMetadata = EasyMock.newCapture(); - KinesisDataSourceMetadata KinesisDataSourceMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( + KinesisDataSourceMetadata KinesisDataSourceMetadata = new KinesisDataSourceMetadata(new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER ) )); - KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( + KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER ) )); - KinesisDataSourceMetadata expectedMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( + KinesisDataSourceMetadata expectedMetadata = new KinesisDataSourceMetadata(new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER ) )); @@ -2277,11 +2277,11 @@ public void testResetNoDataSourceMetadata() throws Exception supervisor.runInternal(); verifyAll(); - KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new KinesisPartitions( + KinesisDataSourceMetadata resetMetadata = new KinesisDataSourceMetadata(new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER ) )); @@ -2309,7 +2309,7 @@ public void testResetRunningTasks() throws Exception "id1", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 0), @@ -2318,11 +2318,11 @@ public void testResetRunningTasks() throws Exception getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2332,7 +2332,7 @@ public void testResetRunningTasks() throws Exception "id2", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), @@ -2341,11 +2341,11 @@ public void testResetRunningTasks() throws Exception getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2355,7 +2355,7 @@ public void testResetRunningTasks() throws Exception "id3", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), @@ -2364,11 +2364,11 @@ public void testResetRunningTasks() throws Exception getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2454,7 +2454,7 @@ public void testNoDataIngestionTasks() throws Exception "id1", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 0), @@ -2463,11 +2463,11 @@ public void testNoDataIngestionTasks() throws Exception getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2477,7 +2477,7 @@ public void testNoDataIngestionTasks() throws Exception "id2", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), @@ -2486,11 +2486,11 @@ public void testNoDataIngestionTasks() throws Exception getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2500,7 +2500,7 @@ public void testNoDataIngestionTasks() throws Exception "id3", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), @@ -2509,11 +2509,11 @@ public void testNoDataIngestionTasks() throws Exception getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2591,7 +2591,7 @@ public void testCheckpointForInactiveTaskGroup() "id1", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 0), @@ -2600,11 +2600,11 @@ public void testCheckpointForInactiveTaskGroup() getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2614,7 +2614,7 @@ public void testCheckpointForInactiveTaskGroup() "id2", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), @@ -2623,11 +2623,11 @@ public void testCheckpointForInactiveTaskGroup() getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2637,7 +2637,7 @@ public void testCheckpointForInactiveTaskGroup() "id3", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), @@ -2646,11 +2646,11 @@ public void testCheckpointForInactiveTaskGroup() getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2717,8 +2717,8 @@ public void testCheckpointForInactiveTaskGroup() supervisor.checkpoint( 0, ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), - new KinesisDataSourceMetadata(new KinesisPartitions(stream, checkpoints.get(0))), - new KinesisDataSourceMetadata(new KinesisPartitions(stream, fakeCheckpoints)) + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, checkpoints.get(0))), + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, fakeCheckpoints)) ); while (supervisor.getNoticesQueueSize() > 0) { @@ -2742,7 +2742,7 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException "id1", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 0), @@ -2751,11 +2751,11 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2765,7 +2765,7 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException "id2", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), @@ -2774,11 +2774,11 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2788,7 +2788,7 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException "id3", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), @@ -2797,11 +2797,11 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions(stream, ImmutableMap.of( + new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -2828,8 +2828,8 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException supervisor.checkpoint( 0, ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), - new KinesisDataSourceMetadata(new KinesisPartitions(stream, Collections.emptyMap())), - new KinesisDataSourceMetadata(new KinesisPartitions(stream, Collections.emptyMap())) + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, Collections.emptyMap())), + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, Collections.emptyMap())) ); while (supervisor.getNoticesQueueSize() > 0) { @@ -2861,8 +2861,8 @@ public void testCheckpointWithNullTaskGroupId() "id1", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), - new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE)), + new SeekableStreamPartitions<>(stream, ImmutableMap.of(0, 0L)), + new SeekableStreamPartitions<>(stream, ImmutableMap.of(0, Long.MAX_VALUE)), null, null ); @@ -2871,8 +2871,8 @@ public void testCheckpointWithNullTaskGroupId() "id2", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), - new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE)), + new SeekableStreamPartitions<>(stream, ImmutableMap.of(0, 0L)), + new SeekableStreamPartitions<>(stream, ImmutableMap.of(0, Long.MAX_VALUE)), null, null ); @@ -2881,8 +2881,8 @@ public void testCheckpointWithNullTaskGroupId() "id3", DATASOURCE, 0, - new KinesisPartitions(stream, ImmutableMap.of(0, 0L)), - new KinesisPartitions(stream, ImmutableMap.of(0, Long.MAX_VALUE)), + new SeekableStreamPartitions<>(stream, ImmutableMap.of(0, 0L)), + new SeekableStreamPartitions<>(stream, ImmutableMap.of(0, Long.MAX_VALUE)), null, null ); @@ -2928,8 +2928,8 @@ public void testCheckpointWithNullTaskGroupId() supervisor.checkpoint( null, ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), - new KinesisDataSourceMetadata(new KinesisPartitions(stream, checkpoints.get(0))), - new KinesisDataSourceMetadata(new KinesisPartitions(stream, newCheckpoints.get(0))) + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, checkpoints.get(0))), + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, newCheckpoints.get(0))) ); while (supervisor.getNoticesQueueSize() > 0) { @@ -2983,17 +2983,17 @@ public void testSuspendedRunningTasks() throws Exception "id1", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 0), shardId0, getSequenceNumber(res, shardId0, 0) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -3003,17 +3003,17 @@ public void testSuspendedRunningTasks() throws Exception "id2", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), shardId0, getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -3023,17 +3023,17 @@ public void testSuspendedRunningTasks() throws Exception "id3", DATASOURCE, 0, - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), shardId0, getSequenceNumber(res, shardId0, 1) )), - new KinesisPartitions("stream", ImmutableMap.of( + new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - KinesisPartitions.NO_END_SEQUENCE_NUMBER, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, shardId0, - KinesisPartitions.NO_END_SEQUENCE_NUMBER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), null, null @@ -3350,8 +3350,8 @@ private KinesisIndexTask createKinesisIndexTask( String id, String dataSource, int taskGroupId, - KinesisPartitions startPartitions, - KinesisPartitions endPartitions, + SeekableStreamPartitions startPartitions, + SeekableStreamPartitions endPartitions, DateTime minimumMessageTime, DateTime maximumMessageTime ) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java index 4ee23863b8dc..3712690ef944 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java @@ -81,11 +81,11 @@ public DataSourceMetadata plus(DataSourceMetadata other) // Same topic, merge offsets. final Map newMap = Maps.newHashMap(); - for (Map.Entry entry : seekableStreamPartitions.getPartitionSequenceMap().entrySet()) { + for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { newMap.put(entry.getKey(), entry.getValue()); } - for (Map.Entry entry : that.getSeekableStreamPartitions().getPartitionSequenceMap().entrySet()) { + for (Map.Entry entry : that.getSeekableStreamPartitions().getMap().entrySet()) { newMap.put(entry.getKey(), entry.getValue()); } @@ -115,8 +115,8 @@ public DataSourceMetadata minus(DataSourceMetadata other) // Same stream, remove partitions present in "that" from "this" final Map newMap = Maps.newHashMap(); - for (Map.Entry entry : seekableStreamPartitions.getPartitionSequenceMap().entrySet()) { - if (!that.getSeekableStreamPartitions().getPartitionSequenceMap().containsKey(entry.getKey())) { + for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { + if (!that.getSeekableStreamPartitions().getMap().containsKey(entry.getKey())) { newMap.put(entry.getKey(), entry.getValue()); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java index b0f055c16b8e..0cbf8e4a6738 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java @@ -67,17 +67,17 @@ public SeekableStreamIOConfig( ); Preconditions.checkArgument( - startPartitions.getPartitionSequenceMap().keySet().equals(endPartitions.getPartitionSequenceMap().keySet()), + startPartitions.getMap().keySet().equals(endPartitions.getMap().keySet()), "start partition set and end partition set must match" ); // are sequence numbers guranteed to be greater? /* - for (T1 partition : endPartitions.getPartitionSequenceMap().keySet()) { + for (T1 partition : endPartitions.getMap().keySet()) { Preconditions.checkArgument( - endPartitions.getPartitionSequenceMap() + endPartitions.getMap() .get(partition) - .compareTo(startPartitions.getPartitionSequenceMap().get(partition)) >= 0, + .compareTo(startPartitions.getMap().get(partition)) >= 0, "end offset must be >= start offset for partition[%s]", partition ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index 55f1ba873c09..c833fb832cdf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -24,25 +24,33 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; +import javax.validation.constraints.NotNull; import java.util.Map; import java.util.Objects; -// TODO: may consider deleting Kinesis and KafaPartitions classes and just use this instead -public abstract class SeekableStreamPartitions +public class SeekableStreamPartitions { + public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER"; + private final String id; - private final Map partitionSequenceMap; + private final Map map; @JsonCreator public SeekableStreamPartitions( - @JsonProperty("id") final String id, - @JsonProperty("partitionSequenceMap") final Map partitionOffsetMap + @JsonProperty("stream") final String stream, + @JsonProperty("topic") final String topic, + @JsonProperty("partitionSequenceNumberMap") final Map partitionSequenceNumberMap, + @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap ) { - this.id = id; - this.partitionSequenceMap = ImmutableMap.copyOf(partitionOffsetMap); - // Validate partitionSequenceNumberMap - for (Map.Entry entry : partitionOffsetMap.entrySet()) { + this.id = stream == null ? topic : stream; + this.map = ImmutableMap.copyOf(partitionOffsetMap == null + ? partitionSequenceNumberMap + : partitionOffsetMap); + Preconditions.checkArgument(id != null); + Preconditions.checkArgument(map != null); + // Validate map + for (Map.Entry entry : map.entrySet()) { Preconditions.checkArgument( entry.getValue() != null, String.format( @@ -54,16 +62,44 @@ public SeekableStreamPartitions( } } - @JsonProperty + // for backward compatibility + public SeekableStreamPartitions(@NotNull final String id, final Map partitionOffsetMap) + { + this(id, null, partitionOffsetMap, null); + } + public String getId() { return id; } @JsonProperty - public Map getPartitionSequenceMap() + public String getStream() + { + return id; + } + + @JsonProperty + public String getTopic() { - return partitionSequenceMap; + return id; + } + + public Map getMap() + { + return map; + } + + @JsonProperty + public Map getPartitionSequenceNumberMap() + { + return map; + } + + @JsonProperty + public Map getPartitionOffsetMap() + { + return map; } @Override @@ -77,13 +113,13 @@ public boolean equals(Object o) } SeekableStreamPartitions that = (SeekableStreamPartitions) o; return Objects.equals(id, that.id) && - Objects.equals(partitionSequenceMap, that.partitionSequenceMap); + Objects.equals(map, that.map); } @Override public int hashCode() { - return Objects.hash(id, partitionSequenceMap); + return Objects.hash(id, map); } @Override @@ -91,9 +127,7 @@ public String toString() { return "SeekableStreamPartitions{" + "stream/topic='" + id + '\'' + - ", partitionSequenceMap=" + partitionSequenceMap + + ", partitionSequenceNumberMap/partitionOffsetMap=" + map + '}'; } - - public abstract T2 getNoEndSequenceNumber(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 05a85e54cde0..d4adaa12c0e7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -701,12 +701,12 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) // as well as the case where the metadata store do not have an entry for the reset partitions boolean doReset = false; for (Map.Entry resetPartitionOffset : resetMetadata.getSeekableStreamPartitions() - .getPartitionSequenceMap() + .getMap() .entrySet()) { final T2 partitionOffsetInMetadataStore = currentMetadata == null ? null : currentMetadata.getSeekableStreamPartitions() - .getPartitionSequenceMap() + .getMap() .get(resetPartitionOffset.getKey()); final TaskGroup partitionTaskGroup = taskGroups.get( getTaskGroupIdForPartition(resetPartitionOffset.getKey()) @@ -739,7 +739,7 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) } } if (metadataUpdateSuccess) { - resetMetadata.getSeekableStreamPartitions().getPartitionSequenceMap().keySet().forEach(partition -> { + resetMetadata.getSeekableStreamPartitions().getMap().keySet().forEach(partition -> { final int groupId = getTaskGroupIdForPartition(partition); killTaskGroupForPartitions(ImmutableSet.of(partition)); taskGroups.remove(groupId); @@ -829,7 +829,7 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti Iterator it = seekableStreamIndexTask.getIOConfig() .getStartPartitions() - .getPartitionSequenceMap() + .getMap() .keySet() .iterator(); final Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null); @@ -853,7 +853,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) if (status == SeekableStreamIndexTask.Status.PUBLISHING) { seekableStreamIndexTask.getIOConfig() .getStartPartitions() - .getPartitionSequenceMap() + .getMap() .keySet() .forEach( partition -> addDiscoveredTaskToPendingCompletionTaskGroups( @@ -861,7 +861,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) taskId, seekableStreamIndexTask.getIOConfig() .getStartPartitions() - .getPartitionSequenceMap() + .getMap() )); // update partitionGroups with the publishing task's offsets (if they are greater than what is @@ -888,7 +888,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) } else { for (T1 partition : seekableStreamIndexTask.getIOConfig() .getStartPartitions() - .getPartitionSequenceMap() + .getMap() .keySet()) { if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { log.warn( @@ -928,7 +928,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) ImmutableMap.copyOf( seekableStreamIndexTask.getIOConfig() .getStartPartitions() - .getPartitionSequenceMap() + .getMap() ), seekableStreamIndexTask.getIOConfig().getMinimumMessageTime(), seekableStreamIndexTask.getIOConfig().getMaximumMessageTime(), @@ -1063,7 +1063,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) ); final Map latestOffsetsFromDb; if (hasValidOffsetsFromDb) { - latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamPartitions().getPartitionSequenceMap(); + latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamPartitions().getMap(); } else { latestOffsetsFromDb = null; } @@ -1235,7 +1235,7 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) return generateSequenceName( task.getIOConfig() .getStartPartitions() - .getPartitionSequenceMap(), + .getMap(), task.getIOConfig().getMinimumMessageTime(), task.getIOConfig().getMaximumMessageTime() ).equals(taskSequenceName); @@ -1978,8 +1978,8 @@ && checkSourceMetaDataMatch(dataSourceMetadata)) { ioConfig.getId() ); return Collections.emptyMap(); - } else if (partitions.getPartitionSequenceMap() != null) { - return partitions.getPartitionSequenceMap(); + } else if (partitions.getMap() != null) { + return partitions.getMap(); } } } @@ -2396,7 +2396,7 @@ public void handle() throws ExecutionException, InterruptedException // We have already verified the topic of the current checkpoint is same with that in ioConfig. // See checkpoint(). if (checkpoint.equals(previousCheckpoint.getSeekableStreamPartitions() - .getPartitionSequenceMap() + .getMap() )) { break; } From dd2ad96392d2c3d243142499ee054251547d68ad Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 4 Oct 2018 13:41:54 -0700 Subject: [PATCH 18/87] kinesis-indexing-service code cleanup and docs --- ...ementalPublishingKafkaIndexTaskRunner.java | 40 +- .../kafka/KafkaDataSourceMetadata.java | 2 +- .../druid/indexing/kafka/KafkaIOConfig.java | 14 - .../druid/indexing/kafka/KafkaIndexTask.java | 6 +- .../indexing/kafka/KafkaSequenceNumber.java | 7 + .../indexing/kafka/KafkaTuningConfig.java | 6 +- .../kafka/LegacyKafkaIndexTaskRunner.java | 18 +- .../KafkaSupervisorTuningConfig.java | 2 +- .../kafka/KafkaIndexTaskClientTest.java | 211 ++- .../indexing/kafka/KafkaIndexTaskTest.java | 115 +- .../kafka/supervisor/KafkaSupervisorTest.java | 966 ++++++------ .../kinesis/KinesisDataSourceMetadata.java | 2 +- .../indexing/kinesis/KinesisIOConfig.java | 16 +- .../indexing/kinesis/KinesisIndexTask.java | 1246 ++++++++-------- .../kinesis/KinesisIndexTaskRunner.java | 101 -- .../kinesis/KinesisRecordSupplier.java | 30 +- .../druid/indexing/kinesis/KinesisRegion.java | 20 +- .../kinesis/KinesisSequenceNumber.java | 21 +- .../indexing/kinesis/KinesisTuningConfig.java | 12 +- .../kinesis/supervisor/KinesisSupervisor.java | 18 +- .../supervisor/KinesisSupervisorIOConfig.java | 12 +- .../supervisor/KinesisSupervisorSpec.java | 24 +- .../KinesisSupervisorTuningConfig.java | 4 +- .../kinesis/KinesisIndexTaskClientTest.java | 336 +++-- .../kinesis/KinesisIndexTaskTest.java | 13 +- .../supervisor/KinesisSupervisorTest.java | 1299 +++++++++-------- .../indexing/kinesis/test/TestBroker.java | 121 -- .../SeekableStreamDataSourceMetadata.java | 8 +- .../SeekableStreamIOConfig.java | 16 +- .../SeekableStreamIndexTask.java | 2 +- .../SeekableStreamIndexTaskRunner.java | 15 +- .../SeekableStreamPartitions.java | 33 +- .../seekablestream/common/Record.java | 1 - .../seekablestream/common/RecordSupplier.java | 89 +- .../seekablestream/common/SequenceNumber.java | 23 +- .../common/StreamPartition.java | 5 + .../supervisor/SeekableStreamSupervisor.java | 28 +- 37 files changed, 2488 insertions(+), 2394 deletions(-) delete mode 100644 extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java delete mode 100644 extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestBroker.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 2e4d06068e92..5d682b9c7cb7 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -87,6 +87,7 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; +import javax.annotation.ParametersAreNonnullByDefault; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; import javax.ws.rs.DefaultValue; @@ -344,18 +345,21 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception ), "Sequence offsets are not compatible with start offsets of task"); nextOffsets.putAll(sequences.get(0).startOffsets); } else { + @SuppressWarnings("unchecked") final Map restoredMetadataMap = (Map) restoredMetadata; - final SeekableStreamPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( - restoredMetadataMap.get(METADATA_NEXT_PARTITIONS), - toolbox.getObjectMapper() - .getTypeFactory() - .constructParametrizedType( - SeekableStreamPartitions.class, - SeekableStreamPartitions.class, - Integer.class, - Long.class - ) - ); + final SeekableStreamPartitions restoredNextPartitions = toolbox + .getObjectMapper() + .convertValue( + restoredMetadataMap.get(METADATA_NEXT_PARTITIONS), + toolbox.getObjectMapper() + .getTypeFactory() + .constructParametrizedType( + SeekableStreamPartitions.class, + SeekableStreamPartitions.class, + Integer.class, + Long.class + ) + ); nextOffsets.putAll(restoredNextPartitions.getPartitionOffsetMap()); @@ -569,7 +573,7 @@ public void onSuccess(@Nullable Object result) } @Override - public void onFailure(Throwable t) + public void onFailure(@ParametersAreNonnullByDefault Throwable t) { log.error("Persist failed, dying"); backgroundThreadException = t; @@ -881,7 +885,7 @@ public Void apply(@Nullable SegmentsAndMetadata handoffSegmentsAndMetadata) } @Override - public void onFailure(Throwable t) + public void onFailure(@ParametersAreNonnullByDefault Throwable t) { log.error(t, "Error while publishing segments for sequence[%s]", sequenceMetadata); handoffFuture.setException(t); @@ -1448,11 +1452,11 @@ public Map> getCheckpointsHTTP(@Context final HttpSe private Map> getCheckpoints() { - TreeMap> result = new TreeMap<>(); - result.putAll( - sequences.stream().collect(Collectors.toMap(SequenceMetadata::getSequenceId, SequenceMetadata::getStartOffsets)) - ); - return result; + return new TreeMap<>(sequences.stream() + .collect(Collectors.toMap( + SequenceMetadata::getSequenceId, + SequenceMetadata::getStartOffsets + ))); } /** diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java index 2ad9c443f66a..cfb3908ad9a1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java @@ -48,6 +48,6 @@ protected SeekableStreamDataSourceMetadata createConcretDataSourc String streamId, Map newMap ) { - return new KafkaDataSourceMetadata(new SeekableStreamPartitions(streamId, newMap)); + return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(streamId, newMap)); } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java index d4d7b3d23cd3..6c6ac36da88e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java @@ -75,20 +75,6 @@ public KafkaIOConfig( } } - @Override - @JsonProperty - public SeekableStreamPartitions getStartPartitions() - { - return super.getStartPartitions(); - } - - @Override - @JsonProperty - public SeekableStreamPartitions getEndPartitions() - { - return super.getEndPartitions(); - } - @Override public Set getExclusiveStartSequenceNumberPartitions() { diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 792b95d8b7dd..827e5b6908d6 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -32,9 +32,9 @@ import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.TaskResource; -import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; @@ -96,8 +96,8 @@ public KafkaIndexTask( rowIngestionMetersFactory, "index_kafka" ); - if (context != null && context.get(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED) != null - && ((boolean) context.get(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED))) { + if (context != null && context.get(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED) != null + && ((boolean) context.get(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED))) { runner = new IncrementalPublishingKafkaIndexTaskRunner( this, parser, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java index 626a8d8c6161..2f411bfe99ab 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java @@ -52,4 +52,11 @@ public boolean equals(Object o) return this.compareTo((KafkaSequenceNumber) o) == 0; } + @Override + public int hashCode() + { + return super.hashCode(); + } + + } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java index d35af2f1f8b6..e275322e9218 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java @@ -85,7 +85,7 @@ public KafkaTuningConfig copyOf() getMaxTotalRows(), getIntermediatePersistPeriod(), getBasePersistDirectory(), - getMaxPendingPersists(), + 0, getIndexSpec(), true, isReportParseExceptions(), @@ -109,7 +109,7 @@ public KafkaTuningConfig withBasePersistDirectory(File dir) getMaxTotalRows(), getIntermediatePersistPeriod(), dir, - getMaxPendingPersists(), + 0, getIndexSpec(), true, isReportParseExceptions(), @@ -133,7 +133,7 @@ public String toString() ", maxBytesInMemory=" + getMaxBytesInMemory() + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + - ", maxPendingPersists=" + getMaxPendingPersists() + + ", maxPendingPersists=" + 0 + ", indexSpec=" + getIndexSpec() + ", reportParseExceptions=" + isReportParseExceptions() + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index f2fc35f8db0b..f9576e3a620c 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -89,6 +89,7 @@ import javax.ws.rs.core.Response; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -462,9 +463,9 @@ public void run() if (isPersistRequired) { driver.persist(committerSupplier.get()); } - segmentsToMoveOut.entrySet().forEach(sequenceSegments -> driver.moveSegmentOut( - sequenceSegments.getKey(), - sequenceSegments.getValue().stream().collect(Collectors.toList()) + segmentsToMoveOut.forEach((key, value) -> driver.moveSegmentOut( + key, + new ArrayList(value) )); } catch (ParseException e) { @@ -503,7 +504,14 @@ public void run() final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { final SeekableStreamPartitions finalPartitions = toolbox.getObjectMapper().convertValue( ((Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_NEXT_PARTITIONS), - SeekableStreamPartitions.class + toolbox.getObjectMapper() + .getTypeFactory() + .constructParametrizedType( + SeekableStreamPartitions.class, + SeekableStreamPartitions.class, + Integer.class, + Long.class + ) ); // Sanity check, we should only be publishing things that match our desired end state. @@ -622,7 +630,7 @@ private Set assignPartitionsAndSeekToNext(KafkaConsumer consumer, Strin } } - task.assignPartitions(consumer, topic, assignment); + KafkaIndexTask.assignPartitions(consumer, topic, assignment); // Seek to starting offsets. for (final int partition : assignment) { diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index 651380382d5c..fe2362ffad7a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -146,7 +146,7 @@ public String toString() ", maxBytesInMemory=" + TuningConfigs.getMaxBytesInMemoryOrDefault(getMaxBytesInMemory()) + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + - ", maxPendingPersists=" + getMaxPendingPersists() + + ", maxPendingPersists=" + 0 + ", indexSpec=" + getIndexSpec() + ", reportParseExceptions=" + isReportParseExceptions() + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java index faa51a7e0ca6..c441efd626ac 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -42,6 +42,7 @@ import org.apache.druid.java.util.http.client.response.FullResponseHolder; import org.easymock.Capture; import org.easymock.CaptureType; +import org.easymock.EasyMock; import org.easymock.EasyMockSupport; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -64,12 +65,6 @@ import java.util.List; import java.util.Map; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.capture; -import static org.easymock.EasyMock.eq; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.reset; - @RunWith(Parameterized.class) public class KafkaIndexTaskClientTest extends EasyMockSupport { @@ -116,15 +111,15 @@ public void setUp() headers = createMock(HttpHeaders.class); client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider); - expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) - .anyTimes(); - expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); for (String testId : TEST_IDS) { - expect(taskInfoProvider.getTaskLocation(testId)) + EasyMock.expect(taskInfoProvider.getTaskLocation(testId)) .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) .anyTimes(); - expect(taskInfoProvider.getTaskStatus(testId)) + EasyMock.expect(taskInfoProvider.getTaskStatus(testId)) .andReturn(Optional.of(TaskStatus.running(testId))) .anyTimes(); } @@ -139,9 +134,9 @@ public void tearDown() @Test public void testNoTaskLocation() throws IOException { - reset(taskInfoProvider); - expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); - expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); + EasyMock.reset(taskInfoProvider); + EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); replayAll(); Assert.assertEquals(false, client.stop(TEST_ID, true)); @@ -164,10 +159,10 @@ public void testTaskNotRunnableException() expectedException.expect(IndexTaskClient.TaskNotRunnableException.class); expectedException.expectMessage("Aborting request because task [test-id] is not runnable"); - reset(taskInfoProvider); - expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + EasyMock.reset(taskInfoProvider); + EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) .anyTimes(); - expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.failure(TEST_ID))).anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.failure(TEST_ID))).anyTimes(); replayAll(); client.getCurrentOffsets(TEST_ID, true); @@ -180,12 +175,12 @@ public void testInternalServerError() expectedException.expect(RuntimeException.class); expectedException.expectMessage("org.apache.druid.java.util.common.IOE: Received status [500]"); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2); - expect( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2); + EasyMock.expect( httpClient.go( - anyObject(Request.class), - anyObject(FullResponseHandler.class), - eq(TEST_HTTP_TIMEOUT) + EasyMock.anyObject(Request.class), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) ) ).andReturn( Futures.immediateFuture(responseHolder) @@ -202,13 +197,13 @@ public void testBadRequest() expectedException.expect(IAE.class); expectedException.expectMessage("Received 400 Bad Request with body:"); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2); - expect(responseHolder.getContent()).andReturn(""); - expect( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2); + EasyMock.expect(responseHolder.getContent()).andReturn(""); + EasyMock.expect( httpClient.go( - anyObject(Request.class), - anyObject(FullResponseHandler.class), - eq(TEST_HTTP_TIMEOUT) + EasyMock.anyObject(Request.class), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) ) ).andReturn( Futures.immediateFuture(responseHolder) @@ -222,18 +217,18 @@ public void testBadRequest() @Test public void testTaskLocationMismatch() { - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) .andReturn(HttpResponseStatus.OK); - expect(responseHolder.getResponse()).andReturn(response); - expect(responseHolder.getContent()).andReturn("") + EasyMock.expect(responseHolder.getResponse()).andReturn(response); + EasyMock.expect(responseHolder.getContent()).andReturn("") .andReturn("{}"); - expect(response.headers()).andReturn(headers); - expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id"); - expect( + EasyMock.expect(response.headers()).andReturn(headers); + EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id"); + EasyMock.expect( httpClient.go( - anyObject(Request.class), - anyObject(FullResponseHandler.class), - eq(TEST_HTTP_TIMEOUT) + EasyMock.anyObject(Request.class), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) ) ).andReturn( Futures.immediateFuture(responseHolder) @@ -250,9 +245,9 @@ public void testTaskLocationMismatch() public void testGetCurrentOffsets() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -279,15 +274,15 @@ public void testGetCurrentOffsetsWithRetry() throws Exception client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 3); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) .andReturn(HttpResponseStatus.OK).times(1); - expect(responseHolder.getContent()).andReturn("").times(2) + EasyMock.expect(responseHolder.getContent()).andReturn("").times(2) .andReturn("{\"0\":1, \"1\":10}"); - expect(responseHolder.getResponse()).andReturn(response).times(2); - expect(response.headers()).andReturn(headers).times(2); - expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2); + EasyMock.expect(responseHolder.getResponse()).andReturn(response).times(2); + EasyMock.expect(response.headers()).andReturn(headers).times(2); + EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ).times(3); @@ -319,17 +314,17 @@ public void testGetCurrentOffsetsWithExhaustedRetries() client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); - expect(responseHolder.getContent()).andReturn("").anyTimes(); - expect(responseHolder.getResponse()).andReturn(response).anyTimes(); - expect(response.headers()).andReturn(headers).anyTimes(); - expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).anyTimes(); + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); + EasyMock.expect(responseHolder.getContent()).andReturn("").anyTimes(); + EasyMock.expect(responseHolder.getResponse()).andReturn(response).anyTimes(); + EasyMock.expect(response.headers()).andReturn(headers).anyTimes(); + EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).anyTimes(); - expect( + EasyMock.expect( httpClient.go( - anyObject(Request.class), - anyObject(FullResponseHandler.class), - eq(TEST_HTTP_TIMEOUT) + EasyMock.anyObject(Request.class), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) ) ).andReturn(Futures.immediateFuture(responseHolder)).anyTimes(); replayAll(); @@ -342,9 +337,9 @@ public void testGetCurrentOffsetsWithExhaustedRetries() public void testGetEndOffsets() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -372,13 +367,13 @@ public void testGetStartTime() throws Exception DateTime now = DateTimes.nowUtc(); Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) .andReturn(HttpResponseStatus.OK); - expect(responseHolder.getResponse()).andReturn(response); - expect(response.headers()).andReturn(headers); - expect(headers.get("X-Druid-Task-Id")).andReturn(null); - expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getResponse()).andReturn(response); + EasyMock.expect(response.headers()).andReturn(headers); + EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(null); + EasyMock.expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ).times(2); replayAll(); @@ -403,9 +398,9 @@ public void testGetStatus() throws Exception SeekableStreamIndexTask.Status status = SeekableStreamIndexTask.Status.READING; Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - expect(responseHolder.getContent()).andReturn(StringUtils.format("\"%s\"", status.toString())).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + EasyMock.expect(responseHolder.getContent()).andReturn(StringUtils.format("\"%s\"", status.toString())).anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -428,9 +423,9 @@ public void testGetStatus() throws Exception public void testPause() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); - expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); + EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -457,17 +452,17 @@ public void testPauseWithSubsequentGetOffsets() throws Exception Capture captured = Capture.newInstance(); Capture captured2 = Capture.newInstance(); Capture captured3 = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.ACCEPTED).times(2) + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.ACCEPTED).times(2) .andReturn(HttpResponseStatus.OK).times(2); - expect(responseHolder.getContent()).andReturn("\"PAUSED\"") + EasyMock.expect(responseHolder.getContent()).andReturn("\"PAUSED\"") .andReturn("{\"0\":1, \"1\":10}").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); - expect(httpClient.go(capture(captured2), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go(EasyMock.capture(captured2), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); - expect(httpClient.go(capture(captured3), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go(EasyMock.capture(captured3), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); @@ -507,8 +502,8 @@ public void testPauseWithSubsequentGetOffsets() throws Exception public void testResume() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -531,8 +526,8 @@ public void testSetEndOffsets() throws Exception Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -556,8 +551,8 @@ public void testSetEndOffsetsAndResume() throws Exception Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -579,8 +574,8 @@ public void testSetEndOffsetsAndResume() throws Exception public void testStop() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -601,8 +596,8 @@ public void testStop() throws Exception public void testStopAndPublish() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -624,8 +619,8 @@ public void testStopAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -656,8 +651,8 @@ public void testResumeAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -688,9 +683,9 @@ public void testPauseAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -721,9 +716,9 @@ public void testGetStatusAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -755,9 +750,9 @@ public void testGetStartTimeAsync() throws Exception final DateTime now = DateTimes.nowUtc(); final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -788,9 +783,9 @@ public void testGetCurrentOffsetsAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -821,9 +816,9 @@ public void testGetEndOffsetsAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -855,8 +850,8 @@ public void testSetEndOffsetsAsync() throws Exception final Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -894,8 +889,8 @@ public void testSetEndOffsetsAsyncWithResume() throws Exception final Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 9cc2e0e43211..d1dc35d5cea3 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -105,6 +105,7 @@ import org.apache.druid.query.Druids; import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryToolChest; @@ -176,8 +177,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import static org.apache.druid.query.QueryPlus.wrap; - @RunWith(Parameterized.class) public class KafkaIndexTaskTest { @@ -491,12 +490,44 @@ public void testIncrementalHandOff() throws Exception Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); - final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L, 1, 0L)); + final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of( + 0, + 0L, + 1, + 0L + ) + ); // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering // of events fetched across two partitions from Kafka - final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 5L, 1, 0L)); - final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 4L, 1, 2L)); - final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L)); + final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of( + 0, + 5L, + 1, + 0L + ) + ); + final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of( + 0, + 4L, + 1, + 2L + ) + ); + final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of( + 0, + 10L, + 1, + 2L + ) + ); final KafkaIndexTask task = createTask( null, new KafkaIOConfig( @@ -586,11 +617,43 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); - final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L, 1, 0L)); - final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 3L, 1, 0L)); - final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 0L)); + final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of( + 0, + 0L, + 1, + 0L + ) + ); + final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of( + 0, + 3L, + 1, + 0L + ) + ); + final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of( + 0, + 10L, + 1, + 0L + ) + ); - final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 2L)); + final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of( + 0, + 10L, + 1, + 2L + ) + ); final KafkaIndexTask task = createTask( null, new KafkaIOConfig( @@ -703,10 +766,34 @@ public void testTimeBasedIncrementalHandOff() throws Exception Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); - final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L, 1, 0L)); + final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of( + 0, + 0L, + 1, + 0L + ) + ); // Checkpointing will happen at checkpoint - final SeekableStreamPartitions checkpoint = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 1L, 1, 0L)); - final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L, 1, 0L)); + final SeekableStreamPartitions checkpoint = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of( + 0, + 1L, + 1, + 0L + ) + ); + final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of( + 0, + 2L, + 1, + 0L + ) + ); final KafkaIndexTask task = createTask( null, new KafkaIOConfig( @@ -2360,7 +2447,7 @@ public long countEvents(final Task task) .build(); List> results = - task.getQueryRunner(query).run(wrap(query), ImmutableMap.of()).toList(); + task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList(); return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 8c7820dcf9c0..276c2a03a1f8 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -114,16 +114,6 @@ import java.util.concurrent.Executor; import java.util.concurrent.TimeoutException; -import static org.easymock.EasyMock.anyBoolean; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.anyString; -import static org.easymock.EasyMock.capture; -import static org.easymock.EasyMock.eq; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.expectLastCall; -import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.reset; - @RunWith(Parameterized.class) public class KafkaSupervisorTest extends EasyMockSupport { @@ -266,16 +256,16 @@ public void testNoInitialState() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -313,16 +303,16 @@ public void testSkipOffsetGaps() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -342,15 +332,15 @@ public void testMultiTask() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); replayAll(); supervisor.start(); @@ -379,15 +369,15 @@ public void testReplicas() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); replayAll(); supervisor.start(); @@ -416,15 +406,15 @@ public void testLateMessageRejectionPeriod() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); replayAll(); supervisor.start(); @@ -455,15 +445,15 @@ public void testEarlyMessageRejectionPeriod() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); replayAll(); supervisor.start(); @@ -497,15 +487,15 @@ public void testLatestOffset() throws Exception addSomeEvents(1100); Capture captured = Capture.newInstance(); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); replayAll(); supervisor.start(); @@ -529,15 +519,15 @@ public void testDatasourceMetadata() throws Exception addSomeEvents(100); Capture captured = Capture.newInstance(); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); replayAll(); supervisor.start(); @@ -558,9 +548,9 @@ public void testBadMetadataOffsets() throws Exception supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); addSomeEvents(1); - expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) ) @@ -635,34 +625,34 @@ public void testKillIncompatibleTasks() throws Exception List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); - expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); - expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(false)); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); + EasyMock.expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(false)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); taskQueue.shutdown("id3"); - expect(taskQueue.add(anyObject(Task.class))).andReturn(true); + EasyMock.expect(taskQueue.add(EasyMock.anyObject(Task.class))).andReturn(true); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(2); @@ -730,44 +720,44 @@ public void testKillBadPartitionAssignment() throws Exception List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); - expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).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(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); - expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); - expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); - expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); - expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); - expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); + EasyMock.expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); + EasyMock.expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); + EasyMock.expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(1); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(1); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); taskQueue.shutdown("id4"); taskQueue.shutdown("id5"); replayAll(); @@ -784,32 +774,32 @@ public void testRequeueTaskWhenFailed() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); - expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(4); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .anyTimes(); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -820,13 +810,13 @@ public void testRequeueTaskWhenFailed() throws Exception // test that running the main loop again checks the status of the tasks that were created and does nothing if they // are all still running - reset(taskStorage); - expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + EasyMock.reset(taskStorage); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - replay(taskStorage); + EasyMock.replay(taskStorage); supervisor.runInternal(); verifyAll(); @@ -835,18 +825,18 @@ public void testRequeueTaskWhenFailed() throws Exception Capture aNewTaskCapture = Capture.newInstance(); List imStillAlive = tasks.subList(0, 3); KafkaIndexTask iHaveFailed = (KafkaIndexTask) tasks.get(3); - reset(taskStorage); - reset(taskQueue); - expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes(); + EasyMock.reset(taskStorage); + EasyMock.reset(taskQueue); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes(); for (Task task : imStillAlive) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); - expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); - expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); - replay(taskStorage); - replay(taskQueue); + EasyMock.expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); + EasyMock.expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.capture(aNewTaskCapture))).andReturn(true); + EasyMock.replay(taskStorage); + EasyMock.replay(taskQueue); supervisor.runInternal(); verifyAll(); @@ -879,16 +869,16 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception List existingTasks = ImmutableList.of(id1); Capture captured = Capture.newInstance(); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); - expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); - expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) @@ -896,11 +886,11 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L, 2, 0L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(2); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -914,29 +904,29 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception String runningTaskId = captured.getValue().getId(); Capture aNewTaskCapture = Capture.newInstance(); KafkaIndexTask iHaveFailed = (KafkaIndexTask) existingTasks.get(0); - reset(taskStorage); - reset(taskQueue); - reset(taskClient); + EasyMock.reset(taskStorage); + EasyMock.reset(taskQueue); + EasyMock.reset(taskClient); // for the newly created replica task - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(2); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes(); - expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); - expect(taskStorage.getStatus(runningTaskId)).andReturn(Optional.of(TaskStatus.running(runningTaskId))).anyTimes(); - expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); - expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); - expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(Status.READING)); - expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes(); - expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); - replay(taskStorage); - replay(taskQueue); - replay(taskClient); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes(); + EasyMock.expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); + EasyMock.expect(taskStorage.getStatus(runningTaskId)).andReturn(Optional.of(TaskStatus.running(runningTaskId))).anyTimes(); + EasyMock.expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); + EasyMock.expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.capture(aNewTaskCapture))).andReturn(true); + EasyMock.replay(taskStorage); + EasyMock.replay(taskQueue); + EasyMock.replay(taskClient); supervisor.runInternal(); verifyAll(); @@ -963,20 +953,20 @@ public void testQueueNextTasksOnSuccess() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); - expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(4); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -986,31 +976,31 @@ public void testQueueNextTasksOnSuccess() throws Exception List tasks = captured.getValues(); - reset(taskStorage); - reset(taskClient); + EasyMock.reset(taskStorage); + EasyMock.reset(taskClient); - expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); - expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); // there would be 4 tasks, 2 for each task group - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); - expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - replay(taskStorage); - replay(taskClient); + EasyMock.replay(taskStorage); + EasyMock.replay(taskClient); supervisor.runInternal(); verifyAll(); @@ -1021,21 +1011,21 @@ public void testQueueNextTasksOnSuccess() throws Exception Capture shutdownTaskIdCapture = Capture.newInstance(); List imStillRunning = tasks.subList(1, 4); KafkaIndexTask iAmSuccess = (KafkaIndexTask) tasks.get(0); - reset(taskStorage); - reset(taskQueue); - reset(taskClient); - expect(taskStorage.getActiveTasks()).andReturn(imStillRunning).anyTimes(); + EasyMock.reset(taskStorage); + EasyMock.reset(taskQueue); + EasyMock.reset(taskClient); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(imStillRunning).anyTimes(); for (Task task : imStillRunning) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - expect(taskStorage.getStatus(iAmSuccess.getId())).andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId()))); - expect(taskStorage.getTask(iAmSuccess.getId())).andReturn(Optional.of((Task) iAmSuccess)).anyTimes(); - expect(taskQueue.add(capture(newTasksCapture))).andReturn(true).times(2); - expect(taskClient.stopAsync(capture(shutdownTaskIdCapture), eq(false))).andReturn(Futures.immediateFuture(true)); - replay(taskStorage); - replay(taskQueue); - replay(taskClient); + EasyMock.expect(taskStorage.getStatus(iAmSuccess.getId())).andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId()))); + EasyMock.expect(taskStorage.getTask(iAmSuccess.getId())).andReturn(Optional.of((Task) iAmSuccess)).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.capture(newTasksCapture))).andReturn(true).times(2); + EasyMock.expect(taskClient.stopAsync(EasyMock.capture(shutdownTaskIdCapture), EasyMock.eq(false))).andReturn(Futures.immediateFuture(true)); + EasyMock.replay(taskStorage); + EasyMock.replay(taskQueue); + EasyMock.replay(taskClient); supervisor.runInternal(); verifyAll(); @@ -1053,17 +1043,17 @@ public void testBeginPublishAndQueueNextTasks() throws Exception addSomeEvents(100); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(4); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1076,47 +1066,47 @@ public void testBeginPublishAndQueueNextTasks() throws Exception workItems.add(new TestTaskRunnerWorkItem(task, null, location)); } - reset(taskStorage, taskRunner, taskClient, taskQueue); + EasyMock.reset(taskStorage, taskRunner, taskClient, taskQueue); captured = Capture.newInstance(CaptureType.ALL); - expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskClient.getStatusAsync(anyString())) + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(Status.READING)) .anyTimes(); - expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); - expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) .times(2); - expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); - expect( + EasyMock.expect( taskClient.setEndOffsetsAsync( EasyMock.contains("sequenceName-0"), EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), EasyMock.eq(true) ) ).andReturn(Futures.immediateFuture(true)).times(2); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); - replay(taskStorage, taskRunner, taskClient, taskQueue); + EasyMock.replay(taskStorage, taskRunner, taskClient, taskQueue); supervisor.runInternal(); verifyAll(); @@ -1162,28 +1152,28 @@ public void testDiscoverExistingPublishingTask() throws Exception workItems.add(new TestTaskRunnerWorkItem(task, null, location)); Capture captured = Capture.newInstance(); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); - expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); - expect(taskClient.getCurrentOffsetsAsync("id1", false)) + EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); + EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); - expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - expect(taskQueue.add(capture(captured))).andReturn(true); + EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)); - expect(taskClient.getCheckpoints(anyString(), anyBoolean())).andReturn(checkpoints).anyTimes(); + EasyMock.expect(taskClient.getCheckpoints(EasyMock.anyString(), EasyMock.anyBoolean())).andReturn(checkpoints).anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1254,24 +1244,24 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() workItems.add(new TestTaskRunnerWorkItem(task, null, location)); Capture captured = Capture.newInstance(); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); - expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); - expect(taskClient.getCurrentOffsetsAsync("id1", false)) + EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); + EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 2, 30L))); - expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); - expect(taskQueue.add(capture(captured))).andReturn(true); + EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1360,34 +1350,34 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2)).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.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); - expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getCurrentOffsetsAsync("id1", false)) + EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); + EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 1L, 1, 2L, 2, 3L))); - expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); - expect(taskClient.getCurrentOffsetsAsync("id2", false)) + EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); + EasyMock.expect(taskClient.getCurrentOffsetsAsync("id2", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 4L, 1, 5L, 2, 6L))); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); // since id1 is publishing, so getCheckpoints wouldn't be called for it TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); @@ -1438,17 +1428,17 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(4); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1457,30 +1447,30 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception List tasks = captured.getValues(); - reset(taskStorage, taskClient, taskQueue); + EasyMock.reset(taskStorage, taskClient, taskQueue); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); - expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); - expect(taskClient.getStatusAsync(task.getId())) + EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(task.getId())) .andReturn(Futures.immediateFuture(Status.NOT_STARTED)); - expect(taskClient.getStartTimeAsync(task.getId())) + EasyMock.expect(taskClient.getStartTimeAsync(task.getId())) .andReturn(Futures.immediateFailedFuture(new RuntimeException())); taskQueue.shutdown(task.getId()); } - replay(taskStorage, taskClient, taskQueue); + EasyMock.replay(taskStorage, taskClient, taskQueue); supervisor.runInternal(); verifyAll(); @@ -1495,17 +1485,17 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception addSomeEvents(100); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(4); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1518,42 +1508,42 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception workItems.add(new TestTaskRunnerWorkItem(task, null, location)); } - reset(taskStorage, taskRunner, taskClient, taskQueue); + EasyMock.reset(taskStorage, taskRunner, taskClient, taskQueue); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); captured = Capture.newInstance(CaptureType.ALL); - expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskClient.getStatusAsync(anyString())) + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(Status.READING)) .anyTimes(); - expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); - expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) .times(2); - expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); taskQueue.shutdown(EasyMock.contains("sequenceName-0")); - expectLastCall().times(2); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expectLastCall().times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); - replay(taskStorage, taskRunner, taskClient, taskQueue); + EasyMock.replay(taskStorage, taskRunner, taskClient, taskQueue); supervisor.runInternal(); verifyAll(); @@ -1574,17 +1564,17 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception addSomeEvents(100); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(4); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1597,39 +1587,39 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception workItems.add(new TestTaskRunnerWorkItem(task, null, location)); } - reset(taskStorage, taskRunner, taskClient, taskQueue); + EasyMock.reset(taskStorage, taskRunner, taskClient, taskQueue); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); captured = Capture.newInstance(CaptureType.ALL); - expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskClient.getStatusAsync(anyString())) + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(Status.READING)) .anyTimes(); - expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); - expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) .times(2); - expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); - expect( + EasyMock.expect( taskClient.setEndOffsetsAsync( EasyMock.contains("sequenceName-0"), EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), @@ -1637,10 +1627,10 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception ) ).andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); taskQueue.shutdown(EasyMock.contains("sequenceName-0")); - expectLastCall().times(2); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expectLastCall().times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); - replay(taskStorage, taskRunner, taskClient, taskQueue); + EasyMock.replay(taskStorage, taskRunner, taskClient, taskQueue); supervisor.runInternal(); verifyAll(); @@ -1662,7 +1652,7 @@ public void testStopNotStarted() @Test public void testStop() { - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); taskClient.close(); taskRunner.unregisterListener(StringUtils.format("KafkaSupervisor-%s", DATASOURCE)); replayAll(); @@ -1727,55 +1717,55 @@ public void testStopGracefully() throws Exception workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(workItems).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( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); - expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); - expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); + EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); // getCheckpoints will not be called for id1 as it is in publishing state TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); supervisor.runInternal(); verifyAll(); - reset(taskRunner, taskClient, taskQueue); - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskClient.pauseAsync("id2")) + EasyMock.reset(taskRunner, taskClient, taskQueue); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskClient.pauseAsync("id2")) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); - expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) + EasyMock.expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) .andReturn(Futures.immediateFuture(true)); taskQueue.shutdown("id3"); - expectLastCall().times(2); + EasyMock.expectLastCall().times(2); - replay(taskRunner, taskClient, taskQueue); + EasyMock.replay(taskRunner, taskClient, taskQueue); supervisor.gracefulShutdownInternal(); verifyAll(); @@ -1784,11 +1774,11 @@ public void testStopGracefully() throws Exception @Test public void testResetNoTasks() throws Exception { - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); @@ -1796,9 +1786,9 @@ public void testResetNoTasks() throws Exception supervisor.runInternal(); verifyAll(); - reset(indexerMetadataStorageCoordinator); - expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); - replay(indexerMetadataStorageCoordinator); + EasyMock.reset(indexerMetadataStorageCoordinator); + EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + EasyMock.replay(indexerMetadataStorageCoordinator); supervisor.resetInternal(null); verifyAll(); @@ -1809,11 +1799,11 @@ public void testResetNoTasks() throws Exception public void testResetDataSourceMetadata() throws Exception { supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1838,19 +1828,19 @@ public void testResetDataSourceMetadata() throws Exception ImmutableMap.of(0, 1000L) )); - reset(indexerMetadataStorageCoordinator); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(kafkaDataSourceMetadata); - expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata( + EasyMock.reset(indexerMetadataStorageCoordinator); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(kafkaDataSourceMetadata); + EasyMock.expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata( EasyMock.capture(captureDataSource), EasyMock.capture(captureDataSourceMetadata) )).andReturn(true); - replay(indexerMetadataStorageCoordinator); + EasyMock.replay(indexerMetadataStorageCoordinator); try { supervisor.resetInternal(resetMetadata); } catch (NullPointerException npe) { - // Expected as there will be an attempt to reset partitionGroups offsets to NOT_SET + // Expected as there will be an attempt to EasyMock.reset partitionGroups offsets to NOT_SET // however there would be no entries in the map as we have not put nay data in kafka Assert.assertTrue(npe.getCause() == null); } @@ -1864,11 +1854,11 @@ public void testResetDataSourceMetadata() throws Exception public void testResetNoDataSourceMetadata() throws Exception { supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1880,10 +1870,10 @@ public void testResetNoDataSourceMetadata() throws Exception ImmutableMap.of(1, 1000L, 2, 1000L) )); - reset(indexerMetadataStorageCoordinator); + EasyMock.reset(indexerMetadataStorageCoordinator); // no DataSourceMetadata in metadata store - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null); - replay(indexerMetadataStorageCoordinator); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null); + EasyMock.replay(indexerMetadataStorageCoordinator); supervisor.resetInternal(resetMetadata); verifyAll(); @@ -1942,49 +1932,49 @@ public void testResetRunningTasks() throws Exception workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(workItems).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( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); - expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); - expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); + EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); supervisor.runInternal(); verifyAll(); - reset(taskQueue, indexerMetadataStorageCoordinator); - expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + EasyMock.reset(taskQueue, indexerMetadataStorageCoordinator); + EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); taskQueue.shutdown("id2"); taskQueue.shutdown("id3"); - replay(taskQueue, indexerMetadataStorageCoordinator); + EasyMock.replay(taskQueue, indexerMetadataStorageCoordinator); supervisor.resetInternal(null); verifyAll(); @@ -2035,52 +2025,52 @@ public void testNoDataIngestionTasks() throws Exception 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( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); - expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); - expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); supervisor.runInternal(); verifyAll(); - reset(taskQueue, indexerMetadataStorageCoordinator); - expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + EasyMock.reset(taskQueue, indexerMetadataStorageCoordinator); + EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); taskQueue.shutdown("id1"); taskQueue.shutdown("id2"); taskQueue.shutdown("id3"); - replay(taskQueue, indexerMetadataStorageCoordinator); + EasyMock.replay(taskQueue, indexerMetadataStorageCoordinator); supervisor.resetInternal(null); verifyAll(); @@ -2129,41 +2119,41 @@ public void testCheckpointForInactiveTaskGroup() workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - 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( + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect( indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KafkaDataSourceMetadata(null) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); - expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(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)); + EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); final TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -2224,16 +2214,16 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException 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( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect( indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KafkaDataSourceMetadata(null) ).anyTimes(); @@ -2303,32 +2293,32 @@ public void testCheckpointWithNullTaskGroupId() 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( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect( indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KafkaDataSourceMetadata(null) ).anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); - expect(taskClient.getStatusAsync(anyString())) + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(Status.READING)) .anyTimes(); final TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L)); - expect(taskClient.getCheckpointsAsync(anyString(), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.anyString(), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(3); - expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); - expect(taskClient.pauseAsync(anyString())) + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(taskClient.pauseAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L))) .anyTimes(); - expect(taskClient.setEndOffsetsAsync(anyString(), EasyMock.eq(ImmutableMap.of(0, 10L)), anyBoolean())) + EasyMock.expect(taskClient.setEndOffsetsAsync(EasyMock.anyString(), EasyMock.eq(ImmutableMap.of(0, 10L)), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(true)) .anyTimes(); @@ -2360,20 +2350,20 @@ public void testSuspendedNoRunningTasks() throws Exception supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false, true, kafkaHost); addSomeEvents(1); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); // this asserts that taskQueue.add does not in fact get called because supervisor should be suspended - expect(taskQueue.add(anyObject())).andAnswer((IAnswer) () -> { + EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andAnswer((IAnswer) () -> { Assert.fail(); return null; }).anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -2436,46 +2426,46 @@ public void testSuspendedRunningTasks() throws Exception workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(workItems).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( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); // getCheckpoints will not be called for id1 as it is in publishing state TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); - expect(taskClient.pauseAsync("id2")) + EasyMock.expect(taskClient.pauseAsync("id2")) .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); - expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) + EasyMock.expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) .andReturn(Futures.immediateFuture(true)); taskQueue.shutdown("id3"); - expectLastCall().times(2); + EasyMock.expectLastCall().times(2); replayAll(); supervisor.start(); @@ -2486,11 +2476,11 @@ public void testSuspendedRunningTasks() throws Exception @Test public void testResetSuspended() throws Exception { - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false, true, kafkaHost); @@ -2498,9 +2488,9 @@ public void testResetSuspended() throws Exception supervisor.runInternal(); verifyAll(); - reset(indexerMetadataStorageCoordinator); - expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); - replay(indexerMetadataStorageCoordinator); + EasyMock.reset(indexerMetadataStorageCoordinator); + EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + EasyMock.replay(indexerMetadataStorageCoordinator); supervisor.resetInternal(null); verifyAll(); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java index d972fa314cdd..1c7227f95651 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java @@ -45,6 +45,6 @@ public SeekableStreamPartitions getKinesisPartitions() @Override protected KinesisDataSourceMetadata createConcretDataSourceMetaData(String streamName, Map newMap) { - return new KinesisDataSourceMetadata(new SeekableStreamPartitions(streamName, newMap)); + return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(streamName, newMap)); } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java index ee76bad55d5a..0cee633bdb77 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java @@ -74,6 +74,7 @@ public KinesisIOConfig( minimumMessageTime, maximumMessageTime ); + this.pauseAfterRead = pauseAfterRead != null ? pauseAfterRead : DEFAULT_PAUSE_AFTER_READ; this.endpoint = Preconditions.checkNotNull(endpoint, "endpoint"); this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : DEFAULT_RECORDS_PER_FETCH; @@ -84,7 +85,6 @@ public KinesisIOConfig( this.awsAssumedRoleArn = awsAssumedRoleArn; this.awsExternalId = awsExternalId; this.deaggregate = deaggregate; - } @JsonProperty @@ -148,20 +148,6 @@ public boolean isDeaggregate() return deaggregate; } - @Override - @JsonProperty - public SeekableStreamPartitions getStartPartitions() - { - return super.getStartPartitions(); - } - - @Override - @JsonProperty - public SeekableStreamPartitions getEndPartitions() - { - return super.getEndPartitions(); - } - @Override public String toString() { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 4b8791f10e70..be0d038d3ddd 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -25,7 +25,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Supplier; import com.google.common.base.Throwables; @@ -64,12 +63,10 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.collect.Utils; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.NoopQueryRunner; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeIOConfig; @@ -123,7 +120,6 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; -// TODO: kinesis task read from startPartitions to endPartitions inclusive, whereas kafka is exclusive, should change behavior to that of kafka's public class KinesisIndexTask extends SeekableStreamIndexTask { public static final long PAUSE_FOREVER = -1L; @@ -141,42 +137,14 @@ public class KinesisIndexTask extends SeekableStreamIndexTask private final Map lastOffsets = new ConcurrentHashMap<>(); private final KinesisIOConfig ioConfig; private final KinesisTuningConfig tuningConfig; - private ObjectMapper mapper; - - private volatile Appenderator appenderator = null; - private volatile FireDepartmentMetrics fireDepartmentMetrics = null; - private volatile DateTime startTime; - private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread) - private volatile Thread runThread = null; - private volatile boolean stopRequested = false; - private volatile boolean publishOnStop = false; - - // The pause lock and associated conditions are to support coordination between the Jetty threads and the main - // ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully - // the ingestion loop has been stopped at the returned offsets and will not ingest any more data until resumed. The - // fields are used as follows (every step requires acquiring [pauseLock]): - // Pausing: - // - In pause(), [pauseRequested] is set to true and then execution waits for [status] to change to PAUSED, with the - // condition checked when [hasPaused] is signalled. - // - In possiblyPause() called from the main loop, if [pauseRequested] is true, [status] is set to PAUSED, - // [hasPaused] is signalled, and execution pauses until [pauseRequested] becomes false, either by being set or by - // the [pauseMillis] timeout elapsing. [pauseRequested] is checked when [shouldResume] is signalled. - // Resuming: - // - In resume(), [pauseRequested] is set to false, [shouldResume] is signalled, and execution waits for [status] to - // change to something other than PAUSED, with the condition checked when [shouldResume] is signalled. - // - In possiblyPause(), when [shouldResume] is signalled, if [pauseRequested] has become false the pause loop ends, - // [status] is changed to STARTING and [shouldResume] is signalled. - private final Lock pauseLock = new ReentrantLock(); private final Condition hasPaused = pauseLock.newCondition(); private final Condition shouldResume = pauseLock.newCondition(); - // [pollRetryLock] and [isAwaitingRetry] is used when the Kafka consumer returns an OffsetOutOfRangeException and we // pause polling from Kafka for POLL_RETRY_MS before trying again. This allows us to signal the sleeping thread and // resume the main run loop in the case of a pause or stop request from a Jetty thread. private final Lock pollRetryLock = new ReentrantLock(); private final Condition isAwaitingRetry = pollRetryLock.newCondition(); - // [statusLock] is used to synchronize the Jetty thread calling stopGracefully() with the main run thread. It prevents // the main run thread from switching into a publishing state while the stopGracefully() thread thinks it's still in // a pre-publishing state. This is important because stopGracefully() will try to use the [stopRequested] flag to stop @@ -185,8 +153,31 @@ public class KinesisIndexTask extends SeekableStreamIndexTask // blocks until after stopGracefully() has set [stopRequested] and then does a final check on [stopRequested] before // transitioning to publishing state. private final Object statusLock = new Object(); - private final RowIngestionMeters rowIngestionMeters; + private ObjectMapper mapper; + + // The pause lock and associated conditions are to support coordination between the Jetty threads and the main + // ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully + // the ingestion loop has been stopped at the returned offsets and will not ingest any more data until resumed. The + // fields are used as follows (every step requires acquiring [pauseLock]): + // Pausing: + // - In pause(), [pauseRequested] is set to true and then execution waits for [status] to change to PAUSED, with the + // condition checked when [hasPaused] is signalled. + // - In possiblyPause() called from the main loop, if [pauseRequested] is true, [status] is set to PAUSED, + // [hasPaused] is signalled, and execution pauses until [pauseRequested] becomes false, either by being set or by + // the [pauseMillis] timeout elapsing. [pauseRequested] is checked when [shouldResume] is signalled. + // Resuming: + // - In resume(), [pauseRequested] is set to false, [shouldResume] is signalled, and execution waits for [status] to + // change to something other than PAUSED, with the condition checked when [shouldResume] is signalled. + // - In possiblyPause(), when [shouldResume] is signalled, if [pauseRequested] has become false the pause loop ends, + // [status] is changed to STARTING and [shouldResume] is signalled. + private volatile Appenderator appenderator = null; + private volatile FireDepartmentMetrics fireDepartmentMetrics = null; + private volatile DateTime startTime; + private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread) + private volatile Thread runThread = null; + private volatile boolean stopRequested = false; + private volatile boolean publishOnStop = false; private IngestionState ingestionState; private volatile boolean pauseRequested = false; @@ -255,7 +246,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception final LookupNodeService lookupNodeService = getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER) == null ? toolbox.getLookupNodeService() : - new LookupNodeService((String) getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER)); + new LookupNodeService(getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER)); final DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( toolbox.getDruidNode(), @@ -277,26 +268,34 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception appenderator = appenderator0; - final String topic = ioConfig.getStartPartitions().getId(); + final String topic = ioConfig.getStartPartitions().getStream(); // Start up, set up initial offsets. final Object restoredMetadata = driver.startJob(); if (restoredMetadata == null) { lastOffsets.putAll(ioConfig.getStartPartitions().getMap()); } else { + @SuppressWarnings("unchecked") final Map restoredMetadataMap = (Map) restoredMetadata; - final SeekableStreamPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( - restoredMetadataMap.get(METADATA_NEXT_PARTITIONS), - SeekableStreamPartitions.class - ); + final SeekableStreamPartitions restoredNextPartitions = toolbox + .getObjectMapper() + .convertValue( + restoredMetadataMap.get(METADATA_NEXT_PARTITIONS), + toolbox.getObjectMapper().getTypeFactory().constructParametrizedType( + SeekableStreamPartitions.class, + SeekableStreamPartitions.class, + String.class, + String.class + ) + ); lastOffsets.putAll(restoredNextPartitions.getMap()); // Sanity checks. - if (!restoredNextPartitions.getId().equals(ioConfig.getStartPartitions().getId())) { + if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) { throw new ISE( "WTF?! Restored stream[%s] but expected stream[%s]", - restoredNextPartitions.getId(), - ioConfig.getStartPartitions().getId() + restoredNextPartitions.getStream(), + ioConfig.getStartPartitions().getStream() ); } @@ -326,33 +325,28 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception } // Set up committer. - final Supplier committerSupplier = new Supplier() - { - @Override - public Committer get() - { - final Map snapshot = ImmutableMap.copyOf(lastOffsets); + final Supplier committerSupplier = () -> { + final Map snapshot = ImmutableMap.copyOf(lastOffsets); - return new Committer() + return new Committer() + { + @Override + public Object getMetadata() { - @Override - public Object getMetadata() - { - return ImmutableMap.of( - METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>( - ioConfig.getStartPartitions().getId(), - snapshot - ) - ); - } + return ImmutableMap.of( + METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>( + ioConfig.getStartPartitions().getStream(), + snapshot + ) + ); + } - @Override - public void run() - { - // Do nothing. - } - }; - } + @Override + public void run() + { + // Do nothing. + } + }; }; Set assignment = assignPartitions(recordSupplier, topic); @@ -488,9 +482,9 @@ public void run() if (isPersistRequired) { driver.persist(committerSupplier.get()); } - segmentsToMoveOut.entrySet().forEach(sequenceSegments -> driver.moveSegmentOut( - sequenceSegments.getKey(), - sequenceSegments.getValue().stream().collect(Collectors.toList()) + segmentsToMoveOut.forEach((key, value) -> driver.moveSegmentOut( + key, + new ArrayList(value) )); } catch (ParseException e) { @@ -536,7 +530,12 @@ public void run() final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { final SeekableStreamPartitions finalPartitions = toolbox.getObjectMapper().convertValue( ((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS), - SeekableStreamPartitions.class + toolbox.getObjectMapper().getTypeFactory().constructParametrizedType( + SeekableStreamPartitions.class, + SeekableStreamPartitions.class, + String.class, + String.class + ) ); // Sanity check, we should only be publishing things that match our desired end state. @@ -586,14 +585,7 @@ public void run() Joiner.on(", ").join( Iterables.transform( handedOff.getSegments(), - new Function() - { - @Override - public String apply(DataSegment input) - { - return input.getIdentifier(); - } - } + DataSegment::getIdentifier ) ), handedOff.getCommitMetadata() @@ -635,291 +627,369 @@ public String apply(DataSegment input) return success(); } - private Map getTaskCompletionReports(@Nullable String errorMsg) + private RecordSupplier getRecordSupplier() { - return TaskReport.buildTaskReports( - new IngestionStatsAndErrorsTaskReport( - getId(), - new IngestionStatsAndErrorsTaskReportData( - ingestionState, - getTaskCompletionUnparseableEvents(), - getTaskCompletionRowStats(), - errorMsg - ) - ) + int fetchThreads = tuningConfig.getFetchThreads() != null + ? tuningConfig.getFetchThreads() + : Math.max(1, ioConfig.getStartPartitions().getMap().size()); + + return new KinesisRecordSupplier( + ioConfig.getEndpoint(), + ioConfig.getAwsAccessKeyId(), + ioConfig.getAwsSecretAccessKey(), + ioConfig.getRecordsPerFetch(), + ioConfig.getFetchDelayMillis(), + fetchThreads, + ioConfig.getAwsAssumedRoleArn(), + ioConfig.getAwsExternalId(), + ioConfig.isDeaggregate(), + tuningConfig.getRecordBufferSize(), + tuningConfig.getRecordBufferOfferTimeout(), + tuningConfig.getRecordBufferFullWait(), + tuningConfig.getFetchSequenceNumberTimeout() ); } - private Map getTaskCompletionUnparseableEvents() + private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) { - Map unparseableEventsMap = Maps.newHashMap(); - List buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions( - savedParseExceptions + final int maxRowsInMemoryPerPartition = (tuningConfig.getMaxRowsInMemory() / + ioConfig.getStartPartitions().getMap().size()); + return Appenderators.createRealtime( + dataSchema, + tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), + metrics, + toolbox.getSegmentPusher(), + toolbox.getObjectMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMergerV9(), + toolbox.getQueryRunnerFactoryConglomerate(), + toolbox.getSegmentAnnouncer(), + toolbox.getEmitter(), + toolbox.getQueryExecutorService(), + toolbox.getCache(), + toolbox.getCacheConfig(), + toolbox.getCachePopulatorStats() ); - if (buildSegmentsParseExceptionMessages != null) { - unparseableEventsMap.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegmentsParseExceptionMessages); - } - return unparseableEventsMap; } - @Override - public boolean canRestore() + private StreamAppenderatorDriver newDriver( + final Appenderator appenderator, + final TaskToolbox toolbox, + final FireDepartmentMetrics metrics + ) { - return true; + return new StreamAppenderatorDriver( + appenderator, + new ActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + dataSchema, + (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck + ) + ), + toolbox.getSegmentHandoffNotifierFactory(), + new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), + toolbox.getDataSegmentKiller(), + toolbox.getObjectMapper(), + metrics + ); } - /** - * Authorizes action to be performed on this task's datasource - * - * @return authorization result - */ - private Access authorizationCheck(final HttpServletRequest req, Action action) + private Set assignPartitions(RecordSupplier recordSupplier, String topic) { - ResourceAction resourceAction = new ResourceAction( - new Resource(dataSchema.getDataSource(), ResourceType.DATASOURCE), - action - ); - - Access access = AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); - if (!access.isAllowed()) { - throw new ForbiddenException(access.toString()); + // Initialize consumer assignment. + final Set assignment = Sets.newHashSet(); + for (Map.Entry entry : lastOffsets.entrySet()) { + final String endOffset = endOffsets.get(entry.getKey()); + if (Record.END_OF_SHARD_MARKER.equals(endOffset) + || SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) + || KinesisSequenceNumber.of(entry.getValue()).compareTo(KinesisSequenceNumber.of(endOffset)) < 0) { + assignment.add(entry.getKey()); + } else if (entry.getValue().equals(endOffset)) { + log.info("Finished reading partition[%s].", entry.getKey()); + } else { + throw new ISE( + "WTF?! Cannot start from offset[%s] > endOffset[%s]", + entry.getValue(), + endOffset + ); + } } - return access; + assignPartitions(recordSupplier, topic, assignment); + + return assignment; } - private void handleParseException(ParseException pe, Record record) + private void seekToStartingRecords( + RecordSupplier recordSupplier, + String topic, + Set assignment, + TaskToolbox toolbox + ) { - if (pe.isFromPartiallyValidRow()) { - rowIngestionMeters.incrementProcessedWithError(); - } else { - rowIngestionMeters.incrementUnparseable(); - } + // Seek to starting offsets. + for (final String partition : assignment) { + final String offset = lastOffsets.get(partition); + final StreamPartition streamPartition = StreamPartition.of(topic, partition); - if (tuningConfig.isLogParseExceptions()) { - log.error( - pe, - "Encountered parse exception on row from partition[%s] sequenceNumber[%s]", - record.getPartitionId(), - record.getSequenceNumber() - ); - } + if (!tuningConfig.isSkipSequenceNumberAvailabilityCheck()) { + try { + String earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); + if (earliestSequenceNumber == null + || KinesisSequenceNumber.of(earliestSequenceNumber).compareTo(KinesisSequenceNumber.of(offset)) > 0) { + if (tuningConfig.isResetOffsetAutomatically()) { + log.info("Attempting to reset offsets automatically for all partitions"); + try { + sendResetRequestAndWait( + assignment.stream() + .collect(Collectors.toMap((x) -> new StreamPartition<>(topic, x), lastOffsets::get)), + toolbox + ); + } + catch (IOException e) { + throw new ISE(e, "Exception while attempting to automatically reset offsets"); + } + } else { + throw new ISE( + "Starting sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) and resetOffsetAutomatically is not enabled", + offset, + partition, + earliestSequenceNumber + ); + } + } + } + catch (TimeoutException e) { + throw new ISE(e, "Timeout while fetching earliest sequence number for partition [%s]", partition); + } + } - if (savedParseExceptions != null) { - savedParseExceptions.add(pe); - } + log.info("Seeking partition[%s] to sequenceNumber[%s].", partition, offset); - if (rowIngestionMeters.getUnparseable() + rowIngestionMeters.getProcessedWithError() - > tuningConfig.getMaxParseExceptions()) { - log.error("Max parse exceptions exceeded, terminating task..."); - throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); + // We will seek to and start reading from the last offset that we read on the previous run so that we can confirm + // that the sequenceNumbers match, but we will discard the event instead of indexing it so we don't read it twice. + recordSupplier.seek(streamPartition, offset); } } - @Override - public void stopGracefully() + /** + * Checks if the pauseRequested flag was set and if so blocks: + * a) if pauseMillis == PAUSE_FOREVER, until pauseRequested is cleared + * b) if pauseMillis != PAUSE_FOREVER, until pauseMillis elapses -or- pauseRequested is cleared + *

+ * If pauseMillis is changed while paused, the new pause timeout will be applied. This allows adjustment of the + * pause timeout (making a timed pause into an indefinite pause and vice versa is valid) without having to resume + * and ensures that the loop continues to stay paused without ingesting any new events. You will need to signal + * shouldResume after adjusting pauseMillis for the new value to take effect. + *

+ * Sets paused = true and signals paused so callers can be notified when the pause command has been accepted. + *

+ * Additionally, pauses if all partitions assignments have been read and pauseAfterRead flag is set. + * + * @return true if a pause request was handled, false otherwise + */ + private boolean possiblyPause(Set assignment) throws InterruptedException { - log.info("Stopping gracefully (status: [%s])", status); - stopRequested = true; - - synchronized (statusLock) { - if (status == Status.PUBLISHING) { - runThread.interrupt(); - return; + pauseLock.lockInterruptibly(); + try { + /* + if (ioConfig.isPauseAfterRead() && assignment.isEmpty()) { + pauseMillis = PAUSE_FOREVER; + pauseRequested = true; } - } + */ - try { - if (pauseLock.tryLock(LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { - try { - if (pauseRequested) { - pauseRequested = false; - shouldResume.signalAll(); + if (pauseRequested) { + status = Status.PAUSED; + long nanos = 0; + hasPaused.signalAll(); + + while (pauseRequested) { + if (pauseMillis == PAUSE_FOREVER) { + log.info("Pausing ingestion until resumed"); + shouldResume.await(); + } else { + if (pauseMillis > 0) { + log.info("Pausing ingestion for [%,d] ms", pauseMillis); + nanos = TimeUnit.MILLISECONDS.toNanos(pauseMillis); + pauseMillis = 0; + } + if (nanos <= 0L) { + pauseRequested = false; // timeout elapsed + } + nanos = shouldResume.awaitNanos(nanos); } } - finally { - pauseLock.unlock(); - } - } else { - log.warn("While stopping: failed to acquire pauseLock before timeout, interrupting run thread"); - runThread.interrupt(); - return; - } - if (pollRetryLock.tryLock(LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { - try { - isAwaitingRetry.signalAll(); - } - finally { - pollRetryLock.unlock(); - } - } else { - log.warn("While stopping: failed to acquire pollRetryLock before timeout, interrupting run thread"); - runThread.interrupt(); + status = Status.READING; + shouldResume.signalAll(); + log.info("Ingestion loop resumed"); + return true; } } - catch (Exception e) { - Throwables.propagate(e); + finally { + pauseLock.unlock(); } + + return false; } - @Override - public QueryRunner getQueryRunner(Query query) + private boolean withinMinMaxRecordTime(final InputRow row) { - if (appenderator == null) { - // Not yet initialized, no data yet, just return a noop runner. - return new NoopQueryRunner<>(); + final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent() + && ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp()); + + final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent() + && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp()); + + if (!Intervals.ETERNITY.contains(row.getTimestamp())) { + final String errorMsg = StringUtils.format( + "Encountered row with timestamp that cannot be represented as a long: [%s]", + row + ); + throw new ParseException(errorMsg); } - return new QueryRunner() - { - @Override - public Sequence run(final QueryPlus query, final Map responseContext) - { - return query.run(appenderator, responseContext); + if (log.isDebugEnabled()) { + if (beforeMinimumMessageTime) { + log.debug( + "CurrentTimeStamp[%s] is before MinimumMessageTime[%s]", + row.getTimestamp(), + ioConfig.getMinimumMessageTime().get() + ); + } else if (afterMaximumMessageTime) { + log.debug( + "CurrentTimeStamp[%s] is after MaximumMessageTime[%s]", + row.getTimestamp(), + ioConfig.getMaximumMessageTime().get() + ); } - }; - } + } - @POST - @Path("/stop") - public Response stop(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.WRITE); - stopGracefully(); - return Response.status(Response.Status.OK).build(); + return !beforeMinimumMessageTime && !afterMaximumMessageTime; } - @GET - @Path("/status") - @Produces(MediaType.APPLICATION_JSON) - public Status getStatusHTTP(@Context final HttpServletRequest req) + private void handleParseException(ParseException pe, Record record) { - authorizationCheck(req, Action.READ); - return status; - } + if (pe.isFromPartiallyValidRow()) { + rowIngestionMeters.incrementProcessedWithError(); + } else { + rowIngestionMeters.incrementUnparseable(); + } - public Status getStatus() - { - return status; + if (tuningConfig.isLogParseExceptions()) { + log.error( + pe, + "Encountered parse exception on row from partition[%s] sequenceNumber[%s]", + record.getPartitionId(), + record.getSequenceNumber() + ); + } + + if (savedParseExceptions != null) { + savedParseExceptions.add(pe); + } + + if (rowIngestionMeters.getUnparseable() + rowIngestionMeters.getProcessedWithError() + > tuningConfig.getMaxParseExceptions()) { + log.error("Max parse exceptions exceeded, terminating task..."); + throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); + } } - @GET - @Path("/offsets/current") - @Produces(MediaType.APPLICATION_JSON) - public Map getCurrentOffsets(@Context final HttpServletRequest req) + private static void assignPartitions( + final RecordSupplier recordSupplier, + final String topic, + final Set partitions + ) { - authorizationCheck(req, Action.READ); - return getCurrentOffsets(); + recordSupplier.assign(partitions.stream().map(x -> StreamPartition.of(topic, x)).collect(Collectors.toSet())); } - public Map getCurrentOffsets() + private Map getTaskCompletionReports(@Nullable String errorMsg) { - return lastOffsets; + return TaskReport.buildTaskReports( + new IngestionStatsAndErrorsTaskReport( + getId(), + new IngestionStatsAndErrorsTaskReportData( + ingestionState, + getTaskCompletionUnparseableEvents(), + getTaskCompletionRowStats(), + errorMsg + ) + ) + ); } - @GET - @Path("/offsets/end") - @Produces(MediaType.APPLICATION_JSON) - public Map getEndOffsetsHTTP(@Context final HttpServletRequest req) + private void sendResetRequestAndWait( + Map, String> outOfRangePartitions, + TaskToolbox taskToolbox + ) + throws IOException { - authorizationCheck(req, Action.READ); - return getEndOffsets(); + Map partitionOffsetMap = outOfRangePartitions + .entrySet().stream().collect(Collectors.toMap(x -> x.getKey().getPartitionId(), Map.Entry::getValue)); + + boolean result = taskToolbox + .getTaskActionClient() + .submit( + new ResetDataSourceMetadataAction( + getDataSource(), + new KinesisDataSourceMetadata( + new SeekableStreamPartitions<>( + ioConfig.getStartPartitions().getStream(), + partitionOffsetMap + ) + ) + ) + ); + + if (result) { + log.makeAlert("Resetting Kinesis offsets for datasource [%s]", getDataSource()) + .addData("partitions", partitionOffsetMap.keySet()) + .emit(); + // wait for being killed by supervisor + try { + pause(-1); + } + catch (InterruptedException e) { + throw new RuntimeException("Got interrupted while pausing task"); + } + } else { + log.makeAlert("Failed to send reset request for partitions [%s]", partitionOffsetMap.keySet()).emit(); + } } - public Map getEndOffsets() + private Map getTaskCompletionUnparseableEvents() { - return endOffsets; + Map unparseableEventsMap = Maps.newHashMap(); + List buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions( + savedParseExceptions + ); + if (buildSegmentsParseExceptionMessages != null) { + unparseableEventsMap.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegmentsParseExceptionMessages); + } + return unparseableEventsMap; } - @POST - @Path("/offsets/end") - @Consumes(MediaType.APPLICATION_JSON) - @Produces(MediaType.APPLICATION_JSON) - public Response setEndOffsetsHTTP( - Map offsets, - @QueryParam("resume") @DefaultValue("false") final boolean resume, - @Context final HttpServletRequest req - ) throws InterruptedException + private Map getTaskCompletionRowStats() { - authorizationCheck(req, Action.WRITE); - return setEndOffsets(offsets, resume); + Map metrics = Maps.newHashMap(); + metrics.put( + RowIngestionMeters.BUILD_SEGMENTS, + rowIngestionMeters.getTotals() + ); + return metrics; } - public Response setEndOffsets(Map offsets, final boolean resume) throws InterruptedException - { - if (offsets == null) { - return Response.status(Response.Status.BAD_REQUEST) - .entity("Request body must contain a map of { partition:endOffset }") - .build(); - } else if (!endOffsets.keySet().containsAll(offsets.keySet())) { - return Response.status(Response.Status.BAD_REQUEST) - .entity( - String.format( - "Request contains partitions not being handled by this task, my partitions: %s", - endOffsets.keySet() - ) - ) - .build(); - } - - pauseLock.lockInterruptibly(); - try { - if (!isPaused()) { - return Response.status(Response.Status.BAD_REQUEST) - .entity("Task must be paused before changing the end offsets") - .build(); - } - - for (Map.Entry entry : offsets.entrySet()) { - if (entry.getValue().compareTo(lastOffsets.get(entry.getKey())) < 0) { - return Response.status(Response.Status.BAD_REQUEST) - .entity( - String.format( - "End offset must be >= current offset for partition [%s] (current: %s)", - entry.getKey(), - lastOffsets.get(entry.getKey()) - ) - ) - .build(); - } - } - - endOffsets.putAll(offsets); - log.info("endOffsets changed to %s", endOffsets); - } - finally { - pauseLock.unlock(); - } - - if (resume) { - resume(); - } - - return Response.ok(endOffsets).build(); - } - - /** - * Signals the ingestion loop to pause. - * - * @param timeout how long to pause for before resuming in milliseconds, <= 0 means indefinitely - * - * @return one of the following Responses: 400 Bad Request if the task has started publishing; 202 Accepted if the - * method has timed out and returned before the task has paused; 200 OK with a map of the current partition offsets - * in the response body if the task successfully paused - */ - @POST - @Path("/pause") - @Produces(MediaType.APPLICATION_JSON) - public Response pauseHTTP( - @QueryParam("timeout") @DefaultValue("0") final long timeout, @Context final HttpServletRequest req - ) throws InterruptedException - { - authorizationCheck(req, Action.WRITE); - return pause(timeout); - } - - public Response pause(final long timeout) throws InterruptedException + public Response pause(final long timeout) throws InterruptedException { if (!(status == Status.PAUSED || status == Status.READING)) { return Response.status(Response.Status.BAD_REQUEST) @@ -966,407 +1036,325 @@ public Response pause(final long timeout) throws InterruptedException } } + private boolean isPaused() + { + return status == Status.PAUSED; + } + + public Map getCurrentOffsets() + { + return lastOffsets; + } + + @Override + public boolean canRestore() + { + return true; + } + + @Override + public QueryRunner getQueryRunner(Query query) + { + if (appenderator == null) { + // Not yet initialized, no data yet, just return a noop runner. + return new NoopQueryRunner<>(); + } + + return (query1, responseContext) -> query1.run(appenderator, responseContext); + } + @POST - @Path("/resume") - public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException + @Path("/stop") + public Response stop(@Context final HttpServletRequest req) { authorizationCheck(req, Action.WRITE); - resume(); + stopGracefully(); return Response.status(Response.Status.OK).build(); } - public void resume() throws InterruptedException + /** + * Authorizes action to be performed on this task's datasource + * + * @return authorization result + */ + private Access authorizationCheck(final HttpServletRequest req, Action action) { - pauseLock.lockInterruptibly(); + ResourceAction resourceAction = new ResourceAction( + new Resource(dataSchema.getDataSource(), ResourceType.DATASOURCE), + action + ); + + Access access = AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); + if (!access.isAllowed()) { + throw new ForbiddenException(access.toString()); + } + + return access; + } + + @Override + public void stopGracefully() + { + log.info("Stopping gracefully (status: [%s])", status); + stopRequested = true; + + synchronized (statusLock) { + if (status == Status.PUBLISHING) { + runThread.interrupt(); + return; + } + } + try { - pauseRequested = false; - shouldResume.signalAll(); + if (pauseLock.tryLock(LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { + try { + if (pauseRequested) { + pauseRequested = false; + shouldResume.signalAll(); + } + } + finally { + pauseLock.unlock(); + } + } else { + log.warn("While stopping: failed to acquire pauseLock before timeout, interrupting run thread"); + runThread.interrupt(); + return; + } - long nanos = TimeUnit.SECONDS.toNanos(5); - while (isPaused()) { - if (nanos <= 0L) { - throw new RuntimeException("Resume command was not accepted within 5 seconds"); + if (pollRetryLock.tryLock(LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { + try { + isAwaitingRetry.signalAll(); } - nanos = shouldResume.awaitNanos(nanos); + finally { + pollRetryLock.unlock(); + } + } else { + log.warn("While stopping: failed to acquire pollRetryLock before timeout, interrupting run thread"); + runThread.interrupt(); } } - finally { - pauseLock.unlock(); + catch (Exception e) { + Throwables.propagate(e); } } @GET - @Path("/time/start") + @Path("/status") @Produces(MediaType.APPLICATION_JSON) - public DateTime getStartTime(@Context final HttpServletRequest req) + public Status getStatusHTTP(@Context final HttpServletRequest req) { authorizationCheck(req, Action.READ); - return startTime; + return status; + } + + public Status getStatus() + { + return status; } @GET - @Path("/rowStats") + @Path("/offsets/current") @Produces(MediaType.APPLICATION_JSON) - public Response getRowStats( - @Context final HttpServletRequest req - ) + public Map getCurrentOffsets(@Context final HttpServletRequest req) { authorizationCheck(req, Action.READ); - Map returnMap = Maps.newHashMap(); - Map totalsMap = Maps.newHashMap(); - Map averagesMap = Maps.newHashMap(); - - totalsMap.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getTotals() - ); - averagesMap.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getMovingAverages() - ); - - returnMap.put("movingAverages", averagesMap); - returnMap.put("totals", totalsMap); - return Response.ok(returnMap).build(); + return getCurrentOffsets(); } @GET - @Path("/unparseableEvents") + @Path("/offsets/end") @Produces(MediaType.APPLICATION_JSON) - public Response getUnparseableEvents( - @Context final HttpServletRequest req - ) + public Map getEndOffsetsHTTP(@Context final HttpServletRequest req) { authorizationCheck(req, Action.READ); - List events = IndexTaskUtils.getMessagesFromSavedParseExceptions(savedParseExceptions); - return Response.ok(events).build(); + return getEndOffsets(); } - @VisibleForTesting - RowIngestionMeters getRowIngestionMeters() + public Map getEndOffsets() { - return rowIngestionMeters; + return endOffsets; } - @VisibleForTesting - Appenderator getAppenderator() + @POST + @Path("/offsets/end") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response setEndOffsetsHTTP( + Map offsets, + @QueryParam("resume") @DefaultValue("false") final boolean resume, + @Context final HttpServletRequest req + ) throws InterruptedException { - return appenderator; + authorizationCheck(req, Action.WRITE); + return setEndOffsets(offsets, resume); } - @VisibleForTesting - FireDepartmentMetrics getFireDepartmentMetrics() + public Response setEndOffsets(Map offsets, final boolean resume) throws InterruptedException { - return fireDepartmentMetrics; - } + if (offsets == null) { + return Response.status(Response.Status.BAD_REQUEST) + .entity("Request body must contain a map of { partition:endOffset }") + .build(); + } else if (!endOffsets.keySet().containsAll(offsets.keySet())) { + return Response.status(Response.Status.BAD_REQUEST) + .entity( + String.format( + "Request contains partitions not being handled by this task, my partitions: %s", + endOffsets.keySet() + ) + ) + .build(); + } - private boolean isPaused() - { - return status == Status.PAUSED; - } - - private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) - { - final int maxRowsInMemoryPerPartition = (tuningConfig.getMaxRowsInMemory() / - ioConfig.getStartPartitions().getMap().size()); - return Appenderators.createRealtime( - dataSchema, - tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), - metrics, - toolbox.getSegmentPusher(), - toolbox.getObjectMapper(), - toolbox.getIndexIO(), - toolbox.getIndexMergerV9(), - toolbox.getQueryRunnerFactoryConglomerate(), - toolbox.getSegmentAnnouncer(), - toolbox.getEmitter(), - toolbox.getQueryExecutorService(), - toolbox.getCache(), - toolbox.getCacheConfig(), - toolbox.getCachePopulatorStats() - ); - } - - private StreamAppenderatorDriver newDriver( - final Appenderator appenderator, - final TaskToolbox toolbox, - final FireDepartmentMetrics metrics - ) - { - return new StreamAppenderatorDriver( - appenderator, - new ActionBasedSegmentAllocator( - toolbox.getTaskActionClient(), - dataSchema, - (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( - schema.getDataSource(), - row.getTimestamp(), - schema.getGranularitySpec().getQueryGranularity(), - schema.getGranularitySpec().getSegmentGranularity(), - sequenceName, - previousSegmentId, - skipSegmentLineageCheck - ) - ), - toolbox.getSegmentHandoffNotifierFactory(), - new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), - toolbox.getDataSegmentKiller(), - toolbox.getObjectMapper(), - metrics - ); - } - - private RecordSupplier getRecordSupplier() - { - int fetchThreads = tuningConfig.getFetchThreads() != null - ? tuningConfig.getFetchThreads() - : Math.max(1, ioConfig.getStartPartitions().getMap().size()); - - return new KinesisRecordSupplier( - ioConfig.getEndpoint(), - ioConfig.getAwsAccessKeyId(), - ioConfig.getAwsSecretAccessKey(), - ioConfig.getRecordsPerFetch(), - ioConfig.getFetchDelayMillis(), - fetchThreads, - ioConfig.getAwsAssumedRoleArn(), - ioConfig.getAwsExternalId(), - ioConfig.isDeaggregate(), - tuningConfig.getRecordBufferSize(), - tuningConfig.getRecordBufferOfferTimeout(), - tuningConfig.getRecordBufferFullWait(), - tuningConfig.getFetchSequenceNumberTimeout() - ); - } - - private static void assignPartitions( - final RecordSupplier recordSupplier, - final String topic, - final Set partitions - ) - { - recordSupplier.assign(partitions.stream().map(x -> StreamPartition.of(topic, x)).collect(Collectors.toSet())); - } - - private Set assignPartitions(RecordSupplier recordSupplier, String topic) - { - // Initialize consumer assignment. - final Set assignment = Sets.newHashSet(); - for (Map.Entry entry : lastOffsets.entrySet()) { - final String endOffset = endOffsets.get(entry.getKey()); - if (Record.END_OF_SHARD_MARKER.equals(endOffset) - || SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) - || KinesisSequenceNumber.of(entry.getValue()).compareTo(KinesisSequenceNumber.of(endOffset)) < 0) { - assignment.add(entry.getKey()); - } else if (entry.getValue().equals(endOffset)) { - log.info("Finished reading partition[%s].", entry.getKey()); - } else { - throw new ISE( - "WTF?! Cannot start from offset[%s] > endOffset[%s]", - entry.getValue(), - endOffset - ); + pauseLock.lockInterruptibly(); + try { + if (!isPaused()) { + return Response.status(Response.Status.BAD_REQUEST) + .entity("Task must be paused before changing the end offsets") + .build(); } - } - - assignPartitions(recordSupplier, topic, assignment); - - return assignment; - } - - private void seekToStartingRecords( - RecordSupplier recordSupplier, - String topic, - Set assignment, - TaskToolbox toolbox - ) - { - // Seek to starting offsets. - for (final String partition : assignment) { - final String offset = lastOffsets.get(partition); - final StreamPartition streamPartition = StreamPartition.of(topic, partition); - if (!tuningConfig.isSkipSequenceNumberAvailabilityCheck()) { - try { - String earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); - if (earliestSequenceNumber == null - || KinesisSequenceNumber.of(earliestSequenceNumber).compareTo(KinesisSequenceNumber.of(offset)) > 0) { - if (tuningConfig.isResetOffsetAutomatically()) { - log.info("Attempting to reset offsets automatically for all partitions"); - try { - sendResetRequestAndWait( - assignment.stream() - .collect(Collectors.toMap((x) -> new StreamPartition<>(topic, x), lastOffsets::get)), - toolbox - ); - } - catch (IOException e) { - throw new ISE(e, "Exception while attempting to automatically reset offsets"); - } - } else { - throw new ISE( - "Starting sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) and resetOffsetAutomatically is not enabled", - offset, - partition, - earliestSequenceNumber - ); - } - } - } - catch (TimeoutException e) { - throw new ISE(e, "Timeout while fetching earliest sequence number for partition [%s]", partition); + for (Map.Entry entry : offsets.entrySet()) { + if (entry.getValue().compareTo(lastOffsets.get(entry.getKey())) < 0) { + return Response.status(Response.Status.BAD_REQUEST) + .entity( + String.format( + "End offset must be >= current offset for partition [%s] (current: %s)", + entry.getKey(), + lastOffsets.get(entry.getKey()) + ) + ) + .build(); } } - log.info("Seeking partition[%s] to sequenceNumber[%s].", partition, offset); + endOffsets.putAll(offsets); + log.info("endOffsets changed to %s", endOffsets); + } + finally { + pauseLock.unlock(); + } - // We will seek to and start reading from the last offset that we read on the previous run so that we can confirm - // that the sequenceNumbers match, but we will discard the event instead of indexing it so we don't read it twice. - recordSupplier.seek(streamPartition, offset); + if (resume) { + resume(); } + + return Response.ok(endOffsets).build(); } - /** - * Checks if the pauseRequested flag was set and if so blocks: - * a) if pauseMillis == PAUSE_FOREVER, until pauseRequested is cleared - * b) if pauseMillis != PAUSE_FOREVER, until pauseMillis elapses -or- pauseRequested is cleared - *

- * If pauseMillis is changed while paused, the new pause timeout will be applied. This allows adjustment of the - * pause timeout (making a timed pause into an indefinite pause and vice versa is valid) without having to resume - * and ensures that the loop continues to stay paused without ingesting any new events. You will need to signal - * shouldResume after adjusting pauseMillis for the new value to take effect. - *

- * Sets paused = true and signals paused so callers can be notified when the pause command has been accepted. - *

- * Additionally, pauses if all partitions assignments have been read and pauseAfterRead flag is set. - * - * @return true if a pause request was handled, false otherwise - */ - private boolean possiblyPause(Set assignment) throws InterruptedException + public void resume() throws InterruptedException { pauseLock.lockInterruptibly(); try { - /* - if (ioConfig.isPauseAfterRead() && assignment.isEmpty()) { - pauseMillis = PAUSE_FOREVER; - pauseRequested = true; - } - */ - - if (pauseRequested) { - status = Status.PAUSED; - long nanos = 0; - hasPaused.signalAll(); + pauseRequested = false; + shouldResume.signalAll(); - while (pauseRequested) { - if (pauseMillis == PAUSE_FOREVER) { - log.info("Pausing ingestion until resumed"); - shouldResume.await(); - } else { - if (pauseMillis > 0) { - log.info("Pausing ingestion for [%,d] ms", pauseMillis); - nanos = TimeUnit.MILLISECONDS.toNanos(pauseMillis); - pauseMillis = 0; - } - if (nanos <= 0L) { - pauseRequested = false; // timeout elapsed - } - nanos = shouldResume.awaitNanos(nanos); - } + long nanos = TimeUnit.SECONDS.toNanos(5); + while (isPaused()) { + if (nanos <= 0L) { + throw new RuntimeException("Resume command was not accepted within 5 seconds"); } - - status = Status.READING; - shouldResume.signalAll(); - log.info("Ingestion loop resumed"); - return true; + nanos = shouldResume.awaitNanos(nanos); } } finally { pauseLock.unlock(); } - - return false; } - private void sendResetRequestAndWait( - Map, String> outOfRangePartitions, - TaskToolbox taskToolbox - ) - throws IOException + /** + * Signals the ingestion loop to pause. + * + * @param timeout how long to pause for before resuming in milliseconds, <= 0 means indefinitely + * + * @return one of the following Responses: 400 Bad Request if the task has started publishing; 202 Accepted if the + * method has timed out and returned before the task has paused; 200 OK with a map of the current partition offsets + * in the response body if the task successfully paused + */ + @POST + @Path("/pause") + @Produces(MediaType.APPLICATION_JSON) + public Response pauseHTTP( + @QueryParam("timeout") @DefaultValue("0") final long timeout, @Context final HttpServletRequest req + ) throws InterruptedException { - Map partitionOffsetMap = outOfRangePartitions - .entrySet().stream().collect(Collectors.toMap(x -> x.getKey().getPartitionId(), Map.Entry::getValue)); + authorizationCheck(req, Action.WRITE); + return pause(timeout); + } - boolean result = taskToolbox - .getTaskActionClient() - .submit( - new ResetDataSourceMetadataAction( - getDataSource(), - new KinesisDataSourceMetadata( - new SeekableStreamPartitions(ioConfig.getStartPartitions().getId(), partitionOffsetMap) - ) - ) - ); + @POST + @Path("/resume") + public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException + { + authorizationCheck(req, Action.WRITE); + resume(); + return Response.status(Response.Status.OK).build(); + } - if (result) { - log.makeAlert("Resetting Kinesis offsets for datasource [%s]", getDataSource()) - .addData("partitions", partitionOffsetMap.keySet()) - .emit(); - // wait for being killed by supervisor - try { - pause(-1); - } - catch (InterruptedException e) { - throw new RuntimeException("Got interrupted while pausing task"); - } - } else { - log.makeAlert("Failed to send reset request for partitions [%s]", partitionOffsetMap.keySet()).emit(); - } + @GET + @Path("/time/start") + @Produces(MediaType.APPLICATION_JSON) + public DateTime getStartTime(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return startTime; } - private boolean withinMinMaxRecordTime(final InputRow row) + @GET + @Path("/rowStats") + @Produces(MediaType.APPLICATION_JSON) + public Response getRowStats( + @Context final HttpServletRequest req + ) { - final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent() - && ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp()); + authorizationCheck(req, Action.READ); + Map returnMap = Maps.newHashMap(); + Map totalsMap = Maps.newHashMap(); + Map averagesMap = Maps.newHashMap(); - final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent() - && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp()); + totalsMap.put( + RowIngestionMeters.BUILD_SEGMENTS, + rowIngestionMeters.getTotals() + ); + averagesMap.put( + RowIngestionMeters.BUILD_SEGMENTS, + rowIngestionMeters.getMovingAverages() + ); - if (!Intervals.ETERNITY.contains(row.getTimestamp())) { - final String errorMsg = StringUtils.format( - "Encountered row with timestamp that cannot be represented as a long: [%s]", - row - ); - throw new ParseException(errorMsg); - } + returnMap.put("movingAverages", averagesMap); + returnMap.put("totals", totalsMap); + return Response.ok(returnMap).build(); + } - if (log.isDebugEnabled()) { - if (beforeMinimumMessageTime) { - log.debug( - "CurrentTimeStamp[%s] is before MinimumMessageTime[%s]", - row.getTimestamp(), - ioConfig.getMinimumMessageTime().get() - ); - } else if (afterMaximumMessageTime) { - log.debug( - "CurrentTimeStamp[%s] is after MaximumMessageTime[%s]", - row.getTimestamp(), - ioConfig.getMaximumMessageTime().get() - ); - } - } + @GET + @Path("/unparseableEvents") + @Produces(MediaType.APPLICATION_JSON) + public Response getUnparseableEvents( + @Context final HttpServletRequest req + ) + { + authorizationCheck(req, Action.READ); + List events = IndexTaskUtils.getMessagesFromSavedParseExceptions(savedParseExceptions); + return Response.ok(events).build(); + } - return !beforeMinimumMessageTime && !afterMaximumMessageTime; + @VisibleForTesting + RowIngestionMeters getRowIngestionMeters() + { + return rowIngestionMeters; } - private Map getTaskCompletionRowStats() + @VisibleForTesting + Appenderator getAppenderator() { - Map metrics = Maps.newHashMap(); - metrics.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getTotals() - ); - return metrics; + return appenderator; + } + + @VisibleForTesting + FireDepartmentMetrics getFireDepartmentMetrics() + { + return fireDepartmentMetrics; } @Override diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java deleted file mode 100644 index c97f6d2c5b4c..000000000000 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.kinesis; - -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.stats.RowIngestionMeters; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.segment.realtime.appenderator.Appenderator; - -import javax.ws.rs.core.Response; -import java.util.Map; - -/** - * Kinesis indexing task runner supporting incremental segments publishing - */ -public class KinesisIndexTaskRunner implements SeekableStreamIndexTaskRunner -{ - private static final EmittingLogger log = new EmittingLogger(KinesisIndexTask.class); - - - - @Override - public Appenderator getAppenderator() - { - return null; - } - - @Override - public TaskStatus run(TaskToolbox toolbox) - { - return null; - } - - @Override - public void stopGracefully() - { - - } - - @Override - public RowIngestionMeters getRowIngestionMeters() - { - return null; - } - - @Override - public SeekableStreamIndexTask.Status getStatus() - { - return null; - } - - @Override - public Map getCurrentOffsets() - { - return null; - } - - @Override - public Map getEndOffsets() - { - return null; - } - - @Override - public Response setEndOffsets(Map offsets, boolean finish) throws InterruptedException - { - return null; - } - - @Override - public Response pause() throws InterruptedException - { - return null; - } - - @Override - public void resume() throws InterruptedException - { - - } -} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index de8628fee808..48b6cb45815b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -22,7 +22,9 @@ import com.amazonaws.ClientConfiguration; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider; -import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.client.builder.AwsClientBuilder; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder; import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxyFactory; import com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisProxyFactory; @@ -31,7 +33,8 @@ import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; import com.amazonaws.services.kinesis.model.ResourceNotFoundException; import com.amazonaws.services.kinesis.model.ShardIteratorType; -import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient; +import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder; +import com.amazonaws.util.AwsHostNameUtils; import com.google.common.collect.ImmutableSet; import org.apache.druid.common.aws.AWSCredentialsUtils; import org.apache.druid.indexing.kinesis.aws.ConstructibleAWSCredentialsConfig; @@ -42,6 +45,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; +import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; @@ -136,7 +140,7 @@ private Runnable getRecordRunnable() log.info("shardIterator[%s] has been closed and has no more records", streamPartition.getPartitionId()); // add an end-of-shard marker so caller knows this shard is closed - Record endOfShardRecord = new Record( + Record endOfShardRecord = new Record<>( streamPartition.getStreamName(), streamPartition.getPartitionId(), Record.END_OF_SHARD_MARKER, null ); @@ -273,7 +277,9 @@ public KinesisRecordSupplier( STSAssumeRoleSessionCredentialsProvider.Builder builder = new STSAssumeRoleSessionCredentialsProvider .Builder(awsAssumedRoleArn, String.format("druid-kinesis-%s", UUID.randomUUID().toString())) - .withStsClient(new AWSSecurityTokenServiceClient(awsCredentialsProvider)); + .withStsClient(AWSSecurityTokenServiceClientBuilder.standard() + .withCredentials(awsCredentialsProvider) + .build()); if (awsExternalId != null) { builder.withExternalId(awsExternalId); @@ -281,9 +287,14 @@ public KinesisRecordSupplier( awsCredentialsProvider = builder.build(); } - - AmazonKinesisClient kinesisClient = new AmazonKinesisClient(awsCredentialsProvider, new ClientConfiguration()); - kinesisClient.setEndpoint(endpoint); + AmazonKinesis kinesisClient = AmazonKinesisClientBuilder.standard() + .withCredentials(awsCredentialsProvider) + .withClientConfiguration(new ClientConfiguration()) + .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration( + endpoint, + AwsHostNameUtils.parseRegion(endpoint, null) + )) + .build(); kinesisProxyFactory = new KinesisProxyFactory(awsCredentialsProvider, kinesisClient); records = new LinkedBlockingQueue<>(recordBufferSize); @@ -304,7 +315,7 @@ public void assign(Set> collection) { checkIfClosed(); - collection.stream().forEach( + collection.forEach( streamPartition -> partitionResources.putIfAbsent( streamPartition, new PartitionResource(streamPartition, getKinesisProxy(streamPartition.getStreamName()), scheduledExec) @@ -356,6 +367,7 @@ public Collection> getAssignment() return partitionResources.keySet(); } + @Nullable @Override public Record poll(long timeout) { @@ -400,10 +412,10 @@ public String getEarliestSequenceNumber(StreamPartition partition) throw return getSequenceNumberInternal(partition, ShardIteratorType.TRIM_HORIZON); } - //TODO: remove in the future @Override public String position(StreamPartition partition) { + // only needed in calcLag, which is not supported in Kinesis throw new UnsupportedOperationException("position in KinesisRecordSupplier not supported"); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java index cd1a3a70c180..867294912592 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java @@ -50,11 +50,6 @@ public enum KinesisRegion SA_EAST_1, US_GOV_WEST_1; - public String getEndpoint() - { - return StringUtils.format("kinesis.%s.amazonaws.com%s", toString(), toString().startsWith("cn-") ? ".cn" : ""); - } - @JsonCreator public static KinesisRegion fromString(String value) { @@ -65,15 +60,20 @@ public static KinesisRegion fromString(String value) .orElseThrow(() -> new IAE("Region must be one of: %s", getNames())); } + private static List getNames() + { + return EnumSet.allOf(KinesisRegion.class).stream().map(KinesisRegion::toString).collect(Collectors.toList()); + } + + public String getEndpoint() + { + return StringUtils.format("kinesis.%s.amazonaws.com%s", toString(), toString().startsWith("cn-") ? ".cn" : ""); + } + @Override @JsonValue public String toString() { return StringUtils.toLowerCase(name()).replace('_', '-'); } - - private static List getNames() - { - return EnumSet.allOf(KinesisRegion.class).stream().map(KinesisRegion::toString).collect(Collectors.toList()); - } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java index 02b91f7cedab..65bb316bd5c1 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java @@ -24,6 +24,7 @@ import javax.validation.constraints.NotNull; import java.math.BigInteger; +import java.util.Objects; public class KinesisSequenceNumber extends SequenceNumber { @@ -33,12 +34,7 @@ public class KinesisSequenceNumber extends SequenceNumber private KinesisSequenceNumber(@NotNull String sequenceNumber, boolean useExclusive, boolean isExclusive) { super(sequenceNumber, useExclusive, isExclusive); - this.intSequence = sequenceNumber.equals("") ? new BigInteger("-1") : new BigInteger(sequenceNumber); - } - - public BigInteger getBigInteger() - { - return intSequence; + this.intSequence = "".equals(sequenceNumber) ? new BigInteger("-1") : new BigInteger(sequenceNumber); } public static KinesisSequenceNumber of(String sequenceNumber) @@ -51,6 +47,11 @@ public static KinesisSequenceNumber of(String sequenceNumber, boolean useExclusi return new KinesisSequenceNumber(sequenceNumber, useExclusive, isExclusive); } + public BigInteger getBigInteger() + { + return intSequence; + } + @Override public boolean equals(Object o) { @@ -62,8 +63,14 @@ public boolean equals(Object o) @Override - public int compareTo(SequenceNumber o) + public int compareTo(@NotNull SequenceNumber o) { return this.intSequence.compareTo(new BigInteger(o.get())); } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), intSequence); + } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java index 83d83cccf908..4032518e0b69 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java @@ -120,7 +120,7 @@ public KinesisTuningConfig copyOf() getMaxRowsPerSegment(), getIntermediatePersistPeriod(), getBasePersistDirectory(), - getMaxPendingPersists(), + 0, getIndexSpec(), true, isReportParseExceptions(), @@ -178,7 +178,7 @@ public KinesisTuningConfig withBasePersistDirectory(File dir) getMaxRowsPerSegment(), getIntermediatePersistPeriod(), dir, - getMaxPendingPersists(), + 0, getIndexSpec(), true, isReportParseExceptions(), @@ -205,7 +205,7 @@ public KinesisTuningConfig withMaxRowsInMemory(int rows) getMaxRowsPerSegment(), getIntermediatePersistPeriod(), getBasePersistDirectory(), - getMaxPendingPersists(), + 0, getIndexSpec(), true, isReportParseExceptions(), @@ -266,9 +266,9 @@ public int hashCode() getMaxRowsPerSegment(), getIntermediatePersistPeriod(), getBasePersistDirectory(), - getMaxPendingPersists(), + 0, getIndexSpec(), - getBuildV9Directly(), + true, isReportParseExceptions(), getHandoffConditionTimeout(), isResetOffsetAutomatically(), @@ -294,7 +294,7 @@ public String toString() ", maxRowsPerSegment=" + getMaxRowsPerSegment() + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + - ", maxPendingPersists=" + getMaxPendingPersists() + + ", maxPendingPersists=" + 0 + ", indexSpec=" + getIndexSpec() + ", reportParseExceptions=" + isReportParseExceptions() + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 870258a820d1..f968e99cc650 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -49,7 +49,6 @@ import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.emitter.EmittingLogger; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -65,16 +64,17 @@ import java.util.concurrent.TimeoutException; /** - * Supervisor responsible for managing the KafkaIndexTasks for a single dataSource. At a high level, the class accepts a + * Supervisor responsible for managing the KinesisIndexTask for a single dataSource. At a high level, the class accepts a * {@link KinesisSupervisorSpec} which includes the Kafka topic and configuration as well as an ingestion spec which will * be used to generate the indexing tasks. The run loop periodically refreshes its view of the Kafka topic's partitions * and the list of running indexing tasks and ensures that all partitions are being read from and that there are enough * tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of - * Kafka offsets. + * Kinesis sequences. + *

+ * the Kinesis supervisor does not yet support incremental handoff and emitLag */ public class KinesisSupervisor extends SeekableStreamSupervisor { - private static final EmittingLogger log = new EmittingLogger(KinesisSupervisor.class); private static final String NOT_SET = ""; private final KinesisSupervisorSpec spec; @@ -132,8 +132,8 @@ protected SeekableStreamIOConfig createIoConfig( KinesisSupervisorIOConfig ioConfig = (KinesisSupervisorIOConfig) ioConfigg; return new KinesisIOConfig( baseSequenceName, - new SeekableStreamPartitions(ioConfig.getStream(), startPartitions), - new SeekableStreamPartitions(ioConfig.getStream(), endPartitions), + new SeekableStreamPartitions<>(ioConfig.getStream(), startPartitions), + new SeekableStreamPartitions<>(ioConfig.getStream(), endPartitions), true, true, // should pause after reading otherwise the task may complete early which will confuse the supervisor minimumMessageTime, @@ -205,7 +205,7 @@ protected RecordSupplier setupRecordSupplier() @Override protected void scheduleReporting(ScheduledExecutorService reportingExec) { - // TODO: Implement this for Kinesis which uses approximate time from latest instead of offset lag + // Implement this for Kinesis which uses approximate time from latest instead of offset lag /* reportingExec.scheduleAtFixedRate( computeAndEmitLag(taskClient), @@ -334,7 +334,7 @@ protected int getNoticesQueueSize() } @Override - protected boolean checkSequenceAvailability(String partition, @NotNull String sequenceFromMetadata) + protected boolean checkSequenceAvailability(@NotNull String partition, @NotNull String sequenceFromMetadata) throws TimeoutException { String earliestSequence = super.getOffsetFromStreamForPartition(partition, true); @@ -343,7 +343,7 @@ protected boolean checkSequenceAvailability(String partition, @NotNull String se } -// TODO: Implement this for Kinesis which uses approximate time from latest instead of offset lag +// Implement this for Kinesis which uses approximate time from latest instead of offset lag /* private Runnable computeAndEmitLag(final KinesisIndexTaskClient taskClient) { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java index 6060729e7aca..0854a88ecb49 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java @@ -96,12 +96,6 @@ public KinesisSupervisorIOConfig( this.deaggregate = deaggregate; } - @JsonProperty - public String getStream() - { - return getId(); - } - @JsonProperty public String getEndpoint() { @@ -175,4 +169,10 @@ public String toString() '}'; } + @JsonProperty + public String getStream() + { + return getId(); + } + } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java index 542e912b6924..18b4feb0c067 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -136,18 +136,6 @@ public String toString() '}'; } - @Override - public KinesisSupervisorSpec createSuspendedSpec() - { - return toggleSuspend(true); - } - - @Override - public KinesisSupervisorSpec createRunningSpec() - { - return toggleSuspend(false); - } - @Override @JsonProperty public KinesisSupervisorTuningConfig getTuningConfig() @@ -162,6 +150,12 @@ public KinesisSupervisorIOConfig getIoConfig() return (KinesisSupervisorIOConfig) super.getIoConfig(); } + @Override + public KinesisSupervisorSpec createSuspendedSpec() + { + return toggleSuspend(true); + } + @Override protected KinesisSupervisorSpec toggleSuspend(boolean suspend) { @@ -181,4 +175,10 @@ protected KinesisSupervisorSpec toggleSuspend(boolean suspend) rowIngestionMetersFactory ); } + + @Override + public KinesisSupervisorSpec createRunningSpec() + { + return toggleSuspend(false); + } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index e56237db2e36..7693f1edd260 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -141,9 +141,9 @@ public String toString() ", maxRowsPerSegment=" + getMaxRowsPerSegment() + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + - ", maxPendingPersists=" + getMaxPendingPersists() + + ", maxPendingPersists=" + 0 + ", indexSpec=" + getIndexSpec() + - ", buildV9Directly=" + getBuildV9Directly() + + ", buildV9Directly=" + true + ", reportParseExceptions=" + isReportParseExceptions() + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java index ee8dde9b4b63..2bc15b3ce01c 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java @@ -42,6 +42,7 @@ import org.apache.druid.java.util.http.client.response.FullResponseHolder; import org.easymock.Capture; import org.easymock.CaptureType; +import org.easymock.EasyMock; import org.easymock.EasyMockSupport; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -64,11 +65,6 @@ import java.util.List; import java.util.Map; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.capture; -import static org.easymock.EasyMock.eq; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.reset; @RunWith(Parameterized.class) public class KinesisIndexTaskClientTest extends EasyMockSupport @@ -116,17 +112,20 @@ public void setUp() headers = createMock(HttpHeaders.class); client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider); - expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) - .anyTimes(); - expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)) + .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)) + .andReturn(Optional.of(TaskStatus.running(TEST_ID))) + .anyTimes(); for (String testId : TEST_IDS) { - expect(taskInfoProvider.getTaskLocation(testId)) - .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) - .anyTimes(); - expect(taskInfoProvider.getTaskStatus(testId)) - .andReturn(Optional.of(TaskStatus.running(testId))) - .anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskLocation(testId)) + .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskStatus(testId)) + .andReturn(Optional.of(TaskStatus.running(testId))) + .anyTimes(); } } @@ -139,9 +138,11 @@ public void tearDown() @Test public void testNoTaskLocation() throws IOException { - reset(taskInfoProvider); - expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); - expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); + EasyMock.reset(taskInfoProvider); + EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)) + .andReturn(Optional.of(TaskStatus.running(TEST_ID))) + .anyTimes(); replayAll(); Assert.assertFalse(client.stop(TEST_ID, true)); @@ -164,10 +165,13 @@ public void testTaskNotRunnableException() expectedException.expect(IndexTaskClient.TaskNotRunnableException.class); expectedException.expectMessage("Aborting request because task [test-id] is not runnable"); - reset(taskInfoProvider); - expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) - .anyTimes(); - expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.failure(TEST_ID))).anyTimes(); + EasyMock.reset(taskInfoProvider); + EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)) + .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)) + .andReturn(Optional.of(TaskStatus.failure(TEST_ID))) + .anyTimes(); replayAll(); client.getCurrentOffsets(TEST_ID, true); @@ -180,12 +184,12 @@ public void testInternalServerError() expectedException.expect(RuntimeException.class); expectedException.expectMessage("org.apache.druid.java.util.common.IOE: Received status [500]"); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2); - expect( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2); + EasyMock.expect( httpClient.go( - anyObject(Request.class), - anyObject(FullResponseHandler.class), - eq(TEST_HTTP_TIMEOUT) + EasyMock.anyObject(Request.class), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) ) ).andReturn( Futures.immediateFuture(responseHolder) @@ -202,13 +206,13 @@ public void testBadRequest() expectedException.expect(IAE.class); expectedException.expectMessage("Received 400 Bad Request with body:"); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2); - expect(responseHolder.getContent()).andReturn(""); - expect( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2); + EasyMock.expect(responseHolder.getContent()).andReturn(""); + EasyMock.expect( httpClient.go( - anyObject(Request.class), - anyObject(FullResponseHandler.class), - eq(TEST_HTTP_TIMEOUT) + EasyMock.anyObject(Request.class), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) ) ).andReturn( Futures.immediateFuture(responseHolder) @@ -222,18 +226,18 @@ public void testBadRequest() @Test public void testTaskLocationMismatch() { - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) - .andReturn(HttpResponseStatus.OK); - expect(responseHolder.getResponse()).andReturn(response); - expect(responseHolder.getContent()).andReturn("") - .andReturn("{}"); - expect(response.headers()).andReturn(headers); - expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id"); - expect( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) + .andReturn(HttpResponseStatus.OK); + EasyMock.expect(responseHolder.getResponse()).andReturn(response); + EasyMock.expect(responseHolder.getContent()).andReturn("") + .andReturn("{}"); + EasyMock.expect(response.headers()).andReturn(headers); + EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id"); + EasyMock.expect( httpClient.go( - anyObject(Request.class), - anyObject(FullResponseHandler.class), - eq(TEST_HTTP_TIMEOUT) + EasyMock.anyObject(Request.class), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) ) ).andReturn( Futures.immediateFuture(responseHolder) @@ -250,9 +254,13 @@ public void testTaskLocationMismatch() public void testGetCurrentOffsets() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -279,15 +287,19 @@ public void testGetCurrentOffsetsWithRetry() throws Exception client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 3); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) - .andReturn(HttpResponseStatus.OK).times(1); - expect(responseHolder.getContent()).andReturn("").times(2) - .andReturn("{\"0\":1, \"1\":10}"); - expect(responseHolder.getResponse()).andReturn(response).times(2); - expect(response.headers()).andReturn(headers).times(2); - expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2); - - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) + .andReturn(HttpResponseStatus.OK).times(1); + EasyMock.expect(responseHolder.getContent()).andReturn("").times(2) + .andReturn("{\"0\":1, \"1\":10}"); + EasyMock.expect(responseHolder.getResponse()).andReturn(response).times(2); + EasyMock.expect(response.headers()).andReturn(headers).times(2); + EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2); + + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(3); @@ -319,17 +331,17 @@ public void testGetCurrentOffsetsWithExhaustedRetries() client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); - expect(responseHolder.getContent()).andReturn("").anyTimes(); - expect(responseHolder.getResponse()).andReturn(response).anyTimes(); - expect(response.headers()).andReturn(headers).anyTimes(); - expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).anyTimes(); + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); + EasyMock.expect(responseHolder.getContent()).andReturn("").anyTimes(); + EasyMock.expect(responseHolder.getResponse()).andReturn(response).anyTimes(); + EasyMock.expect(response.headers()).andReturn(headers).anyTimes(); + EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).anyTimes(); - expect( + EasyMock.expect( httpClient.go( - anyObject(Request.class), - anyObject(FullResponseHandler.class), - eq(TEST_HTTP_TIMEOUT) + EasyMock.anyObject(Request.class), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) ) ).andReturn(Futures.immediateFuture(responseHolder)).anyTimes(); replayAll(); @@ -342,9 +354,13 @@ public void testGetCurrentOffsetsWithExhaustedRetries() public void testGetEndOffsets() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -372,13 +388,17 @@ public void testGetStartTime() throws Exception DateTime now = DateTimes.nowUtc(); Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) - .andReturn(HttpResponseStatus.OK); - expect(responseHolder.getResponse()).andReturn(response); - expect(response.headers()).andReturn(headers); - expect(headers.get("X-Druid-Task-Id")).andReturn(null); - expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) + .andReturn(HttpResponseStatus.OK); + EasyMock.expect(responseHolder.getResponse()).andReturn(response); + EasyMock.expect(response.headers()).andReturn(headers); + EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(null); + EasyMock.expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(2); replayAll(); @@ -403,9 +423,13 @@ public void testGetStatus() throws Exception SeekableStreamIndexTask.Status status = SeekableStreamIndexTask.Status.READING; Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - expect(responseHolder.getContent()).andReturn(StringUtils.format("\"%s\"", status.toString())).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + EasyMock.expect(responseHolder.getContent()).andReturn(StringUtils.format("\"%s\"", status.toString())).anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -428,9 +452,13 @@ public void testGetStatus() throws Exception public void testPause() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); - expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); + EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -457,17 +485,29 @@ public void testPauseWithSubsequentGetOffsets() throws Exception Capture captured = Capture.newInstance(); Capture captured2 = Capture.newInstance(); Capture captured3 = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.ACCEPTED).times(2) - .andReturn(HttpResponseStatus.OK).times(2); - expect(responseHolder.getContent()).andReturn("\"PAUSED\"") - .andReturn("{\"0\":1, \"1\":10}").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.ACCEPTED).times(2) + .andReturn(HttpResponseStatus.OK).times(2); + EasyMock.expect(responseHolder.getContent()).andReturn("\"PAUSED\"") + .andReturn("{\"0\":1, \"1\":10}").anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); - expect(httpClient.go(capture(captured2), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured2), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); - expect(httpClient.go(capture(captured3), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured3), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); @@ -507,8 +547,12 @@ public void testPauseWithSubsequentGetOffsets() throws Exception public void testResume() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -531,8 +575,12 @@ public void testSetEndOffsets() throws Exception Map endOffsets = ImmutableMap.of("0", "15", "1", "120"); Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -556,8 +604,12 @@ public void testSetEndOffsetsAndResume() throws Exception Map endOffsets = ImmutableMap.of("0", "15", "1", "120"); Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -579,8 +631,12 @@ public void testSetEndOffsetsAndResume() throws Exception public void testStop() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -601,8 +657,12 @@ public void testStop() throws Exception public void testStopAndPublish() throws Exception { Capture captured = Capture.newInstance(); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -624,8 +684,12 @@ public void testStopAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -656,8 +720,12 @@ public void testResumeAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -688,9 +756,13 @@ public void testPauseAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -721,9 +793,13 @@ public void testGetStatusAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -755,9 +831,13 @@ public void testGetStartTimeAsync() throws Exception final DateTime now = DateTimes.nowUtc(); final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -788,9 +868,13 @@ public void testGetCurrentOffsetsAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -821,9 +905,13 @@ public void testGetEndOffsetsAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -855,8 +943,12 @@ public void testSetEndOffsetsAsync() throws Exception Map endOffsets = ImmutableMap.of("0", "15L", "1", "120L"); final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -894,8 +986,12 @@ public void testSetEndOffsetsAsyncWithResume() throws Exception Map endOffsets = ImmutableMap.of("0", "15L", "1", "120L"); final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - expect(httpClient.go(capture(captured), anyObject(FullResponseHandler.class), eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index e656877e10e6..3b6b858821ca 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -112,6 +112,7 @@ import org.apache.druid.query.Druids; import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryToolChest; @@ -180,20 +181,12 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import static org.apache.druid.query.QueryPlus.wrap; - -// TODO: improve helper methods like insertData(...) @RunWith(LocalstackDockerTestRunner.class) @LocalstackDockerProperties(services = {"kinesis"}) public class KinesisIndexTaskTest { static { - /* - * Need to disable CBOR protocol, see: - * https://github.com/mhart/kinesalite/blob/master/README.md#cbor-protocol-issues-with-the-java-sdk - */ TestUtils.setEnv("AWS_CBOR_DISABLE", "1"); - /* Disable SSL certificate checks for local testing */ if (Localstack.useSSL()) { TestUtils.disableSslCertChecking(); } @@ -1578,7 +1571,7 @@ public void testRestore() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - Record.END_OF_SHARD_MARKER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), true, null, @@ -2114,7 +2107,7 @@ public long countEvents(final Task task) .build(); List> results = - task.getQueryRunner(query).run(wrap(query), ImmutableMap.of()).toList(); + task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList(); return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index d13b4bf23e3a..560f80fac6ff 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -110,17 +110,6 @@ import java.util.concurrent.Executor; import java.util.stream.Collectors; -import static org.easymock.EasyMock.anyBoolean; -import static org.easymock.EasyMock.anyObject; -import static org.easymock.EasyMock.anyString; -import static org.easymock.EasyMock.capture; -import static org.easymock.EasyMock.eq; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.expectLastCall; -import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.reset; - -// TODO: improve helper methods like insertData(...) @RunWith(LocalstackDockerTestRunner.class) @LocalstackDockerProperties(services = {"kinesis"}) public class KinesisSupervisorTest extends EasyMockSupport @@ -295,16 +284,16 @@ public void testNoInitialState() throws Exception List res = insertData(kinesis, generateRecordsRequests(stream)); Capture captured = Capture.newInstance(); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -351,15 +340,15 @@ public void testMultiTask() throws Exception List res = insertData(kinesis, generateRecordsRequests(stream)); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); replayAll(); supervisor.start(); @@ -399,15 +388,15 @@ public void testReplicas() throws Exception List res = insertData(kinesis, generateRecordsRequests(stream)); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); replayAll(); supervisor.start(); @@ -464,15 +453,15 @@ public void testLateMessageRejectionPeriod() throws Exception List res = insertData(kinesis, generateRecordsRequests(stream)); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); replayAll(); supervisor.start(); @@ -504,15 +493,15 @@ public void testEarlyMessageRejectionPeriod() throws Exception List res = insertData(kinesis, generateRecordsRequests(stream)); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); replayAll(); supervisor.start(); @@ -549,10 +538,10 @@ public void testDatasourceMetadata() throws Exception List res = insertData(kinesis, generateRecordsRequests(stream)); Capture captured = Capture.newInstance(); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -563,7 +552,7 @@ public void testDatasourceMetadata() throws Exception ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); replayAll(); supervisor.start(); @@ -591,9 +580,9 @@ public void testBadMetadataOffsets() throws Exception List res = insertData(kinesis, generateRecordsRequests(stream)); - expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -710,30 +699,33 @@ public void testKillIncompatibleTasks() throws Exception List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) - .anyTimes(); - expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); - expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(false)); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); + EasyMock.expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(false)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); taskQueue.shutdown("id3"); - expect(taskQueue.add(anyObject(Task.class))).andReturn(true); + EasyMock.expect(taskQueue.add(EasyMock.anyObject(Task.class))).andReturn(true); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of( @@ -743,9 +735,9 @@ public void testKillIncompatibleTasks() throws Exception getSequenceNumber(res, shardId1, 0) )); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(2); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(2); replayAll(); supervisor.start(); @@ -843,44 +835,47 @@ public void testKillBadPartitionAssignment() throws Exception List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); - expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).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(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); - expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); - expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) - .anyTimes(); - expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); - expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); - expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); + EasyMock.expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); + EasyMock.expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); + EasyMock.expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of(shardId1, getSequenceNumber(res, shardId1, 0))); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) - .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)); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(1); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(1); + + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); taskQueue.shutdown("id4"); taskQueue.shutdown("id5"); replayAll(); @@ -898,19 +893,22 @@ public void testRequeueTaskWhenFailed() throws Exception List res = insertData(kinesis, generateRecordsRequests(stream)); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) - .anyTimes(); - expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(4); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put( @@ -925,14 +923,14 @@ public void testRequeueTaskWhenFailed() throws Exception shardId0, getSequenceNumber(res, shardId0, 0) )); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .anyTimes(); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .anyTimes(); - - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .anyTimes(); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .anyTimes(); + + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -943,13 +941,15 @@ public void testRequeueTaskWhenFailed() throws Exception // test that running the main loop again checks the status of the tasks that were created and does nothing if they // are all still running - reset(taskStorage); - expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + EasyMock.reset(taskStorage); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - replay(taskStorage); + EasyMock.replay(taskStorage); supervisor.runInternal(); verifyAll(); @@ -958,18 +958,21 @@ public void testRequeueTaskWhenFailed() throws Exception Capture aNewTaskCapture = Capture.newInstance(); List imStillAlive = tasks.subList(0, 3); KinesisIndexTask iHaveFailed = (KinesisIndexTask) tasks.get(3); - reset(taskStorage); - reset(taskQueue); - expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes(); + EasyMock.reset(taskStorage); + EasyMock.reset(taskQueue); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes(); for (Task task : imStillAlive) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); - expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); - expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); - replay(taskStorage); - replay(taskQueue); + EasyMock.expect(taskStorage.getStatus(iHaveFailed.getId())) + .andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); + EasyMock.expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.capture(aNewTaskCapture))).andReturn(true); + EasyMock.replay(taskStorage); + EasyMock.replay(taskQueue); supervisor.runInternal(); verifyAll(); @@ -1013,16 +1016,17 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception List existingTasks = ImmutableList.of(id1); Capture captured = Capture.newInstance(); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); - expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync("id1")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) @@ -1035,11 +1039,11 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception shardId0, getSequenceNumber(res, shardId0, 0) )); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(2); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(2); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1053,29 +1057,33 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception String runningTaskId = captured.getValue().getId(); Capture aNewTaskCapture = Capture.newInstance(); KinesisIndexTask iHaveFailed = (KinesisIndexTask) existingTasks.get(0); - reset(taskStorage); - reset(taskQueue); - reset(taskClient); + EasyMock.reset(taskStorage); + EasyMock.reset(taskQueue); + EasyMock.reset(taskClient); // for the newly created replica task - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .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.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); - expect(taskStorage.getStatus(runningTaskId)).andReturn(Optional.of(TaskStatus.running(runningTaskId))).anyTimes(); - expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); - expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); - expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes(); - expect(taskQueue.add(capture(aNewTaskCapture))).andReturn(true); - replay(taskStorage); - replay(taskQueue); - replay(taskClient); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(2); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes(); + EasyMock.expect(taskStorage.getStatus(iHaveFailed.getId())) + .andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); + EasyMock.expect(taskStorage.getStatus(runningTaskId)) + .andReturn(Optional.of(TaskStatus.running(runningTaskId))) + .anyTimes(); + EasyMock.expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); + EasyMock.expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(runningTaskId)) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.capture(aNewTaskCapture))).andReturn(true); + EasyMock.replay(taskStorage); + EasyMock.replay(taskQueue); + EasyMock.replay(taskClient); supervisor.runInternal(); verifyAll(); @@ -1106,20 +1114,23 @@ public void testQueueNextTasksOnSuccess() throws Exception List res = insertData(kinesis, generateRecordsRequests(stream)); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) - .anyTimes(); - expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(4); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1129,12 +1140,15 @@ public void testQueueNextTasksOnSuccess() throws Exception List tasks = captured.getValues(); - reset(taskStorage); - reset(taskClient); + EasyMock.reset(taskStorage); + EasyMock.reset(taskClient); - expect(taskClient.getStatusAsync(anyString())).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) - .anyTimes(); - expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of( shardId1, @@ -1148,20 +1162,22 @@ public void testQueueNextTasksOnSuccess() throws Exception getSequenceNumber(res, shardId1, 0) )); // there would be 4 tasks, 2 for each task group - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); - - expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); + + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - replay(taskStorage); - replay(taskClient); + EasyMock.replay(taskStorage); + EasyMock.replay(taskClient); supervisor.runInternal(); verifyAll(); @@ -1172,21 +1188,25 @@ public void testQueueNextTasksOnSuccess() throws Exception Capture shutdownTaskIdCapture = Capture.newInstance(); List imStillRunning = tasks.subList(1, 4); KinesisIndexTask iAmSuccess = (KinesisIndexTask) tasks.get(0); - reset(taskStorage); - reset(taskQueue); - reset(taskClient); - expect(taskStorage.getActiveTasks()).andReturn(imStillRunning).anyTimes(); + EasyMock.reset(taskStorage); + EasyMock.reset(taskQueue); + EasyMock.reset(taskClient); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(imStillRunning).anyTimes(); for (Task task : imStillRunning) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - expect(taskStorage.getStatus(iAmSuccess.getId())).andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId()))); - expect(taskStorage.getTask(iAmSuccess.getId())).andReturn(Optional.of((Task) iAmSuccess)).anyTimes(); - expect(taskQueue.add(capture(newTasksCapture))).andReturn(true).times(2); - expect(taskClient.stopAsync(capture(shutdownTaskIdCapture), eq(false))).andReturn(Futures.immediateFuture(true)); - replay(taskStorage); - replay(taskQueue); - replay(taskClient); + EasyMock.expect(taskStorage.getStatus(iAmSuccess.getId())) + .andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId()))); + EasyMock.expect(taskStorage.getTask(iAmSuccess.getId())).andReturn(Optional.of((Task) iAmSuccess)).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.capture(newTasksCapture))).andReturn(true).times(2); + EasyMock.expect(taskClient.stopAsync(EasyMock.capture(shutdownTaskIdCapture), EasyMock.eq(false))) + .andReturn(Futures.immediateFuture(true)); + EasyMock.replay(taskStorage); + EasyMock.replay(taskQueue); + EasyMock.replay(taskClient); supervisor.runInternal(); verifyAll(); @@ -1205,17 +1225,17 @@ public void testBeginPublishAndQueueNextTasks() throws Exception List res = insertData(kinesis, generateRecordsRequests(stream)); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(4); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1228,37 +1248,39 @@ public void testBeginPublishAndQueueNextTasks() throws Exception workItems.add(new TestTaskRunnerWorkItem(task, null, location)); } - reset(taskStorage, taskRunner, taskClient, taskQueue); + EasyMock.reset(taskStorage, taskRunner, taskClient, taskQueue); captured = Capture.newInstance(CaptureType.ALL); - expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskClient.getStatusAsync(anyString())) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)) - .anyTimes(); - expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); - expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .times(2); - expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture(ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 1), - shardId0, - getSequenceNumber(res, shardId0, 0) - ))) - .andReturn(Futures.immediateFuture(ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 3), - shardId0, - getSequenceNumber(res, shardId0, 1) - ))); - expect( + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .times(2); + EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 1), + shardId0, + getSequenceNumber(res, shardId0, 0) + ))) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); + EasyMock.expect( taskClient.setEndOffsetsAsync( EasyMock.contains("sequenceName-0"), EasyMock.eq(ImmutableMap.of( @@ -1270,7 +1292,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception EasyMock.eq(true) ) ).andReturn(Futures.immediateFuture(true)).times(2); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of( @@ -1282,14 +1304,14 @@ public void testBeginPublishAndQueueNextTasks() throws Exception shardId0, getSequenceNumber(res, shardId0, 0) )); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); - replay(taskStorage, taskRunner, taskClient, taskQueue); + EasyMock.replay(taskStorage, taskRunner, taskClient, taskQueue); supervisor.runInternal(); verifyAll(); @@ -1348,32 +1370,33 @@ public void testDiscoverExistingPublishingTask() throws Exception workItems.add(new TestTaskRunnerWorkItem(task, null, location)); Capture captured = Capture.newInstance(); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); - expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); - expect(taskClient.getCurrentOffsetsAsync("id1", false)) - .andReturn(Futures.immediateFuture(ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 2), - shardId0, - getSequenceNumber(res, shardId0, 1) - ))); - expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( + EasyMock.expect(taskClient.getStatusAsync("id1")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); + EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2), shardId0, getSequenceNumber(res, shardId0, 1) )); - expect(taskQueue.add(capture(captured))).andReturn(true); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of( @@ -1382,9 +1405,11 @@ public void testDiscoverExistingPublishingTask() throws Exception shardId0, getSequenceNumber(res, shardId0, 0) )); - expect(taskClient.getCheckpoints(anyString(), anyBoolean())).andReturn(checkpoints).anyTimes(); + EasyMock.expect(taskClient.getCheckpoints(EasyMock.anyString(), EasyMock.anyBoolean())) + .andReturn(checkpoints) + .anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1487,34 +1512,35 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() workItems.add(new TestTaskRunnerWorkItem(task, null, location)); Capture captured = Capture.newInstance(); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); - expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); - expect(taskClient.getCurrentOffsetsAsync("id1", false)) - .andReturn(Futures.immediateFuture(ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 2), - shardId0, - getSequenceNumber(res, shardId0, 1) - ))); - expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( + EasyMock.expect(taskClient.getStatusAsync("id1")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); + EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2), shardId0, getSequenceNumber(res, shardId0, 1) )); - expect(taskQueue.add(capture(captured))).andReturn(true); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1642,44 +1668,46 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2)).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.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getCurrentOffsetsAsync("id1", false)) - .andReturn(Futures.immediateFuture(ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 2), - shardId0, - getSequenceNumber(res, shardId0, 1) - ))); - expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( + EasyMock.expect(taskClient.getStatusAsync("id1")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + EasyMock.expect(taskClient.getStatusAsync("id2")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); + EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 2), shardId0, getSequenceNumber(res, shardId0, 1) )); - expect(taskClient.getCurrentOffsetsAsync("id2", false)) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 12), - shardId0, - getSequenceNumber(res, shardId0, 1) - ))); + EasyMock.expect(taskClient.getCurrentOffsetsAsync("id2", false)) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 12), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); // since id1 is publishing, so getCheckpoints wouldn't be called for it TreeMap> checkpoints = new TreeMap<>(); @@ -1689,9 +1717,9 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception shardId0, getSequenceNumber(res, shardId0, 1) )); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); replayAll(); @@ -1754,17 +1782,17 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception List res = insertData(kinesis, generateRecordsRequests(stream)); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(4); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1773,7 +1801,7 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception List tasks = captured.getValues(); - reset(taskStorage, taskClient, taskQueue); + EasyMock.reset(taskStorage, taskClient, taskQueue); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of( @@ -1787,24 +1815,26 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception )); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); - - expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); + + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); - expect(taskClient.getStatusAsync(task.getId())) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)); - expect(taskClient.getStartTimeAsync(task.getId())) - .andReturn(Futures.immediateFailedFuture(new RuntimeException())); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(task.getId())) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)); + EasyMock.expect(taskClient.getStartTimeAsync(task.getId())) + .andReturn(Futures.immediateFailedFuture(new RuntimeException())); taskQueue.shutdown(task.getId()); } - replay(taskStorage, taskClient, taskQueue); + EasyMock.replay(taskStorage, taskClient, taskQueue); supervisor.runInternal(); verifyAll(); @@ -1820,17 +1850,17 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception List res = insertData(kinesis, generateRecordsRequests(stream)); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(4); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1843,7 +1873,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception workItems.add(new TestTaskRunnerWorkItem(task, null, location)); } - reset(taskStorage, taskRunner, taskClient, taskQueue); + EasyMock.reset(taskStorage, taskRunner, taskClient, taskQueue); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of( @@ -1852,36 +1882,38 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception )); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); captured = Capture.newInstance(CaptureType.ALL); - expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskClient.getStatusAsync(anyString())) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)) - .anyTimes(); - expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); - expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .times(2); - expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .times(2); + EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); taskQueue.shutdown(EasyMock.contains("sequenceName-0")); - expectLastCall().times(2); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expectLastCall().times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); - replay(taskStorage, taskRunner, taskClient, taskQueue); + EasyMock.replay(taskStorage, taskRunner, taskClient, taskQueue); supervisor.runInternal(); verifyAll(); @@ -1908,17 +1940,17 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception List res = insertData(kinesis, generateRecordsRequests(stream)); Capture captured = Capture.newInstance(CaptureType.ALL); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true).times(4); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -1931,7 +1963,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception workItems.add(new TestTaskRunnerWorkItem(task, null, location)); } - reset(taskStorage, taskRunner, taskClient, taskQueue); + EasyMock.reset(taskStorage, taskRunner, taskClient, taskQueue); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of( @@ -1940,39 +1972,41 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception )); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); captured = Capture.newInstance(CaptureType.ALL); - expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); - expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskClient.getStatusAsync(anyString())) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)) - .anyTimes(); - expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); - expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .times(2); - expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture(ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 1) - ))) - .andReturn(Futures.immediateFuture(ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 3) - ))); - expect( + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .times(2); + EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 1) + ))) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3) + ))); + EasyMock.expect( taskClient.setEndOffsetsAsync( EasyMock.contains("sequenceName-0"), EasyMock.eq(ImmutableMap.of( @@ -1983,10 +2017,10 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception ) ).andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); taskQueue.shutdown(EasyMock.contains("sequenceName-0")); - expectLastCall().times(2); - expect(taskQueue.add(capture(captured))).andReturn(true).times(2); + EasyMock.expectLastCall().times(2); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); - replay(taskStorage, taskRunner, taskClient, taskQueue); + EasyMock.replay(taskStorage, taskRunner, taskClient, taskQueue); supervisor.runInternal(); verifyAll(); @@ -2010,7 +2044,7 @@ public void testStopNotStarted() @Test public void testStop() { - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); taskClient.close(); taskRunner.unregisterListener(StringUtils.format("KinesisSupervisor-%s", DATASOURCE)); replayAll(); @@ -2097,27 +2131,30 @@ public void testStopGracefully() throws Exception workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(workItems).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( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( + EasyMock.expect(taskClient.getStatusAsync("id1")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + EasyMock.expect(taskClient.getStatusAsync("id2")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id3")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), shardId0, @@ -2132,40 +2169,40 @@ public void testStopGracefully() throws Exception shardId0, getSequenceNumber(res, shardId0, 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)); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); supervisor.runInternal(); verifyAll(); - reset(taskRunner, taskClient, taskQueue); - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - expect(taskClient.pauseAsync("id2")) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 12), - shardId0, - getSequenceNumber(res, shardId0, 1) - ))); - expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of( + EasyMock.reset(taskRunner, taskClient, taskQueue); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskClient.pauseAsync("id2")) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 12), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); + EasyMock.expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 12), shardId0, getSequenceNumber(res, shardId0, 1) ), true)) - .andReturn(Futures.immediateFuture(true)); + .andReturn(Futures.immediateFuture(true)); taskQueue.shutdown("id3"); - expectLastCall().times(2); + EasyMock.expectLastCall().times(2); - replay(taskRunner, taskClient, taskQueue); + EasyMock.replay(taskRunner, taskClient, taskQueue); supervisor.gracefulShutdownInternal(); verifyAll(); @@ -2174,11 +2211,11 @@ public void testStopGracefully() throws Exception @Test public void testResetNoTasks() throws Exception { - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor = getSupervisor(1, 1, true, "PT1H", null, null); @@ -2187,9 +2224,9 @@ public void testResetNoTasks() throws Exception supervisor.runInternal(); verifyAll(); - reset(indexerMetadataStorageCoordinator); - expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); - replay(indexerMetadataStorageCoordinator); + EasyMock.reset(indexerMetadataStorageCoordinator); + EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + EasyMock.replay(indexerMetadataStorageCoordinator); supervisor.resetInternal(null); verifyAll(); @@ -2200,11 +2237,11 @@ public void testResetNoTasks() throws Exception public void testResetDataSourceMetadata() throws Exception { supervisor = getSupervisor(1, 1, true, "PT1H", null, null); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -2240,19 +2277,20 @@ public void testResetDataSourceMetadata() throws Exception ) )); - reset(indexerMetadataStorageCoordinator); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(KinesisDataSourceMetadata); - expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata( + EasyMock.reset(indexerMetadataStorageCoordinator); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)) + .andReturn(KinesisDataSourceMetadata); + EasyMock.expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata( EasyMock.capture(captureDataSource), EasyMock.capture(captureDataSourceMetadata) )).andReturn(true); - replay(indexerMetadataStorageCoordinator); + EasyMock.replay(indexerMetadataStorageCoordinator); try { supervisor.resetInternal(resetMetadata); } catch (NullPointerException npe) { - // Expected as there will be an attempt to reset partitionGroups offsets to NOT_SET + // Expected as there will be an attempt to EasyMock.reset partitionGroups offsets to NOT_SET // however there would be no entries in the map as we have not put nay data in kafka Assert.assertTrue(npe.getCause() == null); } @@ -2266,11 +2304,11 @@ public void testResetDataSourceMetadata() throws Exception public void testResetNoDataSourceMetadata() throws Exception { supervisor = getSupervisor(1, 1, true, "PT1H", null, null); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -2285,10 +2323,10 @@ public void testResetNoDataSourceMetadata() throws Exception ) )); - reset(indexerMetadataStorageCoordinator); + EasyMock.reset(indexerMetadataStorageCoordinator); // no DataSourceMetadata in metadata store - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null); - replay(indexerMetadataStorageCoordinator); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null); + EasyMock.replay(indexerMetadataStorageCoordinator); supervisor.resetInternal(resetMetadata); verifyAll(); @@ -2378,27 +2416,30 @@ public void testResetRunningTasks() throws Exception workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(workItems).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( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( + EasyMock.expect(taskClient.getStatusAsync("id1")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + EasyMock.expect(taskClient.getStatusAsync("id2")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id3")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), @@ -2418,25 +2459,25 @@ public void testResetRunningTasks() throws Exception getSequenceNumber(res, shardId0, 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)); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); supervisor.runInternal(); verifyAll(); - reset(taskQueue, indexerMetadataStorageCoordinator); - expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + EasyMock.reset(taskQueue, indexerMetadataStorageCoordinator); + EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); taskQueue.shutdown("id2"); taskQueue.shutdown("id3"); - replay(taskQueue, indexerMetadataStorageCoordinator); + EasyMock.replay(taskQueue, indexerMetadataStorageCoordinator); supervisor.resetInternal(null); verifyAll(); @@ -2519,26 +2560,29 @@ public void testNoDataIngestionTasks() throws Exception 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( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStatusAsync("id1")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id2")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id3")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of( @@ -2550,29 +2594,29 @@ public void testNoDataIngestionTasks() throws Exception getSequenceNumber(res, shardId0, 1) )); - 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)); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); supervisor.runInternal(); verifyAll(); - reset(taskQueue, indexerMetadataStorageCoordinator); - expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + EasyMock.reset(taskQueue, indexerMetadataStorageCoordinator); + EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); taskQueue.shutdown("id1"); taskQueue.shutdown("id2"); taskQueue.shutdown("id3"); - replay(taskQueue, indexerMetadataStorageCoordinator); + EasyMock.replay(taskQueue, indexerMetadataStorageCoordinator); supervisor.resetInternal(null); verifyAll(); @@ -2663,28 +2707,28 @@ public void testCheckpointForInactiveTaskGroup() workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - 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( + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect( indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KinesisDataSourceMetadata( null) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.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)); + EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); final TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of( @@ -2696,17 +2740,17 @@ public void testCheckpointForInactiveTaskGroup() getSequenceNumber(res, shardId0, 1) )); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -2807,16 +2851,16 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException 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( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect( indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KinesisDataSourceMetadata( null) ).anyTimes(); @@ -2887,33 +2931,33 @@ public void testCheckpointWithNullTaskGroupId() 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( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect( indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KinesisDataSourceMetadata( null) ).anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); - expect(taskClient.getStatusAsync(anyString())) + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(Status.READING)) .anyTimes(); final TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L)); - expect(taskClient.getCheckpointsAsync(anyString(), anyBoolean())) + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.anyString(), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(3); - expect(taskClient.getStartTimeAsync(anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); - expect(taskClient.pauseAsync(anyString())) + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(taskClient.pauseAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L))) .anyTimes(); - expect(taskClient.setEndOffsetsAsync(anyString(), EasyMock.eq(ImmutableMap.of(0, 10L)), anyBoolean())) + EasyMock.expect(taskClient.setEndOffsetsAsync(EasyMock.anyString(), EasyMock.eq(ImmutableMap.of(0, 10L)), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(true)) .anyTimes(); @@ -2945,20 +2989,20 @@ public void testSuspendedNoRunningTasks() throws Exception { supervisor = getSupervisor(1, 1, true, "PT1H", null, null, true); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); // this asserts that taskQueue.add does not in fact get called because supervisor should be suspended - expect(taskQueue.add(anyObject())).andAnswer((IAnswer) () -> { + EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andAnswer((IAnswer) () -> { Assert.fail(); return null; }).anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -3043,27 +3087,30 @@ public void testSuspendedRunningTasks() throws Exception workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(workItems).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( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); + EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); + EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); - expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); - expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( + EasyMock.expect(taskClient.getStatusAsync("id1")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + EasyMock.expect(taskClient.getStatusAsync("id2")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id3")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 3), shardId0, @@ -3078,31 +3125,31 @@ public void testSuspendedRunningTasks() throws Exception shardId0, getSequenceNumber(res, shardId0, 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); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); - expect(taskClient.pauseAsync("id2")) - .andReturn(Futures.immediateFuture(ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 12), - shardId0, - getSequenceNumber(res, shardId0, 1) - ))); - expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of( + EasyMock.expect(taskClient.pauseAsync("id2")) + .andReturn(Futures.immediateFuture(ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 12), + shardId0, + getSequenceNumber(res, shardId0, 1) + ))); + EasyMock.expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of( shardId1, getSequenceNumber(res, shardId1, 12), shardId0, getSequenceNumber(res, shardId0, 1) ), true)) - .andReturn(Futures.immediateFuture(true)); + .andReturn(Futures.immediateFuture(true)); taskQueue.shutdown("id3"); - expectLastCall().times(2); + EasyMock.expectLastCall().times(2); replayAll(); supervisor.start(); @@ -3113,11 +3160,11 @@ public void testSuspendedRunningTasks() throws Exception @Test public void testResetSuspended() throws Exception { - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); supervisor = getSupervisor(1, 1, true, "PT1H", null, null, true); @@ -3125,9 +3172,9 @@ public void testResetSuspended() throws Exception supervisor.runInternal(); verifyAll(); - reset(indexerMetadataStorageCoordinator); - expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); - replay(indexerMetadataStorageCoordinator); + EasyMock.reset(indexerMetadataStorageCoordinator); + EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + EasyMock.replay(indexerMetadataStorageCoordinator); supervisor.resetInternal(null); verifyAll(); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestBroker.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestBroker.java deleted file mode 100644 index 6ad8f8c2226c..000000000000 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestBroker.java +++ /dev/null @@ -1,121 +0,0 @@ -///* -// * Licensed to Metamarkets Group Inc. (Metamarkets) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. Metamarkets licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, -// * software distributed under the License is distributed on an -// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// * KIND, either express or implied. See the License for the -// * specific language governing permissions and limitations -// * under the License. -// */ -// -//package org.apache.druid.indexing.kinesis.test; -// -//import com.google.common.collect.ImmutableMap; -//import com.google.common.collect.Maps; -//import kafka.server.KafkaConfig; -//import kafka.server.KafkaServer; -//import org.apache.kafka.clients.consumer.KafkaConsumer; -//import org.apache.kafka.clients.producer.KafkaProducer; -//import org.apache.kafka.common.serialization.ByteArrayDeserializer; -//import org.apache.kafka.common.serialization.ByteArraySerializer; -//import org.apache.kafka.common.utils.SystemTime; -//import scala.Some; -//import scala.collection.immutable.List$; -// -//import java.io.Closeable; -//import java.io.File; -//import java.io.IOException; -//import java.util.Map; -//import java.util.Properties; -//import java.util.Random; -// -//public class TestBroker implements Closeable -//{ -// private final static Random RANDOM = new Random(); -// -// private final String zookeeperConnect; -// private final File directory; -// private final int id; -// private final Map brokerProps; -// -// private volatile KafkaServer server; -// -// public TestBroker(String zookeeperConnect, File directory, int id, Map brokerProps) -// { -// this.zookeeperConnect = zookeeperConnect; -// this.directory = directory; -// this.id = id; -// this.brokerProps = brokerProps == null ? ImmutableMap.of() : brokerProps; -// } -// -// public void start() -// { -// final Properties props = new Properties(); -// props.setProperty("zookeeper.connect", zookeeperConnect); -// props.setProperty("zookeeper.session.timeout.ms", "30000"); -// props.setProperty("zookeeper.connection.timeout.ms", "30000"); -// props.setProperty("log.dirs", directory.toString()); -// props.setProperty("broker.id", String.valueOf(id)); -// props.setProperty("port", String.valueOf(new Random().nextInt(9999) + 10000)); -// props.putAll(brokerProps); -// -// final KafkaConfig config = new KafkaConfig(props); -// -// server = new KafkaServer(config, SystemTime.SYSTEM, Some.apply(String.format("TestingBroker[%d]-", id)), List$.MODULE$.empty()); -// server.startup(); -// } -// -// public int getPort() -// { -// return server.socketServer().config().port(); -// } -// -// public KafkaProducer newProducer() -// { -// return new KafkaProducer(producerProperties()); -// } -// -// public KafkaConsumer newConsumer() -// { -// return new KafkaConsumer(consumerProperties()); -// } -// -// public Map producerProperties() -// { -// final Map props = Maps.newHashMap(); -// props.put("bootstrap.servers", String.format("localhost:%d", getPort())); -// props.put("key.serializer", ByteArraySerializer.class.getName()); -// props.put("value.serializer", ByteArraySerializer.class.getName()); -// props.put("acks", "all"); -// return props; -// } -// -// public Map consumerProperties() -// { -// final Map props = Maps.newHashMap(); -// props.put("bootstrap.servers", String.format("localhost:%d", getPort())); -// props.put("key.deserializer", ByteArrayDeserializer.class.getName()); -// props.put("value.deserializer", ByteArrayDeserializer.class.getName()); -// props.put("group.id", String.valueOf(RANDOM.nextInt())); -// props.put("auto.offset.reset", "earliest"); -// return props; -// } -// -// @Override -// public void close() throws IOException -// { -// if (server != null) { -// server.shutdown(); -// server.awaitShutdown(); -// } -// } -//} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java index 3712690ef944..c4f2194db35a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java @@ -77,7 +77,7 @@ public DataSourceMetadata plus(DataSourceMetadata other) @SuppressWarnings("unchecked") final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; - if (that.getSeekableStreamPartitions().getId().equals(seekableStreamPartitions.getId())) { + if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) { // Same topic, merge offsets. final Map newMap = Maps.newHashMap(); @@ -89,7 +89,7 @@ public DataSourceMetadata plus(DataSourceMetadata other) newMap.put(entry.getKey(), entry.getValue()); } - return createConcretDataSourceMetaData(seekableStreamPartitions.getId(), newMap); + return createConcretDataSourceMetaData(seekableStreamPartitions.getStream(), newMap); } else { // Different topic, prefer "other". return other; @@ -111,7 +111,7 @@ public DataSourceMetadata minus(DataSourceMetadata other) @SuppressWarnings("unchecked") final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; - if (that.getSeekableStreamPartitions().getId().equals(seekableStreamPartitions.getId())) { + if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) { // Same stream, remove partitions present in "that" from "this" final Map newMap = Maps.newHashMap(); @@ -121,7 +121,7 @@ public DataSourceMetadata minus(DataSourceMetadata other) } } - return createConcretDataSourceMetaData(seekableStreamPartitions.getId(), newMap); + return createConcretDataSourceMetaData(seekableStreamPartitions.getStream(), newMap); } else { // Different stream, prefer "this". return this; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java index 0cbf8e4a6738..a4026acbe773 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java @@ -62,7 +62,7 @@ public SeekableStreamIOConfig( this.maximumMessageTime = Optional.fromNullable(maximumMessageTime); Preconditions.checkArgument( - startPartitions.getId().equals(endPartitions.getId()), + startPartitions.getStream().equals(endPartitions.getStream()), "start topic/stream and end topic/stream must match" ); @@ -70,20 +70,6 @@ public SeekableStreamIOConfig( startPartitions.getMap().keySet().equals(endPartitions.getMap().keySet()), "start partition set and end partition set must match" ); - - // are sequence numbers guranteed to be greater? - /* - for (T1 partition : endPartitions.getMap().keySet()) { - Preconditions.checkArgument( - endPartitions.getMap() - .get(partition) - .compareTo(startPartitions.getMap().get(partition)) >= 0, - "end offset must be >= start offset for partition[%s]", - partition - ); - } - */ - } @Nullable diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index ed5d6685abdf..30d8cdb0460f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -46,7 +46,7 @@ import java.util.Map; import java.util.Random; -//TODO: need more refactoring for run() + public abstract class SeekableStreamIndexTask extends AbstractTask implements ChatHandler { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index f7c52c67b9e5..f3bb86d67fda 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -29,16 +29,27 @@ import javax.ws.rs.core.Response; import java.util.Map; +/** + * Interface for abstracting the indexing task run logic. Only used by Kafka indexing tasks, + * but will also be used by Kinesis indexing tasks once implemented + * + * @param Partition Number Type + * @param Sequence Number Type + */ public interface SeekableStreamIndexTaskRunner extends ChatHandler { Appenderator getAppenderator(); + /** + * Run the task + */ TaskStatus run(TaskToolbox toolbox); + /** + * Stop the task + */ void stopGracefully(); - // The below methods are mostly for unit testing. - @VisibleForTesting RowIngestionMeters getRowIngestionMeters(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index c833fb832cdf..51e7e70390b6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -28,11 +28,19 @@ import java.util.Map; import java.util.Objects; +/** + * class that encapsulates a map of partitionId -> sequenceNumber. Redundant getters + * are used for proper Jackson serialization/deserialization when processing terminologies + * used by Kafka and kinesis (i.e. topic vs. stream) + * + * @param partition id type + * @param sequence number type + */ public class SeekableStreamPartitions { public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER"; - private final String id; + private final String stream; private final Map map; @JsonCreator @@ -43,18 +51,18 @@ public SeekableStreamPartitions( @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap ) { - this.id = stream == null ? topic : stream; + this.stream = stream == null ? topic : stream; this.map = ImmutableMap.copyOf(partitionOffsetMap == null ? partitionSequenceNumberMap : partitionOffsetMap); - Preconditions.checkArgument(id != null); + Preconditions.checkArgument(this.stream != null); Preconditions.checkArgument(map != null); // Validate map for (Map.Entry entry : map.entrySet()) { Preconditions.checkArgument( entry.getValue() != null, String.format( - "partition id[%s] sequence/offset number[%s] invalid", + "partition stream[%s] sequence/offset number[%s] invalid", entry.getKey(), entry.getValue() ) @@ -62,27 +70,22 @@ public SeekableStreamPartitions( } } - // for backward compatibility + // constructor for backward compatibility public SeekableStreamPartitions(@NotNull final String id, final Map partitionOffsetMap) { this(id, null, partitionOffsetMap, null); } - public String getId() - { - return id; - } - @JsonProperty public String getStream() { - return id; + return stream; } @JsonProperty public String getTopic() { - return id; + return stream; } public Map getMap() @@ -112,21 +115,21 @@ public boolean equals(Object o) return false; } SeekableStreamPartitions that = (SeekableStreamPartitions) o; - return Objects.equals(id, that.id) && + return Objects.equals(stream, that.stream) && Objects.equals(map, that.map); } @Override public int hashCode() { - return Objects.hash(id, map); + return Objects.hash(stream, map); } @Override public String toString() { return "SeekableStreamPartitions{" + - "stream/topic='" + id + '\'' + + "stream/topic='" + stream + '\'' + ", partitionSequenceNumberMap/partitionOffsetMap=" + map + '}'; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java index d102e68537d2..85dee67f72f2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java @@ -23,7 +23,6 @@ public class Record { - // TODO: end of shard marker public static final String END_OF_SHARD_MARKER = "EOS"; private final String streamName; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java index 5a01aece4560..d233d0d81230 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java @@ -19,6 +19,9 @@ package org.apache.druid.indexing.seekablestream.common; +import com.google.common.annotations.Beta; + +import javax.annotation.Nullable; import java.io.Closeable; import java.util.Collection; import java.util.Set; @@ -26,33 +29,111 @@ /** * The RecordSupplier interface is a wrapper for the incoming seekable data stream - * (i.e. Kafka consumer) + * (i.e. Kafka consumer, Kinesis streams) + * + * @param Partition Number Type + * @param Sequence Number Type */ +@Beta public interface RecordSupplier extends Closeable { + /** + * assigns a set of partitions to this RecordSupplier + * + * @param partitions partitions to assign + */ void assign(Set> partitions); + /** + * seek to specified sequence number + * + * @param partition partition to seek + * @param sequenceNumber sequence number to seek to + */ void seek(StreamPartition partition, T2 sequenceNumber); + /** + * seek to the sequence number immediately following the given sequenceNumber + * + * @param partition partition to seek + * @param sequenceNumber sequence number to seek + */ void seekAfter(StreamPartition partition, T2 sequenceNumber); - void seekToEarliest(Set> partition); + /** + * seek a set of partitions to the earliest record position available in the stream + * + * @param partitions partitions to seek + */ + void seekToEarliest(Set> partitions); - void seekToLatest(Set> partition); + /** + * seek a set of partitions to the latest/newest record position available in the stream + * + * @param partitions partitions to seek + */ + void seekToLatest(Set> partitions); + /** + * get the current assignment + * + * @return set of assignments + */ Collection> getAssignment(); + /** + * poll the record at the current seeked to sequence in stream + * + * @param timeout timeout in milliseconds + * + * @return record + */ + @Nullable Record poll(long timeout); + /** + * get the latest sequence number in stream + * + * @param partition target partition + * + * @return latest sequence number + * + * @throws TimeoutException TimeoutException + */ T2 getLatestSequenceNumber(StreamPartition partition) throws TimeoutException; + /** + * get the earliest sequence number in stream + * + * @param partition target partition + * + * @return earliest sequence number + * + * @throws TimeoutException TimeoutException + */ T2 getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException; - // TODO: maybe remove this, not needed in kinesis, only in kafka + /** + * returns the sequence number that the given partition is currently at + * + * @param partition target partition + * + * @return sequence number + */ T2 position(StreamPartition partition); + /** + * returns the set of partitions under the given stream + * + * @param streamName name of stream + * + * @return set of partitions + */ Set getPartitionIds(String streamName); + /** + * close the RecordSupplier + */ @Override void close(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java index 7f080b238fe5..580441ce9aa6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java @@ -22,6 +22,12 @@ import java.util.Objects; +/** + * Wrapper class for Kafka and Kinesis stream sequence numbers. Mainly used to do + * comparison and indicate whether the sequence number should be excluded + * + * @param type of sequence number + */ public abstract class SequenceNumber implements Comparable> { private final T sequenceNumber; @@ -45,12 +51,25 @@ public boolean isExclusive() return useExclusive && isExclusive; } - @Override - public abstract boolean equals(Object o); @Override public int hashCode() { return Objects.hash(sequenceNumber, useExclusive, isExclusive); } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SequenceNumber that = (SequenceNumber) o; + return isExclusive == that.isExclusive && + useExclusive == that.useExclusive && + Objects.equals(sequenceNumber, that.sequenceNumber); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java index 7a6db473aa8d..d424a37c45f0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java @@ -19,6 +19,11 @@ package org.apache.druid.indexing.seekablestream.common; +/** + * wrapper class for a Kinesis/Kafka partition with stream name and partitionId + * + * @param partition id type + */ public class StreamPartition { private final String streamName; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index d4adaa12c0e7..e24bb2c780ba 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -105,8 +105,16 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -//TODO: documentation -//TODO: compare with kinesis supervisor for subsequently discovered partitions +/** + * this class is the parent class of both the Kafka and Kinesis supervisor. All the main run loop + * logic are similar enough so they're grouped together into this class. + *

+ * incremental handoff & checkpointing are not yet supported by Kinesis, but the logic is left in here + * so in the future it's easier to implement + * + * @param partition id type + * @param sequence number type + */ public abstract class SeekableStreamSupervisor implements Supervisor { @@ -684,7 +692,7 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) @SuppressWarnings("unchecked") final SeekableStreamDataSourceMetadata resetMetadata = (SeekableStreamDataSourceMetadata) dataSourceMetadata; - if (resetMetadata.getSeekableStreamPartitions().getId().equals(ioConfig.getId())) { + if (resetMetadata.getSeekableStreamPartitions().getStream().equals(ioConfig.getId())) { // metadata can be null final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); if (metadata != null && !checkSourceMetaDataMatch(metadata)) { @@ -751,7 +759,7 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) } else { log.warn( "Reset metadata topic [%s] and supervisor's stream name [%s] do not match", - resetMetadata.getSeekableStreamPartitions().getId(), + resetMetadata.getSeekableStreamPartitions().getStream(), ioConfig.getId() ); } @@ -992,8 +1000,6 @@ private void verifyAndMergeCheckpoints(final Collection taskGroupsToV } } - //TODO: prob wanna refactor this - /** * This method does two things - * 1. Makes sure the checkpoints information in the taskGroup is consistent with that of the tasks, if not kill @@ -1059,7 +1065,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) final boolean hasValidOffsetsFromDb = latestDataSourceMetadata != null && latestDataSourceMetadata.getSeekableStreamPartitions() != null && ioConfig.getId().equals( - latestDataSourceMetadata.getSeekableStreamPartitions().getId() + latestDataSourceMetadata.getSeekableStreamPartitions().getStream() ); final Map latestOffsetsFromDb; if (hasValidOffsetsFromDb) { @@ -1312,7 +1318,7 @@ private void updatePartitionDataFromStream() if (!initialPartitionDiscovery && !this.partitionIds.contains(partitionId)) { subsequentlyDiscoveredPartitions.add(partitionId); - //TODO: early publish time + // should check for earlyPublishTime (Kinesis) here, not supported yet } int taskGroupId = getTaskGroupIdForPartition(partitionId); @@ -1971,10 +1977,10 @@ && checkSourceMetaDataMatch(dataSourceMetadata)) { @SuppressWarnings("unchecked") SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata).getSeekableStreamPartitions(); if (partitions != null) { - if (!ioConfig.getId().equals(partitions.getId())) { + if (!ioConfig.getId().equals(partitions.getStream())) { log.warn( "Topic/stream in metadata storage [%s] doesn't match spec topic/stream [%s], ignoring stored offsets", - partitions.getId(), + partitions.getStream(), ioConfig.getId() ); return Collections.emptyMap(); @@ -2215,7 +2221,7 @@ private class TaskGroup final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); final Optional minimumMessageTime; final Optional maximumMessageTime; - final Set exclusiveStartSequenceNumberPartitions; //TODO: exclusiveSequence + final Set exclusiveStartSequenceNumberPartitions; final TreeMap> checkpointSequences = new TreeMap<>(); final String baseSequenceName; DateTime completionTimeout; From f585d4c02fdf3825db5ecd89d1b994e90cf442a8 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 4 Oct 2018 16:12:49 -0700 Subject: [PATCH 19/87] merge #6291 merge #6337 merge #6383 --- .../kafka/supervisor/KafkaSupervisor.java | 30 +- .../kafka/supervisor/KafkaSupervisorTest.java | 347 ++++++++++-------- .../supervisor/SeekableStreamSupervisor.java | 244 ++++++++---- 3 files changed, 388 insertions(+), 233 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 337bb85f9bb2..2ab5bfd04948 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -63,6 +63,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ExecutionException; @@ -304,7 +305,7 @@ protected Map getLagPerPartition(Map currentOffset .stream() .collect( Collectors.toMap( - Map.Entry::getKey, + Entry::getKey, e -> latestSequenceFromStream != null && latestSequenceFromStream.get(e.getKey()) != null && e.getValue() != null @@ -376,6 +377,16 @@ private Runnable emitLag() }; } + @Override + protected boolean checkSequenceAvailability( + @NotNull Integer partition, @NotNull Long sequenceFromMetadata + ) throws TimeoutException + { + Long latestOffset = getOffsetFromStreamForPartition(partition, false); + return latestOffset != null + && KafkaSequenceNumber.of(latestOffset).compareTo(KafkaSequenceNumber.of(sequenceFromMetadata)) >= 0; + } + // the following are for unit testing purposes only @Override @VisibleForTesting @@ -420,15 +431,18 @@ protected int getNoticesQueueSize() return super.getNoticesQueueSize(); } + @Override - protected boolean checkSequenceAvailability( - @NotNull Integer partition, @NotNull Long sequenceFromMetadata - ) throws TimeoutException + @VisibleForTesting + public KafkaSupervisorIOConfig getIoConfig() { - Long latestOffset = getOffsetFromStreamForPartition(partition, false); - return latestOffset != null - && KafkaSequenceNumber.of(latestOffset).compareTo(KafkaSequenceNumber.of(sequenceFromMetadata)) >= 0; + return spec.getIoConfig(); } - + @Override + @VisibleForTesting + protected void tryInit() + { + super.tryInit(); + } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 276c2a03a1f8..10030633b176 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -635,9 +635,12 @@ public void testKillIncompatibleTasks() throws Exception EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) - .anyTimes(); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null @@ -653,8 +656,8 @@ public void testKillIncompatibleTasks() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(2); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(2); replayAll(); @@ -734,9 +737,12 @@ public void testKillBadPartitionAssignment() throws Exception EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); EasyMock.expect(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); EasyMock.expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) - .anyTimes(); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null @@ -751,11 +757,11 @@ public void testKillBadPartitionAssignment() throws Exception TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(1); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); taskQueue.shutdown("id4"); @@ -778,9 +784,12 @@ public void testRequeueTaskWhenFailed() throws Exception EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) - .anyTimes(); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null @@ -793,11 +802,11 @@ public void testRequeueTaskWhenFailed() throws Exception TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .anyTimes(); + .andReturn(Futures.immediateFuture(checkpoints1)) + .anyTimes(); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .anyTimes(); + .andReturn(Futures.immediateFuture(checkpoints2)) + .anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -813,7 +822,9 @@ public void testRequeueTaskWhenFailed() throws Exception EasyMock.reset(taskStorage); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } EasyMock.replay(taskStorage); @@ -829,10 +840,13 @@ public void testRequeueTaskWhenFailed() throws Exception EasyMock.reset(taskQueue); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes(); for (Task task : imStillAlive) { - EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - EasyMock.expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); + EasyMock.expect(taskStorage.getStatus(iHaveFailed.getId())) + .andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); EasyMock.expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.capture(aNewTaskCapture))).andReturn(true); EasyMock.replay(taskStorage); @@ -887,8 +901,8 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L, 2, 0L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(2); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(2); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -910,15 +924,18 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception // for the newly created replica task EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(2); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes(); - EasyMock.expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); - EasyMock.expect(taskStorage.getStatus(runningTaskId)).andReturn(Optional.of(TaskStatus.running(runningTaskId))).anyTimes(); + EasyMock.expect(taskStorage.getStatus(iHaveFailed.getId())) + .andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); + EasyMock.expect(taskStorage.getStatus(runningTaskId)) + .andReturn(Optional.of(TaskStatus.running(runningTaskId))) + .anyTimes(); EasyMock.expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); EasyMock.expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(Status.READING)); @@ -957,9 +974,12 @@ public void testQueueNextTasksOnSuccess() throws Exception EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) - .anyTimes(); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null @@ -979,24 +999,29 @@ public void testQueueNextTasksOnSuccess() throws Exception EasyMock.reset(taskStorage); EasyMock.reset(taskClient); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(Status.NOT_STARTED)) - .anyTimes(); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); // there would be 4 tasks, 2 for each task group EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } EasyMock.replay(taskStorage); @@ -1016,13 +1041,17 @@ public void testQueueNextTasksOnSuccess() throws Exception EasyMock.reset(taskClient); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(imStillRunning).anyTimes(); for (Task task : imStillRunning) { - EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - EasyMock.expect(taskStorage.getStatus(iAmSuccess.getId())).andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId()))); + EasyMock.expect(taskStorage.getStatus(iAmSuccess.getId())) + .andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId()))); EasyMock.expect(taskStorage.getTask(iAmSuccess.getId())).andReturn(Optional.of((Task) iAmSuccess)).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.capture(newTasksCapture))).andReturn(true).times(2); - EasyMock.expect(taskClient.stopAsync(EasyMock.capture(shutdownTaskIdCapture), EasyMock.eq(false))).andReturn(Futures.immediateFuture(true)); + EasyMock.expect(taskClient.stopAsync(EasyMock.capture(shutdownTaskIdCapture), EasyMock.eq(false))) + .andReturn(Futures.immediateFuture(true)); EasyMock.replay(taskStorage); EasyMock.replay(taskQueue); EasyMock.replay(taskClient); @@ -1070,22 +1099,24 @@ public void testBeginPublishAndQueueNextTasks() throws Exception captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.READING)) - .anyTimes(); + .andReturn(Futures.immediateFuture(Status.READING)) + .anyTimes(); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); + .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .times(2); + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .times(2); EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); EasyMock.expect( taskClient.setEndOffsetsAsync( EasyMock.contains("sequenceName-0"), @@ -1100,11 +1131,11 @@ public void testBeginPublishAndQueueNextTasks() throws Exception TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); EasyMock.replay(taskStorage, taskRunner, taskClient, taskQueue); @@ -1165,13 +1196,15 @@ public void testDiscoverExistingPublishingTask() throws Exception ).anyTimes(); EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)); - EasyMock.expect(taskClient.getCheckpoints(EasyMock.anyString(), EasyMock.anyBoolean())).andReturn(checkpoints).anyTimes(); + EasyMock.expect(taskClient.getCheckpoints(EasyMock.anyString(), EasyMock.anyBoolean())) + .andReturn(checkpoints) + .anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1257,7 +1290,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() ).anyTimes(); EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 2, 30L))); + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 2, 30L))); EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); @@ -1367,10 +1400,10 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 1L, 1, 2L, 2, 3L))); + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 1L, 1, 2L, 2, 3L))); EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); EasyMock.expect(taskClient.getCurrentOffsetsAsync("id2", false)) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 4L, 1, 5L, 2, 6L))); + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 4L, 1, 5L, 2, 6L))); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); @@ -1378,8 +1411,8 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); replayAll(); @@ -1454,20 +1487,22 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(task.getId())) - .andReturn(Futures.immediateFuture(Status.NOT_STARTED)); + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)); EasyMock.expect(taskClient.getStartTimeAsync(task.getId())) - .andReturn(Futures.immediateFailedFuture(new RuntimeException())); + .andReturn(Futures.immediateFailedFuture(new RuntimeException())); taskQueue.shutdown(task.getId()); } EasyMock.replay(taskStorage, taskClient, taskQueue); @@ -1515,30 +1550,32 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.READING)) - .anyTimes(); + .andReturn(Futures.immediateFuture(Status.READING)) + .anyTimes(); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); + .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .times(2); + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .times(2); EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); + .andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); taskQueue.shutdown(EasyMock.contains("sequenceName-0")); EasyMock.expectLastCall().times(2); EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); @@ -1594,31 +1631,33 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - EasyMock.expect(taskStorage.getStatus(task.getId())).andReturn(Optional.of(TaskStatus.running(task.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.READING)) - .anyTimes(); + .andReturn(Futures.immediateFuture(Status.READING)) + .anyTimes(); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); + .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .times(2); + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .times(2); EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); EasyMock.expect( taskClient.setEndOffsetsAsync( EasyMock.contains("sequenceName-0"), @@ -1743,11 +1782,11 @@ public void testStopGracefully() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1759,9 +1798,9 @@ public void testStopGracefully() throws Exception EasyMock.reset(taskRunner, taskClient, taskQueue); EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); EasyMock.expect(taskClient.pauseAsync("id2")) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); EasyMock.expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) - .andReturn(Futures.immediateFuture(true)); + .andReturn(Futures.immediateFuture(true)); taskQueue.shutdown("id3"); EasyMock.expectLastCall().times(2); @@ -1829,7 +1868,8 @@ public void testResetDataSourceMetadata() throws Exception )); EasyMock.reset(indexerMetadataStorageCoordinator); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(kafkaDataSourceMetadata); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)) + .andReturn(kafkaDataSourceMetadata); EasyMock.expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata( EasyMock.capture(captureDataSource), EasyMock.capture(captureDataSourceMetadata) @@ -1957,11 +1997,11 @@ public void testResetRunningTasks() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2049,14 +2089,14 @@ public void testNoDataIngestionTasks() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2144,14 +2184,14 @@ public void testCheckpointForInactiveTaskGroup() final TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2307,20 +2347,26 @@ public void testCheckpointWithNullTaskGroupId() ).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.READING)) - .anyTimes(); + .andReturn(Futures.immediateFuture(Status.READING)) + .anyTimes(); final TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.anyString(), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(3); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(3); + EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); EasyMock.expect(taskClient.pauseAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L))) - .anyTimes(); - EasyMock.expect(taskClient.setEndOffsetsAsync(EasyMock.anyString(), EasyMock.eq(ImmutableMap.of(0, 10L)), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(true)) - .anyTimes(); + .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L))) + .anyTimes(); + EasyMock.expect(taskClient.setEndOffsetsAsync( + EasyMock.anyString(), + EasyMock.eq(ImmutableMap.of(0, 10L)), + EasyMock.anyBoolean() + )) + .andReturn(Futures.immediateFuture(true)) + .anyTimes(); replayAll(); @@ -2441,9 +2487,12 @@ public void testSuspendedRunningTasks() throws Exception null ) ).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); - EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id1")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + EasyMock.expect(taskClient.getStatusAsync("id2")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + EasyMock.expect(taskClient.getStatusAsync("id3")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); @@ -2452,18 +2501,18 @@ public void testSuspendedRunningTasks() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); EasyMock.expect(taskClient.pauseAsync("id2")) - .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); + .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); EasyMock.expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) - .andReturn(Futures.immediateFuture(true)); + .andReturn(Futures.immediateFuture(true)); taskQueue.shutdown("id3"); EasyMock.expectLastCall().times(2); @@ -2513,10 +2562,10 @@ public void testFailedInitializationAndRecovery() throws Exception ); addSomeEvents(1); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) @@ -2539,16 +2588,16 @@ public void testFailedInitializationAndRecovery() throws Exception resetAll(); Capture captured = Capture.newInstance(); - expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - expect(taskQueue.add(capture(captured))).andReturn(true); - taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); // Fix the bad hostname during the initialization retries and finish the supervisor start. @@ -2565,7 +2614,7 @@ public void testFailedInitializationAndRecovery() throws Exception KafkaIndexTask task = captured.getValue(); Assert.assertEquals(dataSchema, task.getDataSchema()); - Assert.assertEquals(KafkaTuningConfig.copyOf(tuningConfig), task.getTuningConfig()); + Assert.assertEquals(tuningConfig.copyOf(), task.getTuningConfig()); KafkaIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index e24bb2c780ba..cca4b447dcc8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -71,6 +71,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -88,6 +89,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Random; import java.util.Set; import java.util.SortedMap; @@ -123,6 +125,7 @@ public abstract class SeekableStreamSupervisor private static final Random RANDOM = new Random(); private static final long MAX_RUN_FREQUENCY_MILLIS = 1000; private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120; + private static final int MAX_INITIALIZATION_RETRIES = 20; private static final CopyOnWriteArrayList EMPTY_LIST = Lists.newCopyOnWriteArrayList(); protected final List partitionIds = new CopyOnWriteArrayList<>(); protected final Set subsequentlyDiscoveredPartitions = new HashSet<>(); @@ -157,10 +160,13 @@ public abstract class SeekableStreamSupervisor protected volatile DateTime sequenceLastUpdated; private boolean listenerRegistered = false; private long lastRunTime; + private int initRetryCounter = 0; private volatile DateTime firstRunTime; private volatile RecordSupplier recordSupplier; private volatile boolean started = false; private volatile boolean stopped = false; + private volatile boolean lifecycleStarted = false; + public SeekableStreamSupervisor( final String supervisorId, @@ -265,38 +271,45 @@ protected static String getRandomId() return suffix.toString(); } - - @Override - public void start() + @VisibleForTesting + protected void tryInit() { synchronized (stateChangeLock) { - Preconditions.checkState(!started, "already started"); - Preconditions.checkState(!exec.isShutdown(), "already stopped"); + if (started) { + log.warn("SUpervisor was already started, skipping init"); + return; + } - try { - if (recordSupplier == null) { - recordSupplier = setupRecordSupplier(); - } + if (stopped) { + log.warn("Supervisor was already stopped, skipping init."); + return; + } + try { + recordSupplier = setupRecordSupplier(); exec.submit( () -> { try { - while (!Thread.currentThread().isInterrupted()) { - final Notice notice = notices.take(); + long pollTimeout = Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS); + while (!Thread.currentThread().isInterrupted() && !stopped) { + final Notice notice = notices.poll(pollTimeout, TimeUnit.MILLISECONDS); + if (notice == null) { + continue; + } try { notice.handle(); } catch (Throwable e) { - log.makeAlert(e, "[%s] failed to handle notice", supervisorId) + log.makeAlert(e, "KafkaSupervisor[%s] failed to handle notice", dataSource) .addData("noticeClass", notice.getClass().getSimpleName()) .emit(); } } } catch (InterruptedException e) { - log.info("[%s] interrupted, exiting", supervisorId); + log.info("KafkaSupervisor[%s] interrupted, exiting", dataSource); } } ); @@ -308,25 +321,74 @@ public void start() TimeUnit.MILLISECONDS ); + // not yet implemented in kinesis, will remove once implemented scheduleReporting(reportingExec); started = true; log.info( - "Started [%s], first run in [%s], with spec: [%s]", - supervisorId, + "Started KafkaSupervisor[%s], first run in [%s], with spec: [%s]", + dataSource, ioConfig.getStartDelay(), spec.toString() ); - } catch (Exception e) { if (recordSupplier != null) { recordSupplier.close(); } - log.makeAlert(e, "Exception starting [%s]", supervisorId) + initRetryCounter++; + log.makeAlert(e, "Exception starting KafkaSupervisor[%s]", dataSource) .emit(); - throw Throwables.propagate(e); + + throw new RuntimeException(e); + } + } + } + + + @Override + public void start() + { + synchronized (stateChangeLock) { + Preconditions.checkState(!lifecycleStarted, "already started"); + Preconditions.checkState(!exec.isShutdown(), "already stopped"); + + // Try normal initialization first, if that fails then schedule periodic initialization retries + try { + tryInit(); + } + catch (Exception e) { + if (!started) { + log.warn("First initialization attempt failed for KafkaSupervisor[%s], starting retries...", dataSource); + + exec.submit( + () -> { + try { + RetryUtils.retry( + () -> { + tryInit(); + return 0; + }, + (throwable) -> !started, + 0, + MAX_INITIALIZATION_RETRIES, + null, + null + ); + } + catch (Exception e2) { + log.makeAlert( + "Failed to initialize after %s retries, aborting. Please resubmit the supervisor spec to restart this supervisor [%s]", + MAX_INITIALIZATION_RETRIES, + supervisorId + ).emit(); + throw new RuntimeException(e2); + } + } + ); + } } + lifecycleStarted = true; } } @@ -343,7 +405,7 @@ private Runnable buildRunTask() public void stop(boolean stopGracefully) { synchronized (stateChangeLock) { - Preconditions.checkState(started, "not started"); + Preconditions.checkState(lifecycleStarted, "lifecycle not started"); log.info("Beginning shutdown of [%s]", supervisorId); @@ -352,37 +414,39 @@ public void stop(boolean stopGracefully) reportingExec.shutdownNow(); recordSupplier.close(); - Optional taskRunner = taskMaster.getTaskRunner(); - if (taskRunner.isPresent()) { - taskRunner.get().unregisterListener(supervisorId); - } - - // Stopping gracefully will synchronize the end offsets of the tasks and signal them to publish, and will block - // until the tasks have acknowledged or timed out. We want this behavior when we're explicitly shut down through - // the API, but if we shut down for other reasons (e.g. we lose leadership) we want to just stop and leave the - // tasks as they are. - synchronized (stopLock) { - if (stopGracefully) { - log.info("Posting GracefulShutdownNotice, signalling managed tasks to complete and publish"); - notices.add(new GracefulShutdownNotice()); - } else { - log.info("Posting ShutdownNotice"); - notices.add(new ShutdownNotice()); + if (started) { + Optional taskRunner = taskMaster.getTaskRunner(); + if (taskRunner.isPresent()) { + taskRunner.get().unregisterListener(supervisorId); } - long shutdownTimeoutMillis = tuningConfig.getShutdownTimeout().getMillis(); - long endTime = System.currentTimeMillis() + shutdownTimeoutMillis; - while (!stopped) { - long sleepTime = endTime - System.currentTimeMillis(); - if (sleepTime <= 0) { - log.info("Timed out while waiting for shutdown (timeout [%,dms])", shutdownTimeoutMillis); - stopped = true; - break; + // Stopping gracefully will synchronize the end offsets of the tasks and signal them to publish, and will block + // until the tasks have acknowledged or timed out. We want this behavior when we're explicitly shut down through + // the API, but if we shut down for other reasons (e.g. we lose leadership) we want to just stop and leave the + // tasks as they are. + synchronized (stopLock) { + if (stopGracefully) { + log.info("Posting GracefulShutdownNotice, signalling managed tasks to complete and publish"); + notices.add(new GracefulShutdownNotice()); + } else { + log.info("Posting ShutdownNotice"); + notices.add(new ShutdownNotice()); + } + + long shutdownTimeoutMillis = tuningConfig.getShutdownTimeout().getMillis(); + long endTime = System.currentTimeMillis() + shutdownTimeoutMillis; + while (!stopped) { + long sleepTime = endTime - System.currentTimeMillis(); + if (sleepTime <= 0) { + log.info("Timed out while waiting for shutdown (timeout [%,dms])", shutdownTimeoutMillis); + stopped = true; + break; + } + stopLock.wait(sleepTime); } - stopLock.wait(sleepTime); } + log.info("Shutdown notice handled"); } - log.info("Shutdown notice handled"); taskClient.close(); workerExec.shutdownNow(); @@ -429,7 +493,7 @@ private SupervisorReport try { for (TaskGroup taskGroup : taskGroups.values()) { - for (Map.Entry entry : taskGroup.tasks.entrySet()) { + for (Entry entry : taskGroup.tasks.entrySet()) { String taskId = entry.getKey(); @Nullable DateTime startTime = entry.getValue().startTime; @@ -457,7 +521,7 @@ private SupervisorReport for (List taskGroups : pendingCompletionTaskGroups.values()) { for (TaskGroup taskGroup : taskGroups) { - for (Map.Entry entry : taskGroup.tasks.entrySet()) { + for (Entry entry : taskGroup.tasks.entrySet()) { String taskId = entry.getKey(); @Nullable DateTime startTime = entry.getValue().startTime; @@ -657,7 +721,7 @@ public void statusChanged(String taskId, TaskStatus status) protected void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException { for (TaskGroup taskGroup : taskGroups.values()) { - for (Map.Entry entry : + for (Entry entry : taskGroup.tasks.entrySet()) { if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) { killTask(entry.getKey()); @@ -708,9 +772,9 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) // defend against consecutive reset requests from replicas // as well as the case where the metadata store do not have an entry for the reset partitions boolean doReset = false; - for (Map.Entry resetPartitionOffset : resetMetadata.getSeekableStreamPartitions() - .getMap() - .entrySet()) { + for (Entry resetPartitionOffset : resetMetadata.getSeekableStreamPartitions() + .getMap() + .entrySet()) { final T2 partitionOffsetInMetadataStore = currentMetadata == null ? null : currentMetadata.getSeekableStreamPartitions() @@ -876,7 +940,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) // existing) so that the next tasks will start reading from where this task left off Map publishingTaskEndOffsets = taskClient.getEndOffsets(taskId); - for (Map.Entry entry : publishingTaskEndOffsets.entrySet()) { + for (Entry entry : publishingTaskEndOffsets.entrySet()) { T1 partition = entry.getKey(); T2 offset = entry.getValue(); ConcurrentHashMap partitionOffsets = partitionGroups.get( @@ -1256,7 +1320,7 @@ protected String generateSequenceName( { StringBuilder sb = new StringBuilder(); - for (Map.Entry entry : startPartitions.entrySet()) { + for (Entry entry : startPartitions.entrySet()) { sb.append(StringUtils.format("+%s(%s)", entry.getKey().toString(), entry.getValue().toString())); } String partitionOffsetStr = sb.toString().substring(1); @@ -1307,7 +1371,7 @@ private void updatePartitionDataFromStream() .entrySet() .stream() .filter(x -> Record.END_OF_SHARD_MARKER.equals(x.getValue())) - .map(Map.Entry::getKey) + .map(Entry::getKey) .collect(Collectors.toSet()); boolean initialPartitionDiscovery = this.partitionIds.isEmpty(); @@ -1346,7 +1410,7 @@ private void updateTaskStatus() throws ExecutionException, InterruptedException, // update status (and startTime if unknown) of current tasks in taskGroups for (TaskGroup group : taskGroups.values()) { - for (Map.Entry entry : group.tasks.entrySet()) { + for (Entry entry : group.tasks.entrySet()) { final String taskId = entry.getKey(); final TaskData taskData = entry.getValue(); @@ -1388,7 +1452,7 @@ public Boolean apply(@Nullable DateTime startTime) // update status of pending completion tasks in pendingCompletionTaskGroups for (List taskGroups : pendingCompletionTaskGroups.values()) { for (TaskGroup group : taskGroups) { - for (Map.Entry entry : group.tasks.entrySet()) { + for (Entry entry : group.tasks.entrySet()) { entry.getValue().status = taskStorage.getStatus(entry.getKey()).get(); } } @@ -1411,7 +1475,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException final List>> futures = Lists.newArrayList(); final List futureGroupIds = Lists.newArrayList(); - for (Map.Entry entry : taskGroups.entrySet()) { + for (Entry entry : taskGroups.entrySet()) { Integer groupId = entry.getKey(); TaskGroup group = entry.getValue(); @@ -1452,7 +1516,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException pendingCompletionTaskGroups.computeIfAbsent(groupId, k -> new CopyOnWriteArrayList<>()).add(group); // set endOffsets as the next startOffsets - for (Map.Entry entry : endOffsets.entrySet()) { + for (Entry entry : endOffsets.entrySet()) { partitionGroups.get(groupId).put(entry.getKey(), entry.getValue()); } } else { @@ -1478,9 +1542,9 @@ private ListenableFuture> checkpointTaskGroup(final TaskGroup taskGr { if (finalize) { // 1) Check if any task completed (in which case we're done) and kill unassigned tasks - Iterator> i = taskGroup.tasks.entrySet().iterator(); + Iterator> i = taskGroup.tasks.entrySet().iterator(); while (i.hasNext()) { - Map.Entry taskEntry = i.next(); + Entry taskEntry = i.next(); String taskId = taskEntry.getKey(); TaskData task = taskEntry.getValue(); @@ -1541,7 +1605,7 @@ public Map apply(List> input) taskGroup.tasks.remove(taskId); } else { // otherwise build a map of the highest offsets seen - for (Map.Entry offset : result.entrySet()) { + for (Entry offset : result.entrySet()) { if (!endOffsets.containsKey(offset.getKey()) || makeSequenceNumber(endOffsets.get(offset.getKey())).compareTo( makeSequenceNumber(offset.getValue())) < 0) { @@ -1618,7 +1682,7 @@ private ListenableFuture stopTasksInGroup(@Nullable TaskGroup taskGroup) } final List> futures = Lists.newArrayList(); - for (Map.Entry entry : taskGroup.tasks.entrySet()) { + for (Entry entry : taskGroup.tasks.entrySet()) { final String taskId = entry.getKey(); final TaskData taskData = entry.getValue(); if (taskData.status == null) { @@ -1636,7 +1700,7 @@ private void checkPendingCompletionTasks() { List> futures = Lists.newArrayList(); - for (Map.Entry> pendingGroupList : pendingCompletionTaskGroups.entrySet()) { + for (Entry> pendingGroupList : pendingCompletionTaskGroups.entrySet()) { boolean stopTasksInTaskGroup = false; Integer groupId = pendingGroupList.getKey(); @@ -1654,9 +1718,9 @@ private void checkPendingCompletionTasks() continue; } - Iterator> iTask = group.tasks.entrySet().iterator(); + Iterator> iTask = group.tasks.entrySet().iterator(); while (iTask.hasNext()) { - final Map.Entry entry = iTask.next(); + final Entry entry = iTask.next(); final String taskId = entry.getKey(); final TaskData taskData = entry.getValue(); @@ -1717,9 +1781,9 @@ private void checkPendingCompletionTasks() private void checkCurrentTaskState() throws ExecutionException, InterruptedException, TimeoutException { List> futures = Lists.newArrayList(); - Iterator> iTaskGroups = taskGroups.entrySet().iterator(); + Iterator> iTaskGroups = taskGroups.entrySet().iterator(); while (iTaskGroups.hasNext()) { - Map.Entry taskGroupEntry = iTaskGroups.next(); + Entry taskGroupEntry = iTaskGroups.next(); Integer groupId = taskGroupEntry.getKey(); TaskGroup taskGroup = taskGroupEntry.getValue(); @@ -1731,9 +1795,9 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep log.debug("Task group [%d] pre-pruning: %s", groupId, taskGroup.taskIds()); - Iterator> iTasks = taskGroup.tasks.entrySet().iterator(); + Iterator> iTasks = taskGroup.tasks.entrySet().iterator(); while (iTasks.hasNext()) { - Map.Entry task = iTasks.next(); + Entry task = iTasks.next(); String taskId = task.getKey(); TaskData taskData = task.getValue(); @@ -1800,14 +1864,14 @@ private void createNewTasks() throws JsonProcessingException .stream() .filter(x -> x.getValue().get() != null) .collect(Collectors.collectingAndThen( - Collectors.toMap(Map.Entry::getKey, x -> x.getValue().get()), + Collectors.toMap(Entry::getKey, x -> x.getValue().get()), ImmutableMap::copyOf )); Set exclusiveStartSequenceNumberPartitions = startingOffsets .entrySet().stream() .filter(x -> x.getValue().get() != null && x.getValue().isExclusive()) - .map(Map.Entry::getKey) + .map(Entry::getKey) .collect(Collectors.toSet()); taskGroups.put( @@ -1832,7 +1896,7 @@ private void createNewTasks() throws JsonProcessingException // iterate through all the current task groups and make sure each one has the desired number of replica tasks boolean createdTask = false; - for (Map.Entry entry : taskGroups.entrySet()) { + for (Entry entry : taskGroups.entrySet()) { TaskGroup taskGroup = entry.getValue(); Integer groupId = entry.getKey(); @@ -1891,7 +1955,7 @@ private ImmutableMap> generateStartingSequencesForPartiti throws TimeoutException { ImmutableMap.Builder> builder = ImmutableMap.builder(); - for (Map.Entry entry : partitionGroups.get(groupId).entrySet()) { + for (Entry entry : partitionGroups.get(groupId).entrySet()) { T1 partition = entry.getKey(); T2 offset = entry.getValue(); @@ -2132,8 +2196,8 @@ protected Map getHighestCurrentOffsets() .flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()) .flatMap(taskData -> taskData.getValue().currentSequences.entrySet().stream()) .collect(Collectors.toMap( - Map.Entry::getKey, - Map.Entry::getValue, + Entry::getKey, + Entry::getValue, (v1, v2) -> makeSequenceNumber(v1).compareTo(makeSequenceNumber(v2)) > 0 ? v1 : v2 )); } @@ -2371,7 +2435,7 @@ public void handle() throws ExecutionException, InterruptedException return taskGroup.baseSequenceName.equals(baseSequenceName); }) .findAny() - .map(Map.Entry::getKey); + .map(Entry::getKey); taskGroupId = maybeGroupId.orElse( pendingCompletionTaskGroups .entrySet() @@ -2452,4 +2516,32 @@ private SequenceNumber makeSequenceNumber(T2 seq) protected abstract SequenceNumber makeSequenceNumber(T2 seq, boolean useExclusive, boolean isExclusive); + // exposed for testing for visibility into initialization state + @VisibleForTesting + public boolean isStarted() + { + return started; + } + + // exposed for testing for visibility into initialization state + @VisibleForTesting + public boolean isLifecycleStarted() + { + return lifecycleStarted; + } + + // exposed for testing for visibility into initialization state + @VisibleForTesting + public int getInitRetryCounter() + { + return initRetryCounter; + } + + // exposed for testing to allow "bootstrap.servers" to be changed after supervisor is created + @VisibleForTesting + public SeekableStreamSupervisorIOConfig getIoConfig() + { + return ioConfig; + } + } From 1e254e0de6d1ad811b135472c8035a9405aff167 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 4 Oct 2018 17:24:22 -0700 Subject: [PATCH 20/87] added more docs and reordered methods --- .../supervisor/SeekableStreamSupervisor.java | 1029 +++++++++-------- 1 file changed, 540 insertions(+), 489 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index cca4b447dcc8..9a64183d0e39 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -113,6 +113,14 @@ *

* incremental handoff & checkpointing are not yet supported by Kinesis, but the logic is left in here * so in the future it's easier to implement + *

+ * Supervisor responsible for managing the SeekableStreamIndexTasks (Kafka/Kinesis) for a single dataSource. At a high level, the class accepts a + * {@link SeekableStreamSupervisorSpec} which includes the stream name (topic / stream) and configuration as well as an ingestion spec which will + * be used to generate the indexing tasks. The run loop periodically refreshes its view of the stream's partitions + * and the list of running indexing tasks and ensures that all partitions are being read from and that there are enough + * tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of + * stream sequences. + *

* * @param partition id type * @param sequence number type @@ -127,11 +135,325 @@ public abstract class SeekableStreamSupervisor private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120; private static final int MAX_INITIALIZATION_RETRIES = 20; private static final CopyOnWriteArrayList EMPTY_LIST = Lists.newCopyOnWriteArrayList(); - protected final List partitionIds = new CopyOnWriteArrayList<>(); - protected final Set subsequentlyDiscoveredPartitions = new HashSet<>(); + + // Internal data structures + // -------------------------------------------------------- + + /** + * A TaskGroup is the main data structure used by SeekableStreamSupervisor to organize and monitor stream partitions and + * indexing tasks. All the tasks in a TaskGroup should always be doing the same thing (reading the same partitions and + * starting from the same sequences) and if [replicas] is configured to be 1, a TaskGroup will contain a single task (the + * exception being if the supervisor started up and discovered and adopted some already running tasks). At any given + * 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]). + */ + private class TaskGroup + { + final int groupId; + + // 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 + // this task group has completed successfully, at which point this will be destroyed and a new task group will be + // created with new starting offsets. This allows us to create replacement tasks for failed tasks that process the + // same offsets, even if the values in [partitionGroups] has been changed. + final ImmutableMap startingSequences; + + final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); + final Optional minimumMessageTime; + final Optional maximumMessageTime; + final Set exclusiveStartSequenceNumberPartitions; + final TreeMap> checkpointSequences = new TreeMap<>(); + final String baseSequenceName; + DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action + + TaskGroup( + int groupId, + ImmutableMap startingSequences, + Optional minimumMessageTime, + Optional maximumMessageTime, + Set exclusiveStartSequenceNumberPartitions + ) + { + this.groupId = groupId; + this.startingSequences = startingSequences; + this.minimumMessageTime = minimumMessageTime; + this.maximumMessageTime = maximumMessageTime; + this.checkpointSequences.put(0, startingSequences); + this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions != null + ? exclusiveStartSequenceNumberPartitions + : new HashSet<>(); + this.baseSequenceName = generateSequenceName(startingSequences, minimumMessageTime, maximumMessageTime); + } + + int addNewCheckpoint(Map checkpoint) + { + checkpointSequences.put(checkpointSequences.lastKey() + 1, checkpoint); + return checkpointSequences.lastKey(); + } + + Set taskIds() + { + return tasks.keySet(); + } + + } + + private class TaskData + { + volatile TaskStatus status; + volatile DateTime startTime; + volatile Map currentSequences = new HashMap<>(); + + @Override + public String toString() + { + return "TaskData{" + + "status=" + status + + ", startTime=" + startTime + + ", checkpointSequences=" + currentSequences + + '}'; + } + } + + /** + * Notice is used to queue tasks that are internal to the supervisor + */ + private interface Notice + { + void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException; + } + + private static class StatsFromTaskResult + { + private final String groupId; + private final String taskId; + private final Map stats; + + public StatsFromTaskResult( + int groupId, + String taskId, + Map stats + ) + { + this.groupId = String.valueOf(groupId); + this.taskId = taskId; + this.stats = stats; + } + + public String getGroupId() + { + return groupId; + } + + public String getTaskId() + { + return taskId; + } + + public Map getStats() + { + return stats; + } + } + + + private class RunNotice implements Notice + { + @Override + public void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException + { + long nowTime = System.currentTimeMillis(); + if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { + return; + } + lastRunTime = nowTime; + + runInternal(); + } + } + + private class GracefulShutdownNotice extends ShutdownNotice + { + @Override + public void handle() throws InterruptedException, ExecutionException, TimeoutException + { + gracefulShutdownInternal(); + super.handle(); + } + } + + private class ShutdownNotice implements Notice + { + @Override + public void handle() throws InterruptedException, ExecutionException, TimeoutException + { + recordSupplier.close(); + + synchronized (stopLock) { + stopped = true; + stopLock.notifyAll(); + } + } + } + + private class ResetNotice implements Notice + { + final DataSourceMetadata dataSourceMetadata; + + ResetNotice(DataSourceMetadata dataSourceMetadata) + { + this.dataSourceMetadata = dataSourceMetadata; + } + + @Override + public void handle() + { + resetInternal(dataSourceMetadata); + } + } + + protected class CheckpointNotice implements Notice + { + @Nullable + private final Integer nullableTaskGroupId; + @Deprecated + private final String baseSequenceName; + private final SeekableStreamDataSourceMetadata previousCheckpoint; + private final SeekableStreamDataSourceMetadata currentCheckpoint; + + public CheckpointNotice( + @Nullable Integer nullableTaskGroupId, + @Deprecated String baseSequenceName, + SeekableStreamDataSourceMetadata previousCheckpoint, + SeekableStreamDataSourceMetadata currentCheckpoint + ) + { + this.baseSequenceName = baseSequenceName; + this.nullableTaskGroupId = nullableTaskGroupId; + this.previousCheckpoint = previousCheckpoint; + this.currentCheckpoint = currentCheckpoint; + } + + @Override + public void handle() throws ExecutionException, InterruptedException + { + // Find taskGroupId using taskId if it's null. It can be null while rolling update. + final int taskGroupId; + if (nullableTaskGroupId == null) { + // We search taskId in taskGroups and pendingCompletionTaskGroups sequentially. This should be fine because + // 1) a taskGroup can be moved from taskGroups to pendingCompletionTaskGroups in RunNotice + // (see checkTaskDuration()). + // 2) Notices are proceesed by a single thread. So, CheckpointNotice and RunNotice cannot be processed at the + // same time. + final java.util.Optional maybeGroupId = taskGroups + .entrySet() + .stream() + .filter(entry -> { + final TaskGroup taskGroup = entry.getValue(); + return taskGroup.baseSequenceName.equals(baseSequenceName); + }) + .findAny() + .map(Entry::getKey); + taskGroupId = maybeGroupId.orElse( + pendingCompletionTaskGroups + .entrySet() + .stream() + .filter(entry -> { + final List taskGroups = entry.getValue(); + return taskGroups.stream().anyMatch(group -> group.baseSequenceName.equals(baseSequenceName)); + }) + .findAny() + .orElseThrow(() -> new ISE("Cannot find taskGroup for baseSequenceName[%s]", baseSequenceName)) + .getKey() + ); + } else { + taskGroupId = nullableTaskGroupId; + } + + // check for consistency + // if already received request for this sequenceName and dataSourceMetadata combination then return + final TaskGroup taskGroup = taskGroups.get(taskGroupId); + + if (isValidTaskGroup(taskGroupId, taskGroup)) { + final TreeMap> checkpoints = taskGroup.checkpointSequences; + + // check validity of previousCheckpoint + int index = checkpoints.size(); + for (int sequenceId : checkpoints.descendingKeySet()) { + Map checkpoint = checkpoints.get(sequenceId); + // We have already verified the topic of the current checkpoint is same with that in ioConfig. + // See checkpoint(). + if (checkpoint.equals(previousCheckpoint.getSeekableStreamPartitions() + .getMap() + )) { + break; + } + index--; + } + if (index == 0) { + throw new ISE("No such previous checkpoint [%s] found", previousCheckpoint); + } else if (index < checkpoints.size()) { + // if the found checkpoint is not the latest one then already checkpointed by a replica + Preconditions.checkState(index == checkpoints.size() - 1, "checkpoint consistency failure"); + log.info("Already checkpointed with offsets [%s]", checkpoints.lastEntry().getValue()); + return; + } + final Map newCheckpoint = checkpointTaskGroup(taskGroup, false).get(); + taskGroup.addNewCheckpoint(newCheckpoint); + log.info("Handled checkpoint notice, new checkpoint is [%s] for taskGroup [%s]", newCheckpoint, taskGroupId); + } + } + + protected boolean isValidTaskGroup(int taskGroupId, @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); + } + } + + return true; + } + } + + + // Map<{group ID}, {actively reading task group}>; see documentation for TaskGroup class private final ConcurrentHashMap taskGroups = new ConcurrentHashMap<>(); + + // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [taskGroups] to here so + // we can monitor its status while we queue new tasks to read the next range of offsets. This is a list since we could + // have multiple sets of tasks publishing at once if time-to-publish > taskDuration. + // Map<{group ID}, List<{pending completion task groups}>> private final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); + + // 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 + // offset value from the metadata store, and if it can't find it there, from stream. Once a task begins + // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet- + // completed task, which will cause the next set of tasks to begin reading from where the previous task left + // off. If that previous task now fails, we will set the offset in [partitionGroups] back to NOT_SET which will + // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to + // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task + // failures during publishing. + // Map<{group ID}, Map<{partition ID}, {startingOffset}>> private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); + + protected final List partitionIds = new CopyOnWriteArrayList<>(); + protected volatile Map latestSequenceFromStream; + protected volatile DateTime sequenceLastUpdated; + + private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); private final TaskStorage taskStorage; private final TaskMaster taskMaster; private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; @@ -156,8 +478,6 @@ public abstract class SeekableStreamSupervisor private final Object recordSupplierLock = new Object(); private final T2 NOT_SET; private final boolean useExclusiveStartingSequence; - protected volatile Map latestSequenceFromStream; - protected volatile DateTime sequenceLastUpdated; private boolean listenerRegistered = false; private long lastRunTime; private int initRetryCounter = 0; @@ -271,80 +591,6 @@ protected static String getRandomId() return suffix.toString(); } - @VisibleForTesting - protected void tryInit() - { - synchronized (stateChangeLock) { - if (started) { - log.warn("SUpervisor was already started, skipping init"); - return; - } - - if (stopped) { - log.warn("Supervisor was already stopped, skipping init."); - return; - } - - try { - recordSupplier = setupRecordSupplier(); - - exec.submit( - () -> { - try { - long pollTimeout = Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS); - while (!Thread.currentThread().isInterrupted() && !stopped) { - final Notice notice = notices.poll(pollTimeout, TimeUnit.MILLISECONDS); - if (notice == null) { - continue; - } - - try { - notice.handle(); - } - catch (Throwable e) { - log.makeAlert(e, "KafkaSupervisor[%s] failed to handle notice", dataSource) - .addData("noticeClass", notice.getClass().getSimpleName()) - .emit(); - } - } - } - catch (InterruptedException e) { - log.info("KafkaSupervisor[%s] interrupted, exiting", dataSource); - } - } - ); - firstRunTime = DateTimes.nowUtc().plus(ioConfig.getStartDelay()); - scheduledExec.scheduleAtFixedRate( - buildRunTask(), - ioConfig.getStartDelay().getMillis(), - Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS), - TimeUnit.MILLISECONDS - ); - - // not yet implemented in kinesis, will remove once implemented - scheduleReporting(reportingExec); - - started = true; - log.info( - "Started KafkaSupervisor[%s], first run in [%s], with spec: [%s]", - dataSource, - ioConfig.getStartDelay(), - spec.toString() - ); - } - catch (Exception e) { - if (recordSupplier != null) { - recordSupplier.close(); - } - initRetryCounter++; - log.makeAlert(e, "Exception starting KafkaSupervisor[%s]", dataSource) - .emit(); - - throw new RuntimeException(e); - } - } - } - @Override public void start() @@ -359,7 +605,10 @@ public void start() } catch (Exception e) { if (!started) { - log.warn("First initialization attempt failed for KafkaSupervisor[%s], starting retries...", dataSource); + log.warn( + "First initialization attempt failed for SeekableStreamSupervisor[%s], starting retries...", + dataSource + ); exec.submit( () -> { @@ -392,15 +641,6 @@ public void start() } } - protected abstract RecordSupplier setupRecordSupplier(); - - private Runnable buildRunTask() - { - return () -> notices.add(new RunNotice()); - } - - protected abstract void scheduleReporting(ScheduledExecutorService reportingExec); - @Override public void stop(boolean stopGracefully) { @@ -469,6 +709,86 @@ public void reset(DataSourceMetadata dataSourceMetadata) notices.add(new ResetNotice(dataSourceMetadata)); } + + @VisibleForTesting + protected void tryInit() + { + synchronized (stateChangeLock) { + if (started) { + log.warn("SUpervisor was already started, skipping init"); + return; + } + + if (stopped) { + log.warn("Supervisor was already stopped, skipping init."); + return; + } + + try { + recordSupplier = setupRecordSupplier(); + + exec.submit( + () -> { + try { + long pollTimeout = Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS); + while (!Thread.currentThread().isInterrupted() && !stopped) { + final Notice notice = notices.poll(pollTimeout, TimeUnit.MILLISECONDS); + if (notice == null) { + continue; + } + + try { + notice.handle(); + } + catch (Throwable e) { + log.makeAlert(e, "SeekableStreamSupervisor[%s] failed to handle notice", dataSource) + .addData("noticeClass", notice.getClass().getSimpleName()) + .emit(); + } + } + } + catch (InterruptedException e) { + log.info("SeekableStreamSupervisor[%s] interrupted, exiting", dataSource); + } + } + ); + firstRunTime = DateTimes.nowUtc().plus(ioConfig.getStartDelay()); + scheduledExec.scheduleAtFixedRate( + buildRunTask(), + ioConfig.getStartDelay().getMillis(), + Math.max(ioConfig.getPeriod().getMillis(), MAX_RUN_FREQUENCY_MILLIS), + TimeUnit.MILLISECONDS + ); + + // not yet implemented in kinesis, will remove once implemented + scheduleReporting(reportingExec); + + started = true; + log.info( + "Started SeekableStreamSupervisor[%s], first run in [%s], with spec: [%s]", + dataSource, + ioConfig.getStartDelay(), + spec.toString() + ); + } + catch (Exception e) { + if (recordSupplier != null) { + recordSupplier.close(); + } + initRetryCounter++; + log.makeAlert(e, "Exception starting SeekableStreamSupervisor[%s]", dataSource) + .emit(); + + throw new RuntimeException(e); + } + } + } + + private Runnable buildRunTask() + { + return () -> notices.add(new RunNotice()); + } + @Override public SupervisorReport getStatus() { @@ -556,13 +876,6 @@ private SupervisorReport return report; } - protected abstract SeekableStreamSupervisorReportPayload createReportPayload( - int numPartitions, - boolean includeOffsets - ); - - protected abstract Map getLagPerPartition(Map currentOffsets); - @Override public Map> getStats() @@ -643,17 +956,9 @@ private Map> getCurrentTotalStats() } } - return allStats; - } - - @Override - public abstract void checkpoint( - @Nullable Integer taskGroupId, - String baseSequenceName, - DataSourceMetadata previousCheckPoint, - DataSourceMetadata currentCheckPoint - ); - + return allStats; + } + @VisibleForTesting protected void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException @@ -1977,9 +2282,6 @@ private ImmutableMap> generateStartingSequencesForPartiti return builder.build(); } - protected abstract boolean checkSequenceAvailability(@NotNull T1 partition, @NotNull T2 sequenceFromMetadata) - throws TimeoutException; - /** * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't * find any data, it will retrieve the latest or earliest Kafka/Kinesis offset depending on the useEarliestOffset config. @@ -2108,405 +2410,98 @@ private void createTasksForGroup(int groupId, int replicas) throws JsonProcessin SeekableStreamIndexTask indexTask = taskList.get(i); if (taskQueue.isPresent()) { try { - taskQueue.get().add(indexTask); - } - catch (EntryExistsException e) { - log.error("Tried to add task [%s] but it already exists", indexTask.getId()); - } - } else { - log.error("Failed to get task queue because I'm not the leader!"); - } - } - } - - @VisibleForTesting - protected Runnable updateCurrentAndLatestOffsets() - { - return () -> { - try { - updateCurrentOffsets(); - updateLatestOffsetsFromStream(); - sequenceLastUpdated = DateTimes.nowUtc(); - } - catch (Exception e) { - log.warn(e, "Exception while getting current/latest offsets"); - } - }; - } - - private void updateCurrentOffsets() throws InterruptedException, ExecutionException, TimeoutException - { - final List> futures = Stream.concat( - taskGroups.values().stream().flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()), - pendingCompletionTaskGroups.values() - .stream() - .flatMap(List::stream) - .flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()) - ).map( - task -> Futures.transform( - taskClient.getCurrentOffsetsAsync(task.getKey(), false), - (Function, Void>) (currentSequences) -> { - - if (currentSequences != null && !currentSequences.isEmpty()) { - task.getValue().currentSequences = currentSequences; - } - - return null; - } - ) - ).collect(Collectors.toList()); - - Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); - } - - private void updateLatestOffsetsFromStream() - { - synchronized (recordSupplierLock) { - Set partitionIds = null; - try { - partitionIds = recordSupplier.getPartitionIds(ioConfig.getId()); - } - catch (Exception e) { - log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getId()); - Throwables.propagate(e); - } - - Set> partitions = partitionIds - .stream() - .map(e -> new StreamPartition<>(ioConfig.getId(), e)) - .collect(Collectors.toSet()); - - recordSupplier.assign(partitions); - recordSupplier.seekToLatest(partitions); - - latestSequenceFromStream = partitions.stream() - .collect(Collectors.toMap( - StreamPartition::getPartitionId, - x -> recordSupplier.position(x) - )); - } - - } - - protected Map getHighestCurrentOffsets() - { - return taskGroups - .values() - .stream() - .flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()) - .flatMap(taskData -> taskData.getValue().currentSequences.entrySet().stream()) - .collect(Collectors.toMap( - Entry::getKey, - Entry::getValue, - (v1, v2) -> makeSequenceNumber(v1).compareTo(makeSequenceNumber(v2)) > 0 ? v1 : v2 - )); - } - - protected abstract SeekableStreamIOConfig createIoConfig( - int groupId, - Map startPartitions, - Map endPartitions, - String baseSequenceName, - DateTime minimumMessageTime, - DateTime maximumMessageTime, - Set exclusiveStartSequenceNumberPartitions, - SeekableStreamSupervisorIOConfig ioConfig - ); - - protected abstract List> createIndexTasks( - int replicas, - String baseSequenceName, - ObjectMapper sortingMapper, - TreeMap> sequenceOffsets, - SeekableStreamIOConfig taskIoConfig, - SeekableStreamTuningConfig taskTuningConfig, - RowIngestionMetersFactory rowIngestionMetersFactory - ) throws JsonProcessingException; - - protected abstract int getTaskGroupIdForPartition(T1 partition); - - protected abstract boolean checkSourceMetaDataMatch(DataSourceMetadata metadata); - - protected abstract boolean checkTaskInstance(Task task); - - protected abstract SeekableStreamDataSourceMetadata createDataSourceMetaData( - String - stream, Map map - ); - - protected abstract Map createNewTaskEndPartitions(Set startPartitions); - - /** - * Notice is used to queue tasks that are internal to the supervisor - */ - private interface Notice - { - void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException; - } - - private static class StatsFromTaskResult - { - private final String groupId; - private final String taskId; - private final Map stats; - - public StatsFromTaskResult( - int groupId, - String taskId, - Map stats - ) - { - this.groupId = String.valueOf(groupId); - this.taskId = taskId; - this.stats = stats; - } - - public String getGroupId() - { - return groupId; - } - - public String getTaskId() - { - return taskId; - } - - public Map getStats() - { - return stats; - } - } - - private class TaskGroup - { - final int groupId; - - final ImmutableMap startingSequences; - final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); - final Optional minimumMessageTime; - final Optional maximumMessageTime; - final Set exclusiveStartSequenceNumberPartitions; - final TreeMap> checkpointSequences = new TreeMap<>(); - final String baseSequenceName; - DateTime completionTimeout; - - public TaskGroup( - int groupId, - ImmutableMap startingSequences, - Optional minimumMessageTime, - Optional maximumMessageTime, - Set exclusiveStartSequenceNumberPartitions - ) - { - this.groupId = groupId; - this.startingSequences = startingSequences; - this.minimumMessageTime = minimumMessageTime; - this.maximumMessageTime = maximumMessageTime; - this.checkpointSequences.put(0, startingSequences); - this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions != null - ? exclusiveStartSequenceNumberPartitions - : new HashSet<>(); - this.baseSequenceName = generateSequenceName(startingSequences, minimumMessageTime, maximumMessageTime); - } - - int addNewCheckpoint(Map checkpoint) - { - checkpointSequences.put(checkpointSequences.lastKey() + 1, checkpoint); - return checkpointSequences.lastKey(); - } - - public Set taskIds() - { - return tasks.keySet(); - } - - } - - private class TaskData - { - volatile TaskStatus status; - volatile DateTime startTime; - volatile Map currentSequences = new HashMap<>(); - - @Override - public String toString() - { - return "TaskData{" + - "status=" + status + - ", startTime=" + startTime + - ", checkpointSequences=" + currentSequences + - '}'; - } - } - - private class RunNotice implements Notice - { - @Override - public void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException - { - long nowTime = System.currentTimeMillis(); - if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { - return; - } - lastRunTime = nowTime; - - runInternal(); - } - } - - private class GracefulShutdownNotice extends ShutdownNotice - { - @Override - public void handle() throws InterruptedException, ExecutionException, TimeoutException - { - gracefulShutdownInternal(); - super.handle(); + taskQueue.get().add(indexTask); + } + catch (EntryExistsException e) { + log.error("Tried to add task [%s] but it already exists", indexTask.getId()); + } + } else { + log.error("Failed to get task queue because I'm not the leader!"); + } } } - private class ShutdownNotice implements Notice + @VisibleForTesting + protected Runnable updateCurrentAndLatestOffsets() { - @Override - public void handle() throws InterruptedException, ExecutionException, TimeoutException - { - recordSupplier.close(); - - synchronized (stopLock) { - stopped = true; - stopLock.notifyAll(); + return () -> { + try { + updateCurrentOffsets(); + updateLatestOffsetsFromStream(); + sequenceLastUpdated = DateTimes.nowUtc(); } - } + catch (Exception e) { + log.warn(e, "Exception while getting current/latest offsets"); + } + }; } - private class ResetNotice implements Notice + private void updateCurrentOffsets() throws InterruptedException, ExecutionException, TimeoutException { - final DataSourceMetadata dataSourceMetadata; + final List> futures = Stream.concat( + taskGroups.values().stream().flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()), + pendingCompletionTaskGroups.values() + .stream() + .flatMap(List::stream) + .flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()) + ).map( + task -> Futures.transform( + taskClient.getCurrentOffsetsAsync(task.getKey(), false), + (Function, Void>) (currentSequences) -> { - ResetNotice(DataSourceMetadata dataSourceMetadata) - { - this.dataSourceMetadata = dataSourceMetadata; - } + if (currentSequences != null && !currentSequences.isEmpty()) { + task.getValue().currentSequences = currentSequences; + } - @Override - public void handle() - { - resetInternal(dataSourceMetadata); - } + return null; + } + ) + ).collect(Collectors.toList()); + + Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); } - protected class CheckpointNotice implements Notice + private void updateLatestOffsetsFromStream() { - @Nullable - private final Integer nullableTaskGroupId; - @Deprecated - private final String baseSequenceName; - private final SeekableStreamDataSourceMetadata previousCheckpoint; - private final SeekableStreamDataSourceMetadata currentCheckpoint; - - public CheckpointNotice( - @Nullable Integer nullableTaskGroupId, - @Deprecated String baseSequenceName, - SeekableStreamDataSourceMetadata previousCheckpoint, - SeekableStreamDataSourceMetadata currentCheckpoint - ) - { - this.baseSequenceName = baseSequenceName; - this.nullableTaskGroupId = nullableTaskGroupId; - this.previousCheckpoint = previousCheckpoint; - this.currentCheckpoint = currentCheckpoint; - } - - @Override - public void handle() throws ExecutionException, InterruptedException - { - // Find taskGroupId using taskId if it's null. It can be null while rolling update. - final int taskGroupId; - if (nullableTaskGroupId == null) { - // We search taskId in taskGroups and pendingCompletionTaskGroups sequentially. This should be fine because - // 1) a taskGroup can be moved from taskGroups to pendingCompletionTaskGroups in RunNotice - // (see checkTaskDuration()). - // 2) Notices are proceesed by a single thread. So, CheckpointNotice and RunNotice cannot be processed at the - // same time. - final java.util.Optional maybeGroupId = taskGroups - .entrySet() - .stream() - .filter(entry -> { - final TaskGroup taskGroup = entry.getValue(); - return taskGroup.baseSequenceName.equals(baseSequenceName); - }) - .findAny() - .map(Entry::getKey); - taskGroupId = maybeGroupId.orElse( - pendingCompletionTaskGroups - .entrySet() - .stream() - .filter(entry -> { - final List taskGroups = entry.getValue(); - return taskGroups.stream().anyMatch(group -> group.baseSequenceName.equals(baseSequenceName)); - }) - .findAny() - .orElseThrow(() -> new ISE("Cannot find taskGroup for baseSequenceName[%s]", baseSequenceName)) - .getKey() - ); - } else { - taskGroupId = nullableTaskGroupId; + synchronized (recordSupplierLock) { + Set partitionIds = null; + try { + partitionIds = recordSupplier.getPartitionIds(ioConfig.getId()); + } + catch (Exception e) { + log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getId()); + Throwables.propagate(e); } - // check for consistency - // if already received request for this sequenceName and dataSourceMetadata combination then return - final TaskGroup taskGroup = taskGroups.get(taskGroupId); + Set> partitions = partitionIds + .stream() + .map(e -> new StreamPartition<>(ioConfig.getId(), e)) + .collect(Collectors.toSet()); - if (isValidTaskGroup(taskGroupId, taskGroup)) { - final TreeMap> checkpoints = taskGroup.checkpointSequences; + recordSupplier.assign(partitions); + recordSupplier.seekToLatest(partitions); - // check validity of previousCheckpoint - int index = checkpoints.size(); - for (int sequenceId : checkpoints.descendingKeySet()) { - Map checkpoint = checkpoints.get(sequenceId); - // We have already verified the topic of the current checkpoint is same with that in ioConfig. - // See checkpoint(). - if (checkpoint.equals(previousCheckpoint.getSeekableStreamPartitions() - .getMap() - )) { - break; - } - index--; - } - if (index == 0) { - throw new ISE("No such previous checkpoint [%s] found", previousCheckpoint); - } else if (index < checkpoints.size()) { - // if the found checkpoint is not the latest one then already checkpointed by a replica - Preconditions.checkState(index == checkpoints.size() - 1, "checkpoint consistency failure"); - log.info("Already checkpointed with offsets [%s]", checkpoints.lastEntry().getValue()); - return; - } - final Map newCheckpoint = checkpointTaskGroup(taskGroup, false).get(); - taskGroup.addNewCheckpoint(newCheckpoint); - log.info("Handled checkpoint notice, new checkpoint is [%s] for taskGroup [%s]", newCheckpoint, taskGroupId); - } + latestSequenceFromStream = partitions.stream() + .collect(Collectors.toMap( + StreamPartition::getPartitionId, + x -> recordSupplier.position(x) + )); } - protected boolean isValidTaskGroup(int taskGroupId, @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); - } - } + } - return true; - } + protected Map getHighestCurrentOffsets() + { + return taskGroups + .values() + .stream() + .flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()) + .flatMap(taskData -> taskData.getValue().currentSequences.entrySet().stream()) + .collect(Collectors.toMap( + Entry::getKey, + Entry::getValue, + (v1, v2) -> makeSequenceNumber(v1).compareTo(makeSequenceNumber(v2)) > 0 ? v1 : v2 + )); } private SequenceNumber makeSequenceNumber(T2 seq) @@ -2514,7 +2509,6 @@ private SequenceNumber makeSequenceNumber(T2 seq) return makeSequenceNumber(seq, false, false); } - protected abstract SequenceNumber makeSequenceNumber(T2 seq, boolean useExclusive, boolean isExclusive); // exposed for testing for visibility into initialization state @VisibleForTesting @@ -2544,4 +2538,61 @@ public SeekableStreamSupervisorIOConfig getIoConfig() return ioConfig; } + @Override + public abstract void checkpoint( + @Nullable Integer taskGroupId, + String baseSequenceName, + DataSourceMetadata previousCheckPoint, + DataSourceMetadata currentCheckPoint + ); + + protected abstract SeekableStreamIOConfig createIoConfig( + int groupId, + Map startPartitions, + Map endPartitions, + String baseSequenceName, + DateTime minimumMessageTime, + DateTime maximumMessageTime, + Set exclusiveStartSequenceNumberPartitions, + SeekableStreamSupervisorIOConfig ioConfig + ); + + protected abstract List> createIndexTasks( + int replicas, + String baseSequenceName, + ObjectMapper sortingMapper, + TreeMap> sequenceOffsets, + SeekableStreamIOConfig taskIoConfig, + SeekableStreamTuningConfig taskTuningConfig, + RowIngestionMetersFactory rowIngestionMetersFactory + ) throws JsonProcessingException; + + protected abstract int getTaskGroupIdForPartition(T1 partition); + + protected abstract boolean checkSourceMetaDataMatch(DataSourceMetadata metadata); + + protected abstract boolean checkTaskInstance(Task task); + + protected abstract SeekableStreamDataSourceMetadata createDataSourceMetaData( + String + stream, Map map + ); + + protected abstract Map createNewTaskEndPartitions(Set startPartitions); + + protected abstract SequenceNumber makeSequenceNumber(T2 seq, boolean useExclusive, boolean isExclusive); + + protected abstract void scheduleReporting(ScheduledExecutorService reportingExec); + + protected abstract Map getLagPerPartition(Map currentOffsets); + + protected abstract RecordSupplier setupRecordSupplier(); + + protected abstract SeekableStreamSupervisorReportPayload createReportPayload( + int numPartitions, + boolean includeOffsets + ); + + protected abstract boolean checkSequenceAvailability(@NotNull T1 partition, @NotNull T2 sequenceFromMetadata) + throws TimeoutException; } From 8357fb034bd0bb9e769fb296947bea4057c16db6 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 8 Oct 2018 14:49:14 -0700 Subject: [PATCH 21/87] fixd kinesis tests after merging master and added docs in seekablestream --- .../indexing/kafka/KafkaRecordSupplier.java | 3 +- .../kafka/supervisor/KafkaSupervisor.java | 14 +- .../kinesis/supervisor/KinesisSupervisor.java | 16 +- .../indexing/kinesis/KinesisIOConfigTest.java | 19 - .../kinesis/KinesisIndexTaskTest.java | 8 +- .../supervisor/KinesisSupervisorTest.java | 366 ------------------ .../SeekableStreamIndexTask.java | 3 +- .../SeekableStreamPartitions.java | 10 +- .../seekablestream/common/Record.java | 7 + .../seekablestream/common/SequenceNumber.java | 4 +- .../common/StreamPartition.java | 4 +- .../supervisor/SeekableStreamSupervisor.java | 108 +++++- 12 files changed, 128 insertions(+), 434 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 540c3eb00d46..72599d0779e5 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -43,13 +43,14 @@ import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; public class KafkaRecordSupplier implements RecordSupplier { private static final EmittingLogger log = new EmittingLogger(KafkaRecordSupplier.class); - private static final Random RANDOM = new Random(); + private static final Random RANDOM = ThreadLocalRandom.current(); private final KafkaConsumer consumer; private final KafkaSupervisorIOConfig ioConfig; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 2ab5bfd04948..cc8a8a7b1792 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -60,7 +60,6 @@ import javax.annotation.Nullable; import javax.validation.constraints.NotNull; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -114,6 +113,7 @@ public KafkaSupervisor( spec, rowIngestionMetersFactory, NOT_SET, + Long.MAX_VALUE, false ); @@ -192,7 +192,7 @@ protected int getTaskGroupIdForPartition(Integer partition) } @Override - protected boolean checkSourceMetaDataMatch(DataSourceMetadata metadata) + protected boolean checkSourceMetadataMatch(DataSourceMetadata metadata) { return metadata instanceof KafkaDataSourceMetadata; } @@ -323,16 +323,6 @@ protected KafkaDataSourceMetadata createDataSourceMetaData( return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, map)); } - @Override - protected Map createNewTaskEndPartitions(Set startPartitions) - { - Map endPartitions = new HashMap<>(); - for (int partition : startPartitions) { - endPartitions.put(partition, Long.MAX_VALUE); - } - return endPartitions; - } - @Override protected SequenceNumber makeSequenceNumber( Long seq, boolean useExclusive, boolean isExclusive diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index f968e99cc650..7f42b76a4879 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -54,7 +54,6 @@ import javax.annotation.Nullable; import javax.validation.constraints.NotNull; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -75,7 +74,7 @@ */ public class KinesisSupervisor extends SeekableStreamSupervisor { - private static final String NOT_SET = ""; + private static final String NOT_SET = "-1"; private final KinesisSupervisorSpec spec; public KinesisSupervisor( @@ -98,6 +97,7 @@ public KinesisSupervisor( spec, rowIngestionMetersFactory, NOT_SET, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, true ); @@ -227,7 +227,7 @@ protected int getTaskGroupIdForPartition(String partitionId) } @Override - protected boolean checkSourceMetaDataMatch(DataSourceMetadata metadata) + protected boolean checkSourceMetadataMatch(DataSourceMetadata metadata) { return metadata instanceof KinesisDataSourceMetadata; } @@ -271,16 +271,6 @@ protected SeekableStreamDataSourceMetadata createDataSourceMetaD ); } - @Override - protected Map createNewTaskEndPartitions(Set startPartitions) - { - Map endPartitions = new HashMap<>(); - for (String partition : startPartitions) { - endPartitions.put(partition, SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER); - } - return endPartitions; - } - @Override protected SequenceNumber makeSequenceNumber( String seq, boolean useExclusive, boolean isExclusive diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index 8b09a71d5c76..10e8f125e19f 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -233,25 +233,6 @@ public void testStartAndEndPartitionSetMatch() throws Exception mapper.readValue(jsonStr, IOConfig.class); } - /* - @Test - public void testEndOffsetGreaterThanStart() throws Exception - { - String jsonStr = "{\n" - + " \"type\": \"kinesis\",\n" - + " \"baseSequenceName\": \"my-sequence-name\",\n" - + " \"endpoint\": \"kinesis.us-east-1.amazonaws.com\",\n" - + " \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n" - + " \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"9\"}}\n" - + "}"; - - exception.expect(JsonMappingException.class); - exception.expectCause(CoreMatchers.isA(IllegalArgumentException.class)); - exception.expectMessage(CoreMatchers.containsString("end offset must be >= start offset")); - mapper.readValue(jsonStr, IOConfig.class); - } - */ - @Test public void testEndPointRequired() throws Exception { diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 3b6b858821ca..2ef4b469e331 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -72,6 +72,7 @@ import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionToolbox; +import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.stats.RowIngestionMeters; @@ -1944,8 +1945,10 @@ public boolean checkPointDataSourceMetadata( ); final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( taskStorage, - taskActionToolbox + taskActionToolbox, + new TaskAuditLogConfig(false) ); + final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier() { @Override @@ -2082,7 +2085,8 @@ private List readSegmentColumn(final String column, final SegmentDescrip ); IndexIO indexIO = new org.apache.druid.indexing.common.TestUtils().getTestIndexIO(); QueryableIndex index = indexIO.loadIndex(outputLocation); - DictionaryEncodedColumn theColumn = index.getColumn(column).getDictionaryEncoding(); + DictionaryEncodedColumn theColumn = (DictionaryEncodedColumn) index.getColumnHolder(column) + .getColumn(); List values = Lists.newArrayList(); for (int i = 0; i < theColumn.length(); i++) { int id = theColumn.getSingleValueRow(i); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 560f80fac6ff..e18d6d10a9cf 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -211,10 +211,6 @@ public KinesisSupervisorTest() @BeforeClass public static void setupClass() { - /* - * Need to disable CBOR protocol, see: - * https://github.com/mhart/kinesalite/blob/master/README.md#cbor-protocol-issues-with-the-java-sdk - */ cloud.localstack.TestUtils.setEnv("AWS_CBOR_DISABLE", "1"); /* Disable SSL certificate checks for local testing */ if (Localstack.useSSL()) { @@ -2622,368 +2618,6 @@ public void testNoDataIngestionTasks() throws Exception verifyAll(); } - /* - @Test(timeout = 60_000L) - public void testCheckpointForInactiveTaskGroup() - throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException - { - supervisor = getSupervisor(2, 1, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); - //not adding any events - final Task id1 = createKinesisIndexTask( - "id1", - DATASOURCE, - 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - - getSequenceNumber(res, shardId1, 0), - - shardId0, - - getSequenceNumber(res, shardId0, 0) - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - shardId0, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER - )), - null, - null - ); - - final Task id2 = createKinesisIndexTask( - "id2", - DATASOURCE, - 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - - getSequenceNumber(res, shardId1, 3), - - shardId0, - - getSequenceNumber(res, shardId0, 1) - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - shardId0, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER - )), - null, - null - ); - - final Task id3 = createKinesisIndexTask( - "id3", - DATASOURCE, - 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - - getSequenceNumber(res, shardId1, 3), - - shardId0, - - getSequenceNumber(res, shardId0, 1) - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - shardId0, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER - )), - null, - null - ); - - final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); - final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); - Collection workItems = new ArrayList<>(); - workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); - workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect( - indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KinesisDataSourceMetadata( - null) - ).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - - final DateTime startTime = DateTimes.nowUtc(); - EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); - EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); - - final TreeMap> checkpoints = new TreeMap<>(); - checkpoints.put(0, ImmutableMap.of( - shardId1, - - getSequenceNumber(res, shardId1, 3), - - shardId0, - - getSequenceNumber(res, shardId0, 1) - )); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); - - taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); - replayAll(); - - supervisor.start(); - supervisor.runInternal(); - - final Map fakeCheckpoints = Collections.emptyMap(); - supervisor.moveTaskGroupToPendingCompletion(0); - supervisor.checkpoint( - 0, - ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), - new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, checkpoints.get(0))), - new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, fakeCheckpoints)) - ); - - while (supervisor.getNoticesQueueSize() > 0) { - Thread.sleep(100); - } - - verifyAll(); - - Assert.assertNull(serviceEmitter.getStackTrace(), serviceEmitter.getStackTrace()); - Assert.assertNull(serviceEmitter.getExceptionMessage(), 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 = createKinesisIndexTask( - "id1", - DATASOURCE, - 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - - getSequenceNumber(res, shardId1, 0), - - shardId0, - - getSequenceNumber(res, shardId0, 0) - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - shardId0, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER - )), - null, - null - ); - - final Task id2 = createKinesisIndexTask( - "id2", - DATASOURCE, - 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - - getSequenceNumber(res, shardId1, 3), - - shardId0, - - getSequenceNumber(res, shardId0, 1) - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - shardId0, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER - )), - null, - null - ); - - final Task id3 = createKinesisIndexTask( - "id3", - DATASOURCE, - 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - - getSequenceNumber(res, shardId1, 3), - - shardId0, - - getSequenceNumber(res, shardId0, 1) - )), - new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - shardId0, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER - )), - null, - null - ); - - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect( - indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KinesisDataSourceMetadata( - null) - ).anyTimes(); - - replayAll(); - - supervisor.start(); - - supervisor.checkpoint( - 0, - ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), - new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, Collections.emptyMap())), - new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, Collections.emptyMap())) - ); - - while (supervisor.getNoticesQueueSize() > 0) { - Thread.sleep(100); - } - - verifyAll(); - - while (serviceEmitter.getStackTrace() == null) { - Thread.sleep(100); - } - - Assert.assertTrue(serviceEmitter.getStackTrace() - .startsWith("org.apache.druid.java.util.common.ISE: WTH?! cannot find")); - Assert.assertEquals( - "WTH?! cannot find taskGroup [0] among all taskGroups [{}]", - serviceEmitter.getExceptionMessage() - ); - Assert.assertEquals(ISE.class, serviceEmitter.getExceptionClass()); - } - - @Test(timeout = 60_000L) - public void testCheckpointWithNullTaskGroupId() - throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException - { - supervisor = getSupervisor(1, 3, true, "PT1S", null, null, false); - //not adding any events - final Task id1 = createKinesisIndexTask( - "id1", - DATASOURCE, - 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of(0, 0L)), - new SeekableStreamPartitions<>(stream, ImmutableMap.of(0, Long.MAX_VALUE)), - null, - null - ); - - final Task id2 = createKinesisIndexTask( - "id2", - DATASOURCE, - 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of(0, 0L)), - new SeekableStreamPartitions<>(stream, ImmutableMap.of(0, Long.MAX_VALUE)), - null, - null - ); - - final Task id3 = createKinesisIndexTask( - "id3", - DATASOURCE, - 0, - new SeekableStreamPartitions<>(stream, ImmutableMap.of(0, 0L)), - new SeekableStreamPartitions<>(stream, ImmutableMap.of(0, Long.MAX_VALUE)), - null, - null - ); - - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect( - indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KinesisDataSourceMetadata( - null) - ).anyTimes(); - taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.READING)) - .anyTimes(); - final TreeMap> checkpoints = new TreeMap<>(); - checkpoints.put(0, ImmutableMap.of(0, 0L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.anyString(), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(3); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); - EasyMock.expect(taskClient.pauseAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L))) - .anyTimes(); - EasyMock.expect(taskClient.setEndOffsetsAsync(EasyMock.anyString(), EasyMock.eq(ImmutableMap.of(0, 10L)), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(true)) - .anyTimes(); - - replayAll(); - - supervisor.start(); - - supervisor.runInternal(); - - final TreeMap> newCheckpoints = new TreeMap<>(); - newCheckpoints.put(0, ImmutableMap.of(0, 10L)); - supervisor.checkpoint( - null, - ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), - new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, checkpoints.get(0))), - new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, newCheckpoints.get(0))) - ); - - while (supervisor.getNoticesQueueSize() > 0) { - Thread.sleep(100); - } - - verifyAll(); - } - */ - @Test public void testSuspendedNoRunningTasks() throws Exception { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 30d8cdb0460f..d2ffd9780ab3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -45,12 +45,13 @@ import java.nio.ByteBuffer; import java.util.Map; import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; public abstract class SeekableStreamIndexTask extends AbstractTask implements ChatHandler { - private static final Random RANDOM = new Random(); + private static final Random RANDOM = ThreadLocalRandom.current(); protected final DataSchema dataSchema; protected final InputRowParser parser; protected final SeekableStreamTuningConfig tuningConfig; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index 51e7e70390b6..caf58f481577 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -61,11 +61,11 @@ public SeekableStreamPartitions( for (Map.Entry entry : map.entrySet()) { Preconditions.checkArgument( entry.getValue() != null, - String.format( - "partition stream[%s] sequence/offset number[%s] invalid", - entry.getKey(), - entry.getValue() - ) + "partition stream[" + + entry.getKey().toString() + + "] sequence/offset number[%" + + entry.getValue().toString() + + "] invalid" ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java index 85dee67f72f2..ef8a1ca25a0b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java @@ -21,6 +21,13 @@ import java.util.List; +/** + * Represents a generic record with a T1 (partition id) and T2 (sequence number) and data + * from a Kafka/Kinesis stream + * + * @param partition id + * @param sequence number + */ public class Record { public static final String END_OF_SHARD_MARKER = "EOS"; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java index 580441ce9aa6..7d3e551386a3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java @@ -23,8 +23,8 @@ import java.util.Objects; /** - * Wrapper class for Kafka and Kinesis stream sequence numbers. Mainly used to do - * comparison and indicate whether the sequence number should be excluded + * Represents a Kafka/Kinesis stream sequence number. Mainly used to do + * comparison and indicate whether the sequence number is exclusive * * @param type of sequence number */ diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java index d424a37c45f0..802e7ae5f6c1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java @@ -20,7 +20,9 @@ package org.apache.druid.indexing.seekablestream.common; /** - * wrapper class for a Kinesis/Kafka partition with stream name and partitionId + * Reprents a Kinesis/Kafka partition with stream name and partitionId, + * mostly used by {@link RecordSupplier} and + * {@link org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor} * * @param partition id type */ diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 9a64183d0e39..2705eca5d570 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -101,6 +101,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; @@ -130,7 +131,7 @@ public abstract class SeekableStreamSupervisor { public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED"; private static final EmittingLogger log = new EmittingLogger(SeekableStreamSupervisor.class); - private static final Random RANDOM = new Random(); + private static final Random RANDOM = ThreadLocalRandom.current(); private static final long MAX_RUN_FREQUENCY_MILLIS = 1000; private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120; private static final int MAX_INITIALIZATION_RETRIES = 20; @@ -404,7 +405,7 @@ public void handle() throws ExecutionException, InterruptedException } } - protected boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) + boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) { if (taskGroup == null) { // taskGroup might be in pendingCompletionTaskGroups or partitionGroups @@ -476,6 +477,7 @@ protected boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGrou private final Object stopLock = new Object(); private final Object stateChangeLock = new Object(); private final Object recordSupplierLock = new Object(); + private final T2 END_OF_PARTITION; private final T2 NOT_SET; private final boolean useExclusiveStartingSequence; private boolean listenerRegistered = false; @@ -498,6 +500,7 @@ public SeekableStreamSupervisor( final SeekableStreamSupervisorSpec spec, final RowIngestionMetersFactory rowIngestionMetersFactory, final T2 NOT_SET, + final T2 END_OF_PARTITION, final boolean useExclusiveStartingSequence ) { @@ -508,6 +511,7 @@ public SeekableStreamSupervisor( this.spec = spec; this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.NOT_SET = NOT_SET; + this.END_OF_PARTITION = END_OF_PARTITION; this.useExclusiveStartingSequence = useExclusiveStartingSequence; this.dataSource = spec.getDataSchema().getDataSource(); @@ -760,7 +764,8 @@ protected void tryInit() TimeUnit.MILLISECONDS ); - // not yet implemented in kinesis, will remove once implemented + // different in kafka and kinesis as emitLag and checkpointing + // are not yet implemented in Kinesis scheduleReporting(reportingExec); started = true; @@ -1031,7 +1036,7 @@ protected void gracefulShutdownInternal() throws ExecutionException, Interrupted if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) { killTask(entry.getKey()); } else { - entry.getValue().startTime = new DateTime(0); + entry.getValue().startTime = DateTimes.EPOCH; } } } @@ -1051,7 +1056,7 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) partitionGroups.clear(); } else { - if (!checkSourceMetaDataMatch(dataSourceMetadata)) { + if (!checkSourceMetadataMatch(dataSourceMetadata)) { throw new IAE( "Datasource metadata instance does not match required, found instance of [%s]", dataSourceMetadata.getClass() @@ -1064,7 +1069,7 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) if (resetMetadata.getSeekableStreamPartitions().getStream().equals(ioConfig.getId())) { // metadata can be null final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); - if (metadata != null && !checkSourceMetaDataMatch(metadata)) { + if (metadata != null && !checkSourceMetadataMatch(metadata)) { throw new IAE( "Datasource metadata instance does not match required, found instance of [%s]", metadata.getClass() @@ -1421,7 +1426,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) final DataSourceMetadata rawDataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); - if (rawDataSourceMetadata != null && !checkSourceMetaDataMatch(rawDataSourceMetadata)) { + if (rawDataSourceMetadata != null && !checkSourceMetadataMatch(rawDataSourceMetadata)) { throw new IAE( "Datasource metadata instance does not match required, found instance of [%s]", rawDataSourceMetadata.getClass() @@ -2339,7 +2344,7 @@ private Map getOffsetsFromMetadataStorage() { final DataSourceMetadata dataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); if (dataSourceMetadata instanceof SeekableStreamDataSourceMetadata - && checkSourceMetaDataMatch(dataSourceMetadata)) { + && checkSourceMetadataMatch(dataSourceMetadata)) { @SuppressWarnings("unchecked") SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata).getSeekableStreamPartitions(); if (partitions != null) { @@ -2377,7 +2382,10 @@ private void createTasksForGroup(int groupId, int replicas) throws JsonProcessin { TaskGroup group = taskGroups.get(groupId); Map startPartitions = group.startingSequences; - Map endPartitions = createNewTaskEndPartitions(startPartitions.keySet()); + Map endPartitions = new HashMap<>(); + for (T1 partition : startPartitions.keySet()) { + endPartitions.put(partition, END_OF_PARTITION); + } Set exclusiveStartSequenceNumberPartitions = taskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; DateTime minimumMessageTime = taskGroups.get(groupId).minimumMessageTime.orNull(); @@ -2546,6 +2554,11 @@ public abstract void checkpoint( DataSourceMetadata currentCheckPoint ); + /** + * creates a specific task IOConfig instance for Kafka/Kinesis* + * + * @return specific instance of Kafka/Kinesis IOConfig + */ protected abstract SeekableStreamIOConfig createIoConfig( int groupId, Map startPartitions, @@ -2557,6 +2570,14 @@ protected abstract SeekableStreamIOConfig createIoConfig( SeekableStreamSupervisorIOConfig ioConfig ); + /** + * creates a list of specific kafka/kinesis index tasks using + * the given replicas count + * + * @return list of specific kafka/kinesis index taksks + * + * @throws JsonProcessingException + */ protected abstract List> createIndexTasks( int replicas, String baseSequenceName, @@ -2567,32 +2588,95 @@ protected abstract List> createIndexTasks( RowIngestionMetersFactory rowIngestionMetersFactory ) throws JsonProcessingException; + /** + * calculates the taskgroup id that the given partition belongs to. + * different between Kafka/Kinesis since Kinesis uses String as partition id + * + * @param partition paritition id + * + * @return taskgroup id + */ protected abstract int getTaskGroupIdForPartition(T1 partition); - protected abstract boolean checkSourceMetaDataMatch(DataSourceMetadata metadata); + /** + * checks if the passed in DataSourceMetadata is a specific instance + * of [kafka/kinesis]DataSourceMetadata + * + * @param metadata datasource metadata + * + * @return true if isInstance else false + */ + protected abstract boolean checkSourceMetadataMatch(DataSourceMetadata metadata); + /** + * checks if the passed in Task is a specific instance of + * [Kafka/Kinesis]IndexTask + * + * @param task task + * + * @return true if isInstance else false + */ protected abstract boolean checkTaskInstance(Task task); + /** + * creates a specific instance of kafka/kinesis datasource metadata + * + * @param stream stream name + * @param map partitionId -> sequence + * + * @return specific instance of datasource metadata + */ protected abstract SeekableStreamDataSourceMetadata createDataSourceMetaData( String stream, Map map ); - protected abstract Map createNewTaskEndPartitions(Set startPartitions); - + /** + * wraps the passed in T2 sequence number into a {@link SequenceNumber} object + * to facilitate comparison and accomodate exclusive starting sequennce in kinesis + * + * @return specific instance of [Kafka/Kinesis]SequenceNumber + */ protected abstract SequenceNumber makeSequenceNumber(T2 seq, boolean useExclusive, boolean isExclusive); + /** + * schedules periodic emitLag() reporting for Kafka, not yet implemented in Kinesis, + * but will be in the future + */ protected abstract void scheduleReporting(ScheduledExecutorService reportingExec); + /** + * calculate lag per partition for kafka, kinesis implementation returns an empty + * map + * + * @return map of partition id -> lag + */ protected abstract Map getLagPerPartition(Map currentOffsets); + /** + * returns an instance of a specific Kinesis/Kafka recordSupplier + * + * @return specific instance of Kafka/Kinesis RecordSupplier + */ protected abstract RecordSupplier setupRecordSupplier(); + /** + * creates a specific instance of Kafka/Kinesis Supervisor Report Payload + * + * @return specific instance of Kafka/Kinesis Supervisor Report Payload + */ protected abstract SeekableStreamSupervisorReportPayload createReportPayload( int numPartitions, boolean includeOffsets ); + /** + * checks if sequence from metadata storage is still valid + * + * @return true if still valid else false + * + * @throws TimeoutException + */ protected abstract boolean checkSequenceAvailability(@NotNull T1 partition, @NotNull T2 sequenceFromMetadata) throws TimeoutException; } From ed36a68c80f0e97944da70637fd8669fa875e87e Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 9 Oct 2018 13:52:24 -0700 Subject: [PATCH 22/87] fix various things from pr comment --- .../indexing/kafka/KafkaRecordSupplier.java | 34 +-- .../indexing/kafka/KafkaSequenceNumber.java | 6 +- .../kafka/supervisor/KafkaSupervisor.java | 4 +- .../indexing/kinesis/KinesisIndexTask.java | 30 +- .../kinesis/KinesisRecordSupplier.java | 49 ++-- .../kinesis/KinesisSequenceNumber.java | 6 +- .../kinesis/supervisor/KinesisSupervisor.java | 10 +- .../indexing/kinesis/KinesisIOConfigTest.java | 8 +- .../kinesis/KinesisIndexTaskTest.java | 10 +- .../supervisor/KinesisSupervisorTest.java | 14 +- .../SeekableStreamDataSourceMetadata.java | 34 +-- .../SeekableStreamIOConfig.java | 18 +- .../SeekableStreamIndexTask.java | 6 +- .../SeekableStreamIndexTaskClient.java | 22 +- .../SeekableStreamIndexTaskRunner.java | 18 +- .../SeekableStreamPartitions.java | 62 ++-- ...d.java => OrderedPartitionableRecord.java} | 28 +- ...Number.java => OrderedSequenceNumber.java} | 6 +- .../seekablestream/common/RecordSupplier.java | 37 ++- .../common/StreamPartition.java | 28 +- .../supervisor/SeekableStreamSupervisor.java | 266 +++++++++--------- ...SeekableStreamSupervisorReportPayload.java | 14 +- .../supervisor/TaskReportData.java | 22 +- 23 files changed, 363 insertions(+), 369 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/{Record.java => OrderedPartitionableRecord.java} (65%) rename indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/{SequenceNumber.java => OrderedSequenceNumber.java} (88%) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 72599d0779e5..4eb321c977fa 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -19,9 +19,10 @@ package org.apache.druid.indexing.kafka; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; -import org.apache.druid.indexing.seekablestream.common.Record; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; @@ -34,7 +35,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -55,7 +55,7 @@ public class KafkaRecordSupplier implements RecordSupplier private final KafkaConsumer consumer; private final KafkaSupervisorIOConfig ioConfig; private boolean closed; - private final BlockingQueue> records; + private final BlockingQueue> records; public KafkaRecordSupplier( @@ -73,14 +73,14 @@ public void assign(Set> streamPartitions) { consumer.assign(streamPartitions .stream() - .map(x -> new TopicPartition(x.getStreamName(), x.getPartitionId())) + .map(x -> new TopicPartition(x.getStream(), x.getPartitionId())) .collect(Collectors.toSet())); } @Override public void seek(StreamPartition partition, Long sequenceNumber) { - consumer.seek(new TopicPartition(partition.getStreamName(), partition.getPartitionId()), sequenceNumber); + consumer.seek(new TopicPartition(partition.getStream(), partition.getPartitionId()), sequenceNumber); } @Override @@ -94,7 +94,7 @@ public void seekToEarliest(Set> partitions) { consumer.seekToBeginning(partitions .stream() - .map(e -> new TopicPartition(e.getStreamName(), e.getPartitionId())) + .map(e -> new TopicPartition(e.getStream(), e.getPartitionId())) .collect(Collectors.toList())); } @@ -103,7 +103,7 @@ public void seekToLatest(Set> partitions) { consumer.seekToEnd(partitions .stream() - .map(e -> new TopicPartition(e.getStreamName(), e.getPartitionId())) + .map(e -> new TopicPartition(e.getStream(), e.getPartitionId())) .collect(Collectors.toList())); } @@ -118,16 +118,16 @@ public Set> getAssignment() } @Override - public Record poll(long timeout) + public OrderedPartitionableRecord poll(long timeout) { if (records.isEmpty()) { ConsumerRecords polledRecords = consumer.poll(timeout); for (ConsumerRecord record : polledRecords) { - records.offer(new Record<>( + records.offer(new OrderedPartitionableRecord<>( record.topic(), record.partition(), record.offset(), - Arrays.asList(record.value()) + ImmutableList.of(record.value()) )); } } @@ -145,32 +145,32 @@ public Record poll(long timeout) public Long getLatestSequenceNumber(StreamPartition partition) { seekToLatest(Collections.singleton(partition)); - return consumer.position(new TopicPartition(partition.getStreamName(), partition.getPartitionId())); + return consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); } @Override public Long getEarliestSequenceNumber(StreamPartition partition) { seekToEarliest(Collections.singleton(partition)); - return consumer.position(new TopicPartition(partition.getStreamName(), partition.getPartitionId())); + return consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); } @Override public Long position(StreamPartition partition) { - return consumer.position(new TopicPartition(partition.getStreamName(), partition.getPartitionId())); + return consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); } @Override - public Set getPartitionIds(String streamName) + public Set getPartitionIds(String stream) { final Map> topics = consumer.listTopics(); - if (!topics.containsKey(streamName)) { - throw new ISE("Topic [%s] is not found in KafkaConsumer's list of topics", streamName); + if (!topics.containsKey(stream)) { + throw new ISE("Topic [%s] is not found in KafkaConsumer's list of topics", stream); } return topics == null ? ImmutableSet.of() - : topics.get(streamName).stream().map(PartitionInfo::partition).collect(Collectors.toSet()); + : topics.get(stream).stream().map(PartitionInfo::partition).collect(Collectors.toSet()); } @Override diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java index 2f411bfe99ab..de211f0a6fdd 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java @@ -19,11 +19,11 @@ package org.apache.druid.indexing.kafka; -import org.apache.druid.indexing.seekablestream.common.SequenceNumber; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import javax.validation.constraints.NotNull; -public class KafkaSequenceNumber extends SequenceNumber +public class KafkaSequenceNumber extends OrderedSequenceNumber { private KafkaSequenceNumber(Long sequenceNumber, boolean useExclusive, boolean isExclusive) { @@ -37,7 +37,7 @@ public static KafkaSequenceNumber of(Long sequenceNumber) @Override public int compareTo( - @NotNull SequenceNumber o + @NotNull OrderedSequenceNumber o ) { return this.get().compareTo(o.get()); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index cc8a8a7b1792..4770577943b1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -45,7 +45,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; -import org.apache.druid.indexing.seekablestream.common.SequenceNumber; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; @@ -324,7 +324,7 @@ protected KafkaDataSourceMetadata createDataSourceMetaData( } @Override - protected SequenceNumber makeSequenceNumber( + protected OrderedSequenceNumber makeSequenceNumber( Long seq, boolean useExclusive, boolean isExclusive ) { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index be0d038d3ddd..1aaeef919c0b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -56,7 +56,7 @@ import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; -import org.apache.druid.indexing.seekablestream.common.Record; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; @@ -268,7 +268,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception appenderator = appenderator0; - final String topic = ioConfig.getStartPartitions().getStream(); + final String topic = ioConfig.getStartPartitions().getName(); // Start up, set up initial offsets. final Object restoredMetadata = driver.startJob(); @@ -291,11 +291,11 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception lastOffsets.putAll(restoredNextPartitions.getMap()); // Sanity checks. - if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) { + if (!restoredNextPartitions.getName().equals(ioConfig.getStartPartitions().getName())) { throw new ISE( "WTF?! Restored stream[%s] but expected stream[%s]", - restoredNextPartitions.getStream(), - ioConfig.getStartPartitions().getStream() + restoredNextPartitions.getName(), + ioConfig.getStartPartitions().getName() ); } @@ -311,7 +311,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception // Filter out partitions with END_OF_SHARD markers since these partitions have already been fully read. This // should have been done by the supervisor already so this is defensive. int numPreFilterPartitions = lastOffsets.size(); - if (lastOffsets.entrySet().removeIf(x -> Record.END_OF_SHARD_MARKER.equals(x.getValue()))) { + if (lastOffsets.entrySet().removeIf(x -> OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(x.getValue()))) { log.info( "Removed [%d] partitions from assignment which have already been closed", numPreFilterPartitions - lastOffsets.size() @@ -335,7 +335,7 @@ public Object getMetadata() { return ImmutableMap.of( METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>( - ioConfig.getStartPartitions().getStream(), + ioConfig.getStartPartitions().getName(), snapshot ) ); @@ -375,7 +375,7 @@ public void run() break; } - Record record = recordSupplier.poll(POLL_TIMEOUT); + OrderedPartitionableRecord record = recordSupplier.poll(POLL_TIMEOUT); if (record == null) { continue; @@ -415,13 +415,13 @@ public void run() if (log.isTraceEnabled()) { log.trace( "Got topic[%s] partition[%s] offset[%s].", - record.getStreamName(), + record.getStream(), record.getPartitionId(), record.getSequenceNumber() ); } - if (Record.END_OF_SHARD_MARKER.equals(record.getSequenceNumber())) { + if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(record.getSequenceNumber())) { lastOffsets.put(record.getPartitionId(), record.getSequenceNumber()); } else if (SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffsets.get(record.getPartitionId())) @@ -496,10 +496,10 @@ public void run() } if ((lastOffsets.get(record.getPartitionId()).equals(endOffsets.get(record.getPartitionId())) - || Record.END_OF_SHARD_MARKER.equals(lastOffsets.get(record.getPartitionId()))) + || OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(lastOffsets.get(record.getPartitionId()))) && assignment.remove(record.getPartitionId())) { - log.info("Finished reading stream[%s], partition[%s].", record.getStreamName(), record.getPartitionId()); + log.info("Finished reading stream[%s], partition[%s].", record.getStream(), record.getPartitionId()); assignPartitions(recordSupplier, topic, assignment); stillReading = !assignment.isEmpty(); } @@ -707,7 +707,7 @@ private Set assignPartitions(RecordSupplier recordSuppli final Set assignment = Sets.newHashSet(); for (Map.Entry entry : lastOffsets.entrySet()) { final String endOffset = endOffsets.get(entry.getKey()); - if (Record.END_OF_SHARD_MARKER.equals(endOffset) + if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(endOffset) || SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) || KinesisSequenceNumber.of(entry.getValue()).compareTo(KinesisSequenceNumber.of(endOffset)) < 0) { assignment.add(entry.getKey()); @@ -876,7 +876,7 @@ private boolean withinMinMaxRecordTime(final InputRow row) return !beforeMinimumMessageTime && !afterMaximumMessageTime; } - private void handleParseException(ParseException pe, Record record) + private void handleParseException(ParseException pe, OrderedPartitionableRecord record) { if (pe.isFromPartiallyValidRow()) { rowIngestionMeters.incrementProcessedWithError(); @@ -944,7 +944,7 @@ private void sendResetRequestAndWait( getDataSource(), new KinesisDataSourceMetadata( new SeekableStreamPartitions<>( - ioConfig.getStartPartitions().getStream(), + ioConfig.getStartPartitions().getName(), partitionOffsetMap ) ) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 48b6cb45815b..671529dac10b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -31,6 +31,7 @@ import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; import com.amazonaws.services.kinesis.model.GetRecordsResult; import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; +import com.amazonaws.services.kinesis.model.Record; import com.amazonaws.services.kinesis.model.ResourceNotFoundException; import com.amazonaws.services.kinesis.model.ShardIteratorType; import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder; @@ -38,7 +39,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.common.aws.AWSCredentialsUtils; import org.apache.druid.indexing.kinesis.aws.ConstructibleAWSCredentialsConfig; -import org.apache.druid.indexing.seekablestream.common.Record; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; @@ -101,7 +102,7 @@ public void start() log.info( "Starting scheduled fetch runnable for stream[%s] partition[%s]", - streamPartition.getStreamName(), + streamPartition.getStream(), streamPartition.getPartitionId() ); @@ -116,7 +117,7 @@ public void stop() { log.info( "Stopping scheduled fetch runnable for stream[%s] partition[%s]", - streamPartition.getStreamName(), + streamPartition.getStream(), streamPartition.getPartitionId() ); @@ -140,12 +141,15 @@ private Runnable getRecordRunnable() log.info("shardIterator[%s] has been closed and has no more records", streamPartition.getPartitionId()); // add an end-of-shard marker so caller knows this shard is closed - Record endOfShardRecord = new Record<>( - streamPartition.getStreamName(), streamPartition.getPartitionId(), Record.END_OF_SHARD_MARKER, null + OrderedPartitionableRecord endOfShardRecord = new OrderedPartitionableRecord<>( + streamPartition.getStream(), + streamPartition.getPartitionId(), + OrderedPartitionableRecord.END_OF_SHARD_MARKER, + null ); if (!records.offer(endOfShardRecord, recordBufferOfferTimeout, TimeUnit.MILLISECONDS)) { - log.warn("Record buffer full, retrying in [%,dms]", recordBufferFullWait); + log.warn("OrderedPartitionableRecord buffer full, retrying in [%,dms]", recordBufferFullWait); rescheduleRunnable(recordBufferFullWait); } @@ -155,7 +159,7 @@ private Runnable getRecordRunnable() GetRecordsResult recordsResult = kinesisProxy.get(shardIterator, recordsPerFetch); // list will come back empty if there are no records - for (com.amazonaws.services.kinesis.model.Record kinesisRecord : recordsResult.getRecords()) { + for (Record kinesisRecord : recordsResult.getRecords()) { final List data; if (deaggregate) { @@ -169,8 +173,8 @@ private Runnable getRecordRunnable() data = Collections.singletonList(toByteArray(kinesisRecord.getData())); } - final Record record = new Record<>( - streamPartition.getStreamName(), + final OrderedPartitionableRecord record = new OrderedPartitionableRecord<>( + streamPartition.getStream(), streamPartition.getPartitionId(), kinesisRecord.getSequenceNumber(), data @@ -180,7 +184,7 @@ private Runnable getRecordRunnable() if (log.isTraceEnabled()) { log.trace( "Stream[%s] / partition[%s] / sequenceNum[%s] / bufferRemainingCapacity[%d]: %s", - record.getStreamName(), + record.getStream(), record.getPartitionId(), record.getSequenceNumber(), records.remainingCapacity(), @@ -191,7 +195,10 @@ private Runnable getRecordRunnable() // If the buffer was full and we weren't able to add the message, grab a new stream iterator starting // from this message and back off for a bit to let the buffer drain before retrying. if (!records.offer(record, recordBufferOfferTimeout, TimeUnit.MILLISECONDS)) { - log.warn("Record buffer full, storing iterator and retrying in [%,dms]", recordBufferFullWait); + log.warn( + "OrderedPartitionableRecord buffer full, storing iterator and retrying in [%,dms]", + recordBufferFullWait + ); shardIterator = kinesisProxy.getIterator( record.getPartitionId(), ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), record.getSequenceNumber() @@ -240,7 +247,7 @@ private void rescheduleRunnable(long delayMillis) private final Map kinesisProxies = new ConcurrentHashMap<>(); private final Map, PartitionResource> partitionResources = new ConcurrentHashMap<>(); - private final BlockingQueue> records; + private final BlockingQueue> records; private volatile boolean checkPartitionsStarted = false; private volatile boolean closed = false; @@ -318,7 +325,7 @@ public void assign(Set> collection) collection.forEach( streamPartition -> partitionResources.putIfAbsent( streamPartition, - new PartitionResource(streamPartition, getKinesisProxy(streamPartition.getStreamName()), scheduledExec) + new PartitionResource(streamPartition, getKinesisProxy(streamPartition.getStream()), scheduledExec) ) ); @@ -369,7 +376,7 @@ public Collection> getAssignment() @Nullable @Override - public Record poll(long timeout) + public OrderedPartitionableRecord poll(long timeout) { checkIfClosed(); if (checkPartitionsStarted) { @@ -379,13 +386,13 @@ public Record poll(long timeout) try { while (true) { - Record record = records.poll(timeout, TimeUnit.MILLISECONDS); + OrderedPartitionableRecord record = records.poll(timeout, TimeUnit.MILLISECONDS); if (record == null || partitionResources.containsKey(record.getStreamPartition())) { return record; } else if (log.isTraceEnabled()) { log.trace( "Skipping stream[%s] / partition[%s] / sequenceNum[%s] because it is not in current assignment", - record.getStreamName(), + record.getStream(), record.getPartitionId(), record.getSequenceNumber() ); @@ -420,10 +427,10 @@ public String position(StreamPartition partition) } @Override - public Set getPartitionIds(String streamName) + public Set getPartitionIds(String stream) { checkIfClosed(); - Set shardList = getKinesisProxy(streamName).getAllShardIds(); + Set shardList = getKinesisProxy(stream).getAllShardIds(); return shardList != null ? shardList : ImmutableSet.of(); } @@ -472,7 +479,7 @@ private void seekInternal(StreamPartition partition, String sequenceNumb sequenceNumber != null ? sequenceNumber : iteratorEnum.toString() ); - resource.shardIterator = getKinesisProxy(partition.getStreamName()).getIterator( + resource.shardIterator = getKinesisProxy(partition.getStream()).getIterator( partition.getPartitionId(), iteratorEnum.toString(), sequenceNumber ); @@ -483,7 +490,7 @@ private String getSequenceNumberInternal(StreamPartition partition, Shar throws TimeoutException { long timeoutMillis = System.currentTimeMillis() + fetchSequenceNumberTimeout; - IKinesisProxy kinesisProxy = getKinesisProxy(partition.getStreamName()); + IKinesisProxy kinesisProxy = getKinesisProxy(partition.getStream()); String shardIterator = null; try { @@ -517,7 +524,7 @@ private String getSequenceNumberInternal(StreamPartition partition, Shar } } - List records = recordsResult.getRecords(); + List records = recordsResult.getRecords(); if (!records.isEmpty()) { return records.get(0).getSequenceNumber(); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java index 65bb316bd5c1..3d27dae42c4d 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java @@ -20,13 +20,13 @@ package org.apache.druid.indexing.kinesis; -import org.apache.druid.indexing.seekablestream.common.SequenceNumber; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import javax.validation.constraints.NotNull; import java.math.BigInteger; import java.util.Objects; -public class KinesisSequenceNumber extends SequenceNumber +public class KinesisSequenceNumber extends OrderedSequenceNumber { private final BigInteger intSequence; @@ -63,7 +63,7 @@ public boolean equals(Object o) @Override - public int compareTo(@NotNull SequenceNumber o) + public int compareTo(@NotNull OrderedSequenceNumber o) { return this.intSequence.compareTo(new BigInteger(o.get())); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 7f42b76a4879..54fa02e7f1ca 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -44,7 +44,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; -import org.apache.druid.indexing.seekablestream.common.SequenceNumber; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; @@ -272,7 +272,7 @@ protected SeekableStreamDataSourceMetadata createDataSourceMetaD } @Override - protected SequenceNumber makeSequenceNumber( + protected OrderedSequenceNumber makeSequenceNumber( String seq, boolean useExclusive, boolean isExclusive ) { @@ -344,14 +344,14 @@ public void run() { try { final Map> topics = lagComputingConsumer.listTopics(); - final List partitionInfoList = topics.get(ioConfig.getStream()); + final List partitionInfoList = topics.get(ioConfig.getName()); lagComputingConsumer.assign( Lists.transform(partitionInfoList, new Function() { @Override public TopicPartition apply(PartitionInfo input) { - return new TopicPartition(ioConfig.getStream(), input.partition()); + return new TopicPartition(ioConfig.getName(), input.partition()); } }) ); @@ -401,7 +401,7 @@ public Long apply(Integer key, Long existingOffsetInMap) long lag = 0; for (PartitionInfo partitionInfo : partitionInfoList) { long diff; - final TopicPartition topicPartition = new TopicPartition(ioConfig.getStream(), partitionInfo.partition()); + final TopicPartition topicPartition = new TopicPartition(ioConfig.getName(), partitionInfo.partition()); lagComputingConsumer.seekToEnd(ImmutableList.of(topicPartition)); if (offsetsResponse.get(topicPartition.partition()) != null) { diff = lagComputingConsumer.position(topicPartition) - offsetsResponse.get(topicPartition.partition()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index 10e8f125e19f..3ace799c0ce9 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -67,12 +67,12 @@ public void testSerdeWithDefaults() throws Exception ); Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); - Assert.assertEquals("mystream", config.getStartPartitions().getStream()); + Assert.assertEquals("mystream", config.getStartPartitions().getName()); Assert.assertEquals( ImmutableMap.of("0", "1", "1", "10"), config.getStartPartitions().getMap() ); - Assert.assertEquals("mystream", config.getEndPartitions().getStream()); + Assert.assertEquals("mystream", config.getEndPartitions().getName()); Assert.assertEquals( ImmutableMap.of("0", "15", "1", "200"), config.getEndPartitions().getMap() @@ -124,12 +124,12 @@ public void testSerdeWithNonDefaults() throws Exception ); Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); - Assert.assertEquals("mystream", config.getStartPartitions().getStream()); + Assert.assertEquals("mystream", config.getStartPartitions().getName()); Assert.assertEquals( ImmutableMap.of("0", "1", "1", "10"), config.getStartPartitions().getMap() ); - Assert.assertEquals("mystream", config.getEndPartitions().getStream()); + Assert.assertEquals("mystream", config.getEndPartitions().getName()); Assert.assertEquals( ImmutableMap.of("0", "15", "1", "200"), config.getEndPartitions().getMap() diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 2ef4b469e331..847e90ad965f 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -87,7 +87,7 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; -import org.apache.druid.indexing.seekablestream.common.Record; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; import org.apache.druid.indexing.test.TestDataSegmentKiller; @@ -502,7 +502,7 @@ public void testRunBeforeDataInserted() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - Record.END_OF_SHARD_MARKER + OrderedPartitionableRecord.END_OF_SHARD_MARKER )), true, null, @@ -553,7 +553,7 @@ public void testRunBeforeDataInserted() throws Exception stream, ImmutableMap.of( shardId0, - Record.END_OF_SHARD_MARKER + OrderedPartitionableRecord.END_OF_SHARD_MARKER ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1617,7 +1617,7 @@ public void testRestore() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - Record.END_OF_SHARD_MARKER + OrderedPartitionableRecord.END_OF_SHARD_MARKER )), true, null, @@ -1663,7 +1663,7 @@ public void testRestore() throws Exception new KinesisDataSourceMetadata( new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - Record.END_OF_SHARD_MARKER + OrderedPartitionableRecord.END_OF_SHARD_MARKER ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index e18d6d10a9cf..dbc61a776aaa 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -306,7 +306,7 @@ public void testNoInitialState() throws Exception Assert.assertFalse("minimumMessageTime", taskConfig.getMinimumMessageTime().isPresent()); Assert.assertFalse("maximumMessageTime", taskConfig.getMaximumMessageTime().isPresent()); - Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); + Assert.assertEquals(stream, taskConfig.getStartPartitions().getName()); Assert.assertEquals( getSequenceNumber(res, shardId1, 0), taskConfig.getStartPartitions().getMap().get(shardId1) @@ -316,7 +316,7 @@ public void testNoInitialState() throws Exception taskConfig.getStartPartitions().getMap().get(shardId0) ); - Assert.assertEquals(stream, taskConfig.getEndPartitions().getStream()); + Assert.assertEquals(stream, taskConfig.getEndPartitions().getName()); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, taskConfig.getEndPartitions().getMap().get(shardId1) @@ -1321,7 +1321,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); - Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); + Assert.assertEquals(stream, taskConfig.getStartPartitions().getName()); Assert.assertEquals( getSequenceNumber(res, shardId1, 3), taskConfig.getStartPartitions().getMap().get(shardId1) @@ -1454,7 +1454,7 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); // check that the new task was created with starting offsets matching where the publishing task finished - Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); + Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getName()); Assert.assertEquals( getSequenceNumber(res, shardId1, 2), capturedTaskConfig.getStartPartitions().getMap().get(shardId1) @@ -1464,7 +1464,7 @@ public void testDiscoverExistingPublishingTask() throws Exception capturedTaskConfig.getStartPartitions().getMap().get(shardId0) ); - Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream()); + Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getName()); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, capturedTaskConfig.getEndPartitions().getMap().get(shardId1) @@ -1585,7 +1585,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); // check that the new task was created with starting offsets matching where the publishing task finished - Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); + Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getName()); Assert.assertEquals( getSequenceNumber(res, shardId1, 2), capturedTaskConfig.getStartPartitions().getMap().get(shardId1) @@ -1595,7 +1595,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() capturedTaskConfig.getStartPartitions().getMap().get(shardId0) ); - Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream()); + Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getName()); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, capturedTaskConfig.getEndPartitions().getMap().get(shardId1) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java index c4f2194db35a..76d3df5efa3a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java @@ -28,20 +28,20 @@ import java.util.Map; import java.util.Objects; -public abstract class SeekableStreamDataSourceMetadata implements DataSourceMetadata +public abstract class SeekableStreamDataSourceMetadata implements DataSourceMetadata { - private final SeekableStreamPartitions seekableStreamPartitions; + private final SeekableStreamPartitions seekableStreamPartitions; @JsonCreator public SeekableStreamDataSourceMetadata( - @JsonProperty("partitions") SeekableStreamPartitions seekableStreamPartitions + @JsonProperty("partitions") SeekableStreamPartitions seekableStreamPartitions ) { this.seekableStreamPartitions = seekableStreamPartitions; } @JsonProperty("partitions") - public SeekableStreamPartitions getSeekableStreamPartitions() + public SeekableStreamPartitions getSeekableStreamPartitions() { return seekableStreamPartitions; } @@ -75,21 +75,21 @@ public DataSourceMetadata plus(DataSourceMetadata other) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; + final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; - if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) { + if (that.getSeekableStreamPartitions().getName().equals(seekableStreamPartitions.getName())) { // Same topic, merge offsets. - final Map newMap = Maps.newHashMap(); + final Map newMap = Maps.newHashMap(); - for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { + for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { newMap.put(entry.getKey(), entry.getValue()); } - for (Map.Entry entry : that.getSeekableStreamPartitions().getMap().entrySet()) { + for (Map.Entry entry : that.getSeekableStreamPartitions().getMap().entrySet()) { newMap.put(entry.getKey(), entry.getValue()); } - return createConcretDataSourceMetaData(seekableStreamPartitions.getStream(), newMap); + return createConcretDataSourceMetaData(seekableStreamPartitions.getName(), newMap); } else { // Different topic, prefer "other". return other; @@ -109,19 +109,19 @@ public DataSourceMetadata minus(DataSourceMetadata other) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; + final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; - if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) { + if (that.getSeekableStreamPartitions().getName().equals(seekableStreamPartitions.getName())) { // Same stream, remove partitions present in "that" from "this" - final Map newMap = Maps.newHashMap(); + final Map newMap = Maps.newHashMap(); - for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { + for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { if (!that.getSeekableStreamPartitions().getMap().containsKey(entry.getKey())) { newMap.put(entry.getKey(), entry.getValue()); } } - return createConcretDataSourceMetaData(seekableStreamPartitions.getStream(), newMap); + return createConcretDataSourceMetaData(seekableStreamPartitions.getName(), newMap); } else { // Different stream, prefer "this". return this; @@ -155,8 +155,8 @@ public String toString() '}'; } - protected abstract SeekableStreamDataSourceMetadata createConcretDataSourceMetaData( + protected abstract SeekableStreamDataSourceMetadata createConcretDataSourceMetaData( String streamId, - Map newMap + Map newMap ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java index a4026acbe773..afdcf45d0f05 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java @@ -29,15 +29,15 @@ import javax.annotation.Nullable; import java.util.Set; -public abstract class SeekableStreamIOConfig implements IOConfig +public abstract class SeekableStreamIOConfig implements IOConfig { private static final boolean DEFAULT_USE_TRANSACTION = true; @Nullable private final Integer taskGroupId; private final String baseSequenceName; - private final SeekableStreamPartitions startPartitions; - private final SeekableStreamPartitions endPartitions; + private final SeekableStreamPartitions startPartitions; + private final SeekableStreamPartitions endPartitions; private final boolean useTransaction; private final Optional minimumMessageTime; private final Optional maximumMessageTime; @@ -46,8 +46,8 @@ public abstract class SeekableStreamIOConfig implements IOConfig public SeekableStreamIOConfig( @JsonProperty("taskGroupId") @Nullable Integer taskGroupId, // can be null for backward compabitility @JsonProperty("baseSequenceName") String baseSequenceName, - @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, - @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, + @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, + @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, @JsonProperty("useTransaction") Boolean useTransaction, @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, @JsonProperty("maximumMessageTime") DateTime maximumMessageTime @@ -62,7 +62,7 @@ public SeekableStreamIOConfig( this.maximumMessageTime = Optional.fromNullable(maximumMessageTime); Preconditions.checkArgument( - startPartitions.getStream().equals(endPartitions.getStream()), + startPartitions.getName().equals(endPartitions.getName()), "start topic/stream and end topic/stream must match" ); @@ -86,13 +86,13 @@ public String getBaseSequenceName() } @JsonProperty - public SeekableStreamPartitions getStartPartitions() + public SeekableStreamPartitions getStartPartitions() { return startPartitions; } @JsonProperty - public SeekableStreamPartitions getEndPartitions() + public SeekableStreamPartitions getEndPartitions() { return endPartitions; } @@ -116,7 +116,7 @@ public Optional getMinimumMessageTime() } @JsonProperty - public abstract Set getExclusiveStartSequenceNumberPartitions(); + public abstract Set getExclusiveStartSequenceNumberPartitions(); @Override public abstract String toString(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index d2ffd9780ab3..3d8cc65be427 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -48,14 +48,14 @@ import java.util.concurrent.ThreadLocalRandom; -public abstract class SeekableStreamIndexTask extends AbstractTask +public abstract class SeekableStreamIndexTask extends AbstractTask implements ChatHandler { private static final Random RANDOM = ThreadLocalRandom.current(); protected final DataSchema dataSchema; protected final InputRowParser parser; protected final SeekableStreamTuningConfig tuningConfig; - protected final SeekableStreamIOConfig ioConfig; + protected final SeekableStreamIOConfig ioConfig; protected final Optional chatHandlerProvider; protected final String type; protected CircularBuffer savedParseExceptions; @@ -134,7 +134,7 @@ public SeekableStreamTuningConfig getTuningConfig() } @JsonProperty("ioConfig") - public SeekableStreamIOConfig getIOConfig() + public SeekableStreamIOConfig getIOConfig() { return ioConfig; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index c058cb99df3c..c54405f2034a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -42,7 +42,7 @@ import java.util.Map; import java.util.TreeMap; -public abstract class SeekableStreamIndexTaskClient extends IndexTaskClient +public abstract class SeekableStreamIndexTaskClient extends IndexTaskClient { private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTaskClient.class); @@ -97,7 +97,7 @@ public boolean resume(final String id) } - public Map pause(final String id) + public Map pause(final String id) { log.debug("Pause task[%s]", id); @@ -207,7 +207,7 @@ public Map getMovingAverages(final String id) } } - public Map getCurrentOffsets(final String id, final boolean retry) + public Map getCurrentOffsets(final String id, final boolean retry) { log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry); @@ -229,7 +229,7 @@ public Map getCurrentOffsets(final String id, final boolean retry) } } - public TreeMap> getCheckpoints(final String id, final boolean retry) + public TreeMap> getCheckpoints(final String id, final boolean retry) { log.debug("GetCheckpoints task[%s] retry[%s]", id, retry); try { @@ -247,7 +247,7 @@ public TreeMap> getCheckpoints(final String id, final boole } } - public ListenableFuture>> getCheckpointsAsync( + public ListenableFuture>> getCheckpointsAsync( final String id, final boolean retry ) @@ -255,7 +255,7 @@ public ListenableFuture>> getCheckpointsAsync( return doAsync(() -> getCheckpoints(id, retry)); } - public Map getEndOffsets(final String id) + public Map getEndOffsets(final String id) { log.debug("GetEndOffsets task[%s]", id); @@ -273,7 +273,7 @@ public Map getEndOffsets(final String id) public boolean setEndOffsets( final String id, - final Map endOffsets, + final Map endOffsets, final boolean finalize ) throws IOException { @@ -313,26 +313,26 @@ public ListenableFuture getStartTimeAsync(final String id) } - public ListenableFuture> pauseAsync(final String id) + public ListenableFuture> pauseAsync(final String id) { return doAsync(() -> pause(id)); } public ListenableFuture setEndOffsetsAsync( final String id, - final Map endOffsets, + final Map endOffsets, final boolean finalize ) { return doAsync(() -> setEndOffsets(id, endOffsets, finalize)); } - public ListenableFuture> getCurrentOffsetsAsync(final String id, final boolean retry) + public ListenableFuture> getCurrentOffsetsAsync(final String id, final boolean retry) { return doAsync(() -> getCurrentOffsets(id, retry)); } - public ListenableFuture> getEndOffsetsAsync(final String id) + public ListenableFuture> getEndOffsetsAsync(final String id) { return doAsync(() -> getEndOffsets(id)); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index f3bb86d67fda..e4358d159586 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -33,21 +33,15 @@ * Interface for abstracting the indexing task run logic. Only used by Kafka indexing tasks, * but will also be used by Kinesis indexing tasks once implemented * - * @param Partition Number Type - * @param Sequence Number Type + * @param Partition Number Type + * @param Sequence Number Type */ -public interface SeekableStreamIndexTaskRunner extends ChatHandler +public interface SeekableStreamIndexTaskRunner extends ChatHandler { Appenderator getAppenderator(); - /** - * Run the task - */ TaskStatus run(TaskToolbox toolbox); - /** - * Stop the task - */ void stopGracefully(); @VisibleForTesting @@ -57,14 +51,14 @@ public interface SeekableStreamIndexTaskRunner extends ChatHandler SeekableStreamIndexTask.Status getStatus(); @VisibleForTesting - Map getCurrentOffsets(); + Map getCurrentOffsets(); @VisibleForTesting - Map getEndOffsets(); + Map getEndOffsets(); @VisibleForTesting Response setEndOffsets( - Map offsets, + Map offsets, boolean finish // this field is only for internal purposes, shouldn't be usually set by users ) throws InterruptedException; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index caf58f481577..c916d6c74023 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -29,78 +29,76 @@ import java.util.Objects; /** - * class that encapsulates a map of partitionId -> sequenceNumber. Redundant getters + * class that encapsulates a map of partitionId -> sequenceNumber. + * To be backward compatible with both Kafka and Kinesis datasource metadata when + * deserializing json. Redundant constrcturo fields stream, topic and + * partitionSequenceNumberMap and partitionOffsetMap are created. Only one of topic, stream + * should have a non-null value and only one of partitionOffsetMap and partitionSequenceNumberMap + * should have a non-null value. + *

+ * Redundant getters * are used for proper Jackson serialization/deserialization when processing terminologies - * used by Kafka and kinesis (i.e. topic vs. stream) + * used by Kafka and kinesis (i.e. topic vs. name) * - * @param partition id type - * @param sequence number type + * @param partition id type + * @param sequence number type */ -public class SeekableStreamPartitions +public class SeekableStreamPartitions { public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER"; - private final String stream; - private final Map map; + // stream/topic + private final String name; + // partitionId -> sequence number + private final Map map; @JsonCreator public SeekableStreamPartitions( @JsonProperty("stream") final String stream, @JsonProperty("topic") final String topic, - @JsonProperty("partitionSequenceNumberMap") final Map partitionSequenceNumberMap, - @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap + @JsonProperty("partitionSequenceNumberMap") final Map partitionSequenceNumberMap, + @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap ) { - this.stream = stream == null ? topic : stream; + this.name = stream == null ? topic : stream; this.map = ImmutableMap.copyOf(partitionOffsetMap == null ? partitionSequenceNumberMap : partitionOffsetMap); - Preconditions.checkArgument(this.stream != null); + Preconditions.checkArgument(this.name != null); Preconditions.checkArgument(map != null); - // Validate map - for (Map.Entry entry : map.entrySet()) { - Preconditions.checkArgument( - entry.getValue() != null, - "partition stream[" - + entry.getKey().toString() - + "] sequence/offset number[%" - + entry.getValue().toString() - + "] invalid" - ); - } } // constructor for backward compatibility - public SeekableStreamPartitions(@NotNull final String id, final Map partitionOffsetMap) + public SeekableStreamPartitions(@NotNull final String id, final Map partitionOffsetMap) { this(id, null, partitionOffsetMap, null); } @JsonProperty - public String getStream() + public String getName() { - return stream; + return name; } @JsonProperty public String getTopic() { - return stream; + return name; } - public Map getMap() + public Map getMap() { return map; } @JsonProperty - public Map getPartitionSequenceNumberMap() + public Map getPartitionSequenceNumberMap() { return map; } @JsonProperty - public Map getPartitionOffsetMap() + public Map getPartitionOffsetMap() { return map; } @@ -115,21 +113,21 @@ public boolean equals(Object o) return false; } SeekableStreamPartitions that = (SeekableStreamPartitions) o; - return Objects.equals(stream, that.stream) && + return Objects.equals(name, that.name) && Objects.equals(map, that.map); } @Override public int hashCode() { - return Objects.hash(stream, map); + return Objects.hash(name, map); } @Override public String toString() { return "SeekableStreamPartitions{" + - "stream/topic='" + stream + '\'' + + "name/topic='" + name + '\'' + ", partitionSequenceNumberMap/partitionOffsetMap=" + map + '}'; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java similarity index 65% rename from indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java index ef8a1ca25a0b..1dc4769d5bd4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/Record.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java @@ -22,40 +22,40 @@ import java.util.List; /** - * Represents a generic record with a T1 (partition id) and T2 (sequence number) and data + * Represents a generic record with a partitionType (partition id) and sequenceType (sequence number) and data * from a Kafka/Kinesis stream * - * @param partition id - * @param sequence number + * @param partition id + * @param sequence number */ -public class Record +public class OrderedPartitionableRecord { public static final String END_OF_SHARD_MARKER = "EOS"; - private final String streamName; - private final T1 partitionId; - private final T2 sequenceNumber; + private final String stream; + private final partitionType partitionId; + private final sequenceType sequenceNumber; private final List data; - public Record(String streamName, T1 partitionId, T2 sequenceNumber, List data) + public OrderedPartitionableRecord(String stream, partitionType partitionId, sequenceType sequenceNumber, List data) { - this.streamName = streamName; + this.stream = stream; this.partitionId = partitionId; this.sequenceNumber = sequenceNumber; this.data = data; } - public String getStreamName() + public String getStream() { - return streamName; + return stream; } - public T1 getPartitionId() + public partitionType getPartitionId() { return partitionId; } - public T2 getSequenceNumber() + public sequenceType getSequenceNumber() { return sequenceNumber; } @@ -67,6 +67,6 @@ public List getData() public StreamPartition getStreamPartition() { - return StreamPartition.of(streamName, partitionId); + return StreamPartition.of(stream, partitionId); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java similarity index 88% rename from indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java index 7d3e551386a3..05955b4947cd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/SequenceNumber.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java @@ -28,13 +28,13 @@ * * @param type of sequence number */ -public abstract class SequenceNumber implements Comparable> +public abstract class OrderedSequenceNumber implements Comparable> { private final T sequenceNumber; private final boolean isExclusive; private final boolean useExclusive; - protected SequenceNumber(T sequenceNumber, boolean useExclusive, boolean isExclusive) + protected OrderedSequenceNumber(T sequenceNumber, boolean useExclusive, boolean isExclusive) { this.sequenceNumber = sequenceNumber; this.useExclusive = useExclusive; @@ -67,7 +67,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - SequenceNumber that = (SequenceNumber) o; + OrderedSequenceNumber that = (OrderedSequenceNumber) o; return isExclusive == that.isExclusive && useExclusive == that.useExclusive && Objects.equals(sequenceNumber, that.sequenceNumber); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java index d233d0d81230..8568fb1cb382 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java @@ -31,26 +31,21 @@ * The RecordSupplier interface is a wrapper for the incoming seekable data stream * (i.e. Kafka consumer, Kinesis streams) * - * @param Partition Number Type - * @param Sequence Number Type + * @param Partition Number Type + * @param Sequence Number Type */ @Beta -public interface RecordSupplier extends Closeable +public interface RecordSupplier extends Closeable { - /** - * assigns a set of partitions to this RecordSupplier - * - * @param partitions partitions to assign - */ - void assign(Set> partitions); + void assign(Set> partitions); /** - * seek to specified sequence number + * seek to specified sequence number within a specific partition * * @param partition partition to seek * @param sequenceNumber sequence number to seek to */ - void seek(StreamPartition partition, T2 sequenceNumber); + void seek(StreamPartition partition, sequenceType sequenceNumber); /** * seek to the sequence number immediately following the given sequenceNumber @@ -58,28 +53,28 @@ public interface RecordSupplier extends Closeable * @param partition partition to seek * @param sequenceNumber sequence number to seek */ - void seekAfter(StreamPartition partition, T2 sequenceNumber); + void seekAfter(StreamPartition partition, sequenceType sequenceNumber); /** * seek a set of partitions to the earliest record position available in the stream * * @param partitions partitions to seek */ - void seekToEarliest(Set> partitions); + void seekToEarliest(Set> partitions); /** * seek a set of partitions to the latest/newest record position available in the stream * * @param partitions partitions to seek */ - void seekToLatest(Set> partitions); + void seekToLatest(Set> partitions); /** * get the current assignment * * @return set of assignments */ - Collection> getAssignment(); + Collection> getAssignment(); /** * poll the record at the current seeked to sequence in stream @@ -89,7 +84,7 @@ public interface RecordSupplier extends Closeable * @return record */ @Nullable - Record poll(long timeout); + OrderedPartitionableRecord poll(long timeout); /** * get the latest sequence number in stream @@ -100,7 +95,7 @@ public interface RecordSupplier extends Closeable * * @throws TimeoutException TimeoutException */ - T2 getLatestSequenceNumber(StreamPartition partition) throws TimeoutException; + sequenceType getLatestSequenceNumber(StreamPartition partition) throws TimeoutException; /** * get the earliest sequence number in stream @@ -111,7 +106,7 @@ public interface RecordSupplier extends Closeable * * @throws TimeoutException TimeoutException */ - T2 getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException; + sequenceType getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException; /** * returns the sequence number that the given partition is currently at @@ -120,16 +115,16 @@ public interface RecordSupplier extends Closeable * * @return sequence number */ - T2 position(StreamPartition partition); + sequenceType position(StreamPartition partition); /** * returns the set of partitions under the given stream * - * @param streamName name of stream + * @param stream name of stream * * @return set of partitions */ - Set getPartitionIds(String streamName); + Set getPartitionIds(String stream); /** * close the RecordSupplier diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java index 802e7ae5f6c1..406797f0718f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java @@ -24,30 +24,30 @@ * mostly used by {@link RecordSupplier} and * {@link org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor} * - * @param partition id type + * @param partition id type */ -public class StreamPartition +public class StreamPartition { - private final String streamName; - private final T1 partitionId; + private final String stream; + private final partitionType partitionId; - public StreamPartition(String streamName, T1 partitionId) + public StreamPartition(String stream, partitionType partitionId) { - this.streamName = streamName; + this.stream = stream; this.partitionId = partitionId; } - public static StreamPartition of(String streamName, T1 partitionId) + public static StreamPartition of(String stream, partitionType partitionId) { - return new StreamPartition<>(streamName, partitionId); + return new StreamPartition<>(stream, partitionId); } - public String getStreamName() + public String getStream() { - return streamName; + return stream; } - public T1 getPartitionId() + public partitionType getPartitionId() { return partitionId; } @@ -64,7 +64,7 @@ public boolean equals(Object o) StreamPartition that = (StreamPartition) o; - if (streamName != null ? !streamName.equals(that.streamName) : that.streamName != null) { + if (stream != null ? !stream.equals(that.stream) : that.stream != null) { return false; } return !(partitionId != null ? !partitionId.equals(that.partitionId) : that.partitionId != null); @@ -73,7 +73,7 @@ public boolean equals(Object o) @Override public int hashCode() { - int result = streamName != null ? streamName.hashCode() : 0; + int result = stream != null ? stream.hashCode() : 0; result = 31 * result + (partitionId != null ? partitionId.hashCode() : 0); return result; } @@ -82,7 +82,7 @@ public int hashCode() public String toString() { return "StreamPartition{" + - "streamName='" + streamName + '\'' + + "stream='" + stream + '\'' + ", partitionId='" + partitionId + '\'' + '}'; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 2705eca5d570..22a73ca38530 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -63,9 +63,9 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; -import org.apache.druid.indexing.seekablestream.common.Record; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; -import org.apache.druid.indexing.seekablestream.common.SequenceNumber; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; @@ -123,10 +123,10 @@ * stream sequences. *

* - * @param partition id type - * @param sequence number type + * @param partition id type + * @param sequence number type */ -public abstract class SeekableStreamSupervisor +public abstract class SeekableStreamSupervisor implements Supervisor { public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED"; @@ -157,22 +157,22 @@ private class TaskGroup // this task group has completed successfully, at which point this will be destroyed and a new task group will be // created with new starting offsets. This allows us to create replacement tasks for failed tasks that process the // same offsets, even if the values in [partitionGroups] has been changed. - final ImmutableMap startingSequences; + final ImmutableMap startingSequences; final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); final Optional minimumMessageTime; final Optional maximumMessageTime; - final Set exclusiveStartSequenceNumberPartitions; - final TreeMap> checkpointSequences = new TreeMap<>(); + final Set exclusiveStartSequenceNumberPartitions; + final TreeMap> checkpointSequences = new TreeMap<>(); final String baseSequenceName; DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action TaskGroup( int groupId, - ImmutableMap startingSequences, + ImmutableMap startingSequences, Optional minimumMessageTime, Optional maximumMessageTime, - Set exclusiveStartSequenceNumberPartitions + Set exclusiveStartSequenceNumberPartitions ) { this.groupId = groupId; @@ -186,7 +186,7 @@ private class TaskGroup this.baseSequenceName = generateSequenceName(startingSequences, minimumMessageTime, maximumMessageTime); } - int addNewCheckpoint(Map checkpoint) + int addNewCheckpoint(Map checkpoint) { checkpointSequences.put(checkpointSequences.lastKey() + 1, checkpoint); return checkpointSequences.lastKey(); @@ -203,7 +203,7 @@ private class TaskData { volatile TaskStatus status; volatile DateTime startTime; - volatile Map currentSequences = new HashMap<>(); + volatile Map currentSequences = new HashMap<>(); @Override public String toString() @@ -319,14 +319,14 @@ protected class CheckpointNotice implements Notice private final Integer nullableTaskGroupId; @Deprecated private final String baseSequenceName; - private final SeekableStreamDataSourceMetadata previousCheckpoint; - private final SeekableStreamDataSourceMetadata currentCheckpoint; + private final SeekableStreamDataSourceMetadata previousCheckpoint; + private final SeekableStreamDataSourceMetadata currentCheckpoint; public CheckpointNotice( @Nullable Integer nullableTaskGroupId, @Deprecated String baseSequenceName, - SeekableStreamDataSourceMetadata previousCheckpoint, - SeekableStreamDataSourceMetadata currentCheckpoint + SeekableStreamDataSourceMetadata previousCheckpoint, + SeekableStreamDataSourceMetadata currentCheckpoint ) { this.baseSequenceName = baseSequenceName; @@ -376,12 +376,12 @@ public void handle() throws ExecutionException, InterruptedException final TaskGroup taskGroup = taskGroups.get(taskGroupId); if (isValidTaskGroup(taskGroupId, taskGroup)) { - final TreeMap> checkpoints = taskGroup.checkpointSequences; + final TreeMap> checkpoints = taskGroup.checkpointSequences; // check validity of previousCheckpoint int index = checkpoints.size(); for (int sequenceId : checkpoints.descendingKeySet()) { - Map checkpoint = checkpoints.get(sequenceId); + Map checkpoint = checkpoints.get(sequenceId); // We have already verified the topic of the current checkpoint is same with that in ioConfig. // See checkpoint(). if (checkpoint.equals(previousCheckpoint.getSeekableStreamPartitions() @@ -399,7 +399,7 @@ public void handle() throws ExecutionException, InterruptedException log.info("Already checkpointed with offsets [%s]", checkpoints.lastEntry().getValue()); return; } - final Map newCheckpoint = checkpointTaskGroup(taskGroup, false).get(); + final Map newCheckpoint = checkpointTaskGroup(taskGroup, false).get(); taskGroup.addNewCheckpoint(newCheckpoint); log.info("Handled checkpoint notice, new checkpoint is [%s] for taskGroup [%s]", newCheckpoint, taskGroupId); } @@ -448,17 +448,17 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task // failures during publishing. // Map<{group ID}, Map<{partition ID}, {startingOffset}>> - private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); + private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); - protected final List partitionIds = new CopyOnWriteArrayList<>(); - protected volatile Map latestSequenceFromStream; + protected final List partitionIds = new CopyOnWriteArrayList<>(); + protected volatile Map latestSequenceFromStream; protected volatile DateTime sequenceLastUpdated; - private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); + private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); private final TaskStorage taskStorage; private final TaskMaster taskMaster; private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; - private final SeekableStreamIndexTaskClient taskClient; + private final SeekableStreamIndexTaskClient taskClient; private final ObjectMapper sortingMapper; private final SeekableStreamSupervisorSpec spec; private final String dataSource; @@ -477,14 +477,14 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) private final Object stopLock = new Object(); private final Object stateChangeLock = new Object(); private final Object recordSupplierLock = new Object(); - private final T2 END_OF_PARTITION; - private final T2 NOT_SET; + private final sequenceType END_OF_PARTITION; + private final sequenceType NOT_SET; private final boolean useExclusiveStartingSequence; private boolean listenerRegistered = false; private long lastRunTime; private int initRetryCounter = 0; private volatile DateTime firstRunTime; - private volatile RecordSupplier recordSupplier; + private volatile RecordSupplier recordSupplier; private volatile boolean started = false; private volatile boolean stopped = false; private volatile boolean lifecycleStarted = false; @@ -495,12 +495,12 @@ public SeekableStreamSupervisor( final TaskStorage taskStorage, final TaskMaster taskMaster, final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, - final SeekableStreamIndexTaskClientFactory> taskClientFactory, + final SeekableStreamIndexTaskClientFactory> taskClientFactory, final ObjectMapper mapper, final SeekableStreamSupervisorSpec spec, final RowIngestionMetersFactory rowIngestionMetersFactory, - final T2 NOT_SET, - final T2 END_OF_PARTITION, + final sequenceType NOT_SET, + final sequenceType END_OF_PARTITION, final boolean useExclusiveStartingSequence ) { @@ -800,21 +800,21 @@ public SupervisorReport getStatus() return generateReport(true); } - private SupervisorReport> generateReport( + private SupervisorReport> generateReport( boolean includeOffsets ) { int numPartitions = partitionGroups.values().stream().mapToInt(Map::size).sum(); - final SeekableStreamSupervisorReportPayload payload = createReportPayload(numPartitions, includeOffsets); + final SeekableStreamSupervisorReportPayload payload = createReportPayload(numPartitions, includeOffsets); - SupervisorReport> report = new SupervisorReport<>( + SupervisorReport> report = new SupervisorReport<>( dataSource, DateTimes.nowUtc(), payload ); - List> taskReports = Lists.newArrayList(); + List> taskReports = Lists.newArrayList(); try { for (TaskGroup taskGroup : taskGroups.values()) { @@ -822,7 +822,7 @@ private SupervisorReport String taskId = entry.getKey(); @Nullable DateTime startTime = entry.getValue().startTime; - Map currentOffsets = entry.getValue().currentSequences; + Map currentOffsets = entry.getValue().currentSequences; Long remainingSeconds = null; if (startTime != null) { remainingSeconds = Math.max( @@ -850,7 +850,7 @@ private SupervisorReport String taskId = entry.getKey(); @Nullable DateTime startTime = entry.getValue().startTime; - Map currentOffsets = entry.getValue().currentSequences; + Map currentOffsets = entry.getValue().currentSequences; Long remainingSeconds = null; if (taskGroup.completionTimeout != null) { remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - System.currentTimeMillis()) @@ -1064,9 +1064,9 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) } // Reset only the partitions in dataSourceMetadata if it has not been reset yet @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata resetMetadata = (SeekableStreamDataSourceMetadata) dataSourceMetadata; + final SeekableStreamDataSourceMetadata resetMetadata = (SeekableStreamDataSourceMetadata) dataSourceMetadata; - if (resetMetadata.getSeekableStreamPartitions().getStream().equals(ioConfig.getId())) { + if (resetMetadata.getSeekableStreamPartitions().getName().equals(ioConfig.getId())) { // metadata can be null final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); if (metadata != null && !checkSourceMetadataMatch(metadata)) { @@ -1077,15 +1077,15 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata currentMetadata = (SeekableStreamDataSourceMetadata) metadata; + final SeekableStreamDataSourceMetadata currentMetadata = (SeekableStreamDataSourceMetadata) metadata; // defend against consecutive reset requests from replicas // as well as the case where the metadata store do not have an entry for the reset partitions boolean doReset = false; - for (Entry resetPartitionOffset : resetMetadata.getSeekableStreamPartitions() + for (Entry resetPartitionOffset : resetMetadata.getSeekableStreamPartitions() .getMap() .entrySet()) { - final T2 partitionOffsetInMetadataStore = currentMetadata == null + final sequenceType partitionOffsetInMetadataStore = currentMetadata == null ? null : currentMetadata.getSeekableStreamPartitions() .getMap() @@ -1133,7 +1133,7 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) } else { log.warn( "Reset metadata topic [%s] and supervisor's stream name [%s] do not match", - resetMetadata.getSeekableStreamPartitions().getStream(), + resetMetadata.getSeekableStreamPartitions().getName(), ioConfig.getId() ); } @@ -1162,9 +1162,9 @@ private void killTasksInGroup(TaskGroup taskGroup) } } - private void killTaskGroupForPartitions(Set partitions) + private void killTaskGroupForPartitions(Set partitions) { - for (T1 partition : partitions) { + for (partitionType partition : partitions) { int taskGroupId = getTaskGroupIdForPartition(partition); killTasksInGroup(taskGroups.get(taskGroupId)); partitionGroups.remove(taskGroupId); @@ -1199,7 +1199,7 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti taskCount++; @SuppressWarnings("unchecked") - final SeekableStreamIndexTask seekableStreamIndexTask = (SeekableStreamIndexTask) task; + final SeekableStreamIndexTask seekableStreamIndexTask = (SeekableStreamIndexTask) task; final String taskId = task.getId(); // Determine which task group this task belongs to based on one of the partitions handled by this task. If we @@ -1209,7 +1209,7 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti // state, we will permit it to complete even if it doesn't match our current partition allocation to support // seamless schema migration. - Iterator it = seekableStreamIndexTask.getIOConfig() + Iterator it = seekableStreamIndexTask.getIOConfig() .getStartPartitions() .getMap() .keySet() @@ -1248,19 +1248,19 @@ public Boolean apply(SeekableStreamIndexTask.Status status) // update partitionGroups with the publishing task's offsets (if they are greater than what is // existing) so that the next tasks will start reading from where this task left off - Map publishingTaskEndOffsets = taskClient.getEndOffsets(taskId); + Map publishingTaskEndOffsets = taskClient.getEndOffsets(taskId); - for (Entry entry : publishingTaskEndOffsets.entrySet()) { - T1 partition = entry.getKey(); - T2 offset = entry.getValue(); - ConcurrentHashMap partitionOffsets = partitionGroups.get( + for (Entry entry : publishingTaskEndOffsets.entrySet()) { + partitionType partition = entry.getKey(); + sequenceType offset = entry.getValue(); + ConcurrentHashMap partitionOffsets = partitionGroups.get( getTaskGroupIdForPartition(partition) ); boolean succeeded; do { succeeded = true; - T2 previousOffset = partitionOffsets.putIfAbsent(partition, offset); + sequenceType previousOffset = partitionOffsets.putIfAbsent(partition, offset); if (previousOffset != null && (makeSequenceNumber(previousOffset).compareTo(makeSequenceNumber(offset))) < 0) { succeeded = partitionOffsets.replace(partition, previousOffset, offset); @@ -1268,7 +1268,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) } while (!succeeded); } } else { - for (T1 partition : seekableStreamIndexTask.getIOConfig() + for (partitionType partition : seekableStreamIndexTask.getIOConfig() .getStartPartitions() .getMap() .keySet()) { @@ -1384,12 +1384,12 @@ private void verifyAndMergeCheckpoints(final Collection taskGroupsToV private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) { final int groupId = taskGroup.groupId; - final List>>> taskSequences = new ArrayList<>(); - final List>>> futures = new ArrayList<>(); + final List>>> taskSequences = new ArrayList<>(); + final List>>> futures = new ArrayList<>(); final List taskIds = new ArrayList<>(); for (String taskId : taskGroup.taskIds()) { - final ListenableFuture>> checkpointsFuture = taskClient.getCheckpointsAsync( + final ListenableFuture>> checkpointsFuture = taskClient.getCheckpointsAsync( taskId, true ); @@ -1398,10 +1398,10 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) } try { - List>> futuresResult = + List>> futuresResult = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); for (int i = 0; i < futuresResult.size(); i++) { - final TreeMap> checkpoints = futuresResult.get(i); + final TreeMap> checkpoints = futuresResult.get(i); final String taskId = taskIds.get(i); if (checkpoints == null) { try { @@ -1434,14 +1434,14 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata latestDataSourceMetadata = (SeekableStreamDataSourceMetadata) rawDataSourceMetadata; + final SeekableStreamDataSourceMetadata latestDataSourceMetadata = (SeekableStreamDataSourceMetadata) rawDataSourceMetadata; final boolean hasValidOffsetsFromDb = latestDataSourceMetadata != null && latestDataSourceMetadata.getSeekableStreamPartitions() != null && ioConfig.getId().equals( - latestDataSourceMetadata.getSeekableStreamPartitions().getStream() + latestDataSourceMetadata.getSeekableStreamPartitions().getName() ); - final Map latestOffsetsFromDb; + final Map latestOffsetsFromDb; if (hasValidOffsetsFromDb) { latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamPartitions().getMap(); } else { @@ -1456,7 +1456,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) int taskIndex = 0; while (taskIndex < taskSequences.size()) { - TreeMap> taskCheckpoints = taskSequences.get(taskIndex).rhs; + TreeMap> taskCheckpoints = taskSequences.get(taskIndex).rhs; String taskId = taskSequences.get(taskIndex).lhs; if (earliestConsistentSequenceId.get() == -1) { // find the first replica task with earliest sequenceId consistent with datasource metadata in the metadata @@ -1464,8 +1464,8 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) if (taskCheckpoints.entrySet().stream().anyMatch( sequenceCheckpoint -> sequenceCheckpoint.getValue().entrySet().stream().allMatch( partitionOffset -> { - SequenceNumber offset = makeSequenceNumber(partitionOffset.getValue()); - SequenceNumber latestOffset = makeSequenceNumber( + OrderedSequenceNumber offset = makeSequenceNumber(partitionOffset.getValue()); + OrderedSequenceNumber latestOffset = makeSequenceNumber( latestOffsetsFromDb == null ? partitionOffset.getValue() : latestOffsetsFromDb.getOrDefault( partitionOffset @@ -1480,7 +1480,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) ) && earliestConsistentSequenceId.compareAndSet(-1, sequenceCheckpoint.getKey())) || ( pendingCompletionTaskGroups.getOrDefault(groupId, EMPTY_LIST).size() > 0 && earliestConsistentSequenceId.compareAndSet(-1, taskCheckpoints.firstKey()))) { - final SortedMap> latestCheckpoints = new TreeMap<>( + final SortedMap> latestCheckpoints = new TreeMap<>( taskCheckpoints.tailMap(earliestConsistentSequenceId.get()) ); log.info("Setting taskGroup sequences to [%s] for group [%d]", latestCheckpoints, groupId); @@ -1542,7 +1542,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) private void addDiscoveredTaskToPendingCompletionTaskGroups( int groupId, String taskId, - Map startingPartitions + Map startingPartitions ) { final CopyOnWriteArrayList taskGroupList = pendingCompletionTaskGroups.computeIfAbsent( @@ -1603,7 +1603,7 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) } @SuppressWarnings("unchecked") - SeekableStreamIndexTask task = (SeekableStreamIndexTask) taskOptional.get(); + SeekableStreamIndexTask task = (SeekableStreamIndexTask) taskOptional.get(); String taskSequenceName = task.getIOConfig().getBaseSequenceName(); if (taskGroups.get(taskGroupId) != null) { @@ -1623,14 +1623,14 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) } protected String generateSequenceName( - Map startPartitions, + Map startPartitions, Optional minimumMessageTime, Optional maximumMessageTime ) { StringBuilder sb = new StringBuilder(); - for (Entry entry : startPartitions.entrySet()) { + for (Entry entry : startPartitions.entrySet()) { sb.append(StringUtils.format("+%s(%s)", entry.getKey().toString(), entry.getValue().toString())); } String partitionOffsetStr = sb.toString().substring(1); @@ -1659,7 +1659,7 @@ protected String generateSequenceName( private void updatePartitionDataFromStream() { - Set partitionIds; + Set partitionIds; try { synchronized (recordSupplierLock) { partitionIds = recordSupplier.getPartitionIds(ioConfig.getId()); @@ -1677,15 +1677,15 @@ private void updatePartitionDataFromStream() log.debug("Found [%d] partitions for stream [%s]", partitionIds.size(), ioConfig.getId()); - Set closedPartitions = getOffsetsFromMetadataStorage() + Set closedPartitions = getOffsetsFromMetadataStorage() .entrySet() .stream() - .filter(x -> Record.END_OF_SHARD_MARKER.equals(x.getValue())) + .filter(x -> OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(x.getValue())) .map(Entry::getKey) .collect(Collectors.toSet()); boolean initialPartitionDiscovery = this.partitionIds.isEmpty(); - for (T1 partitionId : partitionIds) { + for (partitionType partitionId : partitionIds) { if (closedPartitions.contains(partitionId)) { continue; } @@ -1697,7 +1697,7 @@ private void updatePartitionDataFromStream() int taskGroupId = getTaskGroupIdForPartition(partitionId); - ConcurrentHashMap partitionMap = partitionGroups.computeIfAbsent( + ConcurrentHashMap partitionMap = partitionGroups.computeIfAbsent( taskGroupId, k -> new ConcurrentHashMap<>() ); @@ -1782,7 +1782,7 @@ public Boolean apply(@Nullable DateTime startTime) private void checkTaskDuration() throws ExecutionException, InterruptedException, TimeoutException { - final List>> futures = Lists.newArrayList(); + final List>> futures = Lists.newArrayList(); final List futureGroupIds = Lists.newArrayList(); for (Entry entry : taskGroups.entrySet()) { @@ -1814,11 +1814,11 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } } - List> results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + List> results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); for (int j = 0; j < results.size(); j++) { Integer groupId = futureGroupIds.get(j); TaskGroup group = taskGroups.get(groupId); - Map endOffsets = results.get(j); + Map endOffsets = results.get(j); if (endOffsets != null) { // set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion @@ -1826,7 +1826,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException pendingCompletionTaskGroups.computeIfAbsent(groupId, k -> new CopyOnWriteArrayList<>()).add(group); // set endOffsets as the next startOffsets - for (Entry entry : endOffsets.entrySet()) { + for (Entry entry : endOffsets.entrySet()) { partitionGroups.get(groupId).put(entry.getKey(), entry.getValue()); } } else { @@ -1848,7 +1848,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } } - private ListenableFuture> checkpointTaskGroup(final TaskGroup taskGroup, final boolean finalize) + private ListenableFuture> checkpointTaskGroup(final TaskGroup taskGroup, final boolean finalize) { if (finalize) { // 1) Check if any task completed (in which case we're done) and kill unassigned tasks @@ -1866,11 +1866,11 @@ private ListenableFuture> checkpointTaskGroup(final TaskGroup taskGr // publishing failed and we need to re-ingest) return Futures.transform( stopTasksInGroup(taskGroup), - new Function>() + new Function>() { @Nullable @Override - public Map apply(@Nullable Object input) + public Map apply(@Nullable Object input) { return null; } @@ -1890,23 +1890,23 @@ public Map apply(@Nullable Object input) } // 2) Pause running tasks - final List>> pauseFutures = Lists.newArrayList(); + final List>> pauseFutures = Lists.newArrayList(); final List pauseTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); for (final String taskId : pauseTaskIds) { pauseFutures.add(taskClient.pauseAsync(taskId)); } return Futures.transform( - Futures.successfulAsList(pauseFutures), new Function>, Map>() + Futures.successfulAsList(pauseFutures), new Function>, Map>() { @Nullable @Override - public Map apply(List> input) + public Map apply(List> input) { // 3) Build a map of the highest offset read by any task in the group for each partition - final Map endOffsets = new HashMap<>(); + final Map endOffsets = new HashMap<>(); for (int i = 0; i < input.size(); i++) { - Map result = input.get(i); + Map result = input.get(i); if (result == null || result.isEmpty()) { // kill tasks that didn't return a value String taskId = pauseTaskIds.get(i); @@ -1915,7 +1915,7 @@ public Map apply(List> input) taskGroup.tasks.remove(taskId); } else { // otherwise build a map of the highest offsets seen - for (Entry offset : result.entrySet()) { + for (Entry offset : result.entrySet()) { if (!endOffsets.containsKey(offset.getKey()) || makeSequenceNumber(endOffsets.get(offset.getKey())).compareTo( makeSequenceNumber(offset.getValue())) < 0) { @@ -2167,9 +2167,9 @@ private void createNewTasks() throws JsonProcessingException try { - Map> startingOffsets = generateStartingSequencesForPartitionGroup(groupId); + Map> startingOffsets = generateStartingSequencesForPartitionGroup(groupId); - ImmutableMap simpleStartingOffsets = startingOffsets + ImmutableMap simpleStartingOffsets = startingOffsets .entrySet() .stream() .filter(x -> x.getValue().get() != null) @@ -2178,7 +2178,7 @@ private void createNewTasks() throws JsonProcessingException ImmutableMap::copyOf )); - Set exclusiveStartSequenceNumberPartitions = startingOffsets + Set exclusiveStartSequenceNumberPartitions = startingOffsets .entrySet().stream() .filter(x -> x.getValue().get() != null && x.getValue().isExclusive()) .map(Entry::getKey) @@ -2211,7 +2211,7 @@ private void createNewTasks() throws JsonProcessingException Integer groupId = entry.getKey(); if (taskGroup.startingSequences == null || taskGroup.startingSequences - .values().stream().allMatch(x -> x == null || Record.END_OF_SHARD_MARKER.equals(x))) { + .values().stream().allMatch(x -> x == null || OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(x))) { log.debug("Nothing to read in any partition for taskGroup [%d], skipping task creation", groupId); continue; } @@ -2261,25 +2261,25 @@ protected int getNoticesQueueSize() return notices.size(); } - private ImmutableMap> generateStartingSequencesForPartitionGroup(int groupId) + private ImmutableMap> generateStartingSequencesForPartitionGroup(int groupId) throws TimeoutException { - ImmutableMap.Builder> builder = ImmutableMap.builder(); - for (Entry entry : partitionGroups.get(groupId).entrySet()) { - T1 partition = entry.getKey(); - T2 offset = entry.getValue(); + ImmutableMap.Builder> builder = ImmutableMap.builder(); + for (Entry entry : partitionGroups.get(groupId).entrySet()) { + partitionType partition = entry.getKey(); + sequenceType offset = entry.getValue(); if (!NOT_SET.equals(offset)) { // if we are given a startingOffset (set by a previous task group which is pending completion) then use it - if (!Record.END_OF_SHARD_MARKER.equals(offset)) { + if (!OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(offset)) { builder.put(partition, makeSequenceNumber(offset, useExclusiveStartingSequence, true)); } } else { // if we don't have a startingOffset (first run or we had some previous failures and reset the offsets) then // get the offset from metadata storage (if available) or Kafka/Kinesis (otherwise) - SequenceNumber offsetFromStorage = getOffsetFromStorageForPartition(partition); + OrderedSequenceNumber offsetFromStorage = getOffsetFromStorageForPartition(partition); - if (!Record.END_OF_SHARD_MARKER.equals(offset)) { + if (!OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(offset)) { builder.put(partition, offsetFromStorage); } } @@ -2291,10 +2291,10 @@ private ImmutableMap> generateStartingSequencesForPartiti * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't * find any data, it will retrieve the latest or earliest Kafka/Kinesis offset depending on the useEarliestOffset config. */ - private SequenceNumber getOffsetFromStorageForPartition(T1 partition) throws TimeoutException + private OrderedSequenceNumber getOffsetFromStorageForPartition(partitionType partition) throws TimeoutException { - final Map metadataOffsets = getOffsetsFromMetadataStorage(); - T2 offset = metadataOffsets.get(partition); + final Map metadataOffsets = getOffsetsFromMetadataStorage(); + sequenceType offset = metadataOffsets.get(partition); if (offset != null) { log.debug("Getting offset [%s] from metadata storage for partition [%s]", offset, partition); if (!taskTuningConfig.isSkipSequenceNumberAvailabilityCheck()) { @@ -2340,18 +2340,18 @@ private SequenceNumber getOffsetFromStorageForPartition(T1 partition) throws } } - private Map getOffsetsFromMetadataStorage() + private Map getOffsetsFromMetadataStorage() { final DataSourceMetadata dataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); if (dataSourceMetadata instanceof SeekableStreamDataSourceMetadata && checkSourceMetadataMatch(dataSourceMetadata)) { @SuppressWarnings("unchecked") - SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata).getSeekableStreamPartitions(); + SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata).getSeekableStreamPartitions(); if (partitions != null) { - if (!ioConfig.getId().equals(partitions.getStream())) { + if (!ioConfig.getId().equals(partitions.getName())) { log.warn( "Topic/stream in metadata storage [%s] doesn't match spec topic/stream [%s], ignoring stored offsets", - partitions.getStream(), + partitions.getName(), ioConfig.getId() ); return Collections.emptyMap(); @@ -2364,10 +2364,10 @@ && checkSourceMetadataMatch(dataSourceMetadata)) { return Collections.emptyMap(); } - protected T2 getOffsetFromStreamForPartition(T1 partition, boolean useEarliestOffset) throws TimeoutException + protected sequenceType getOffsetFromStreamForPartition(partitionType partition, boolean useEarliestOffset) throws TimeoutException { synchronized (recordSupplierLock) { - StreamPartition topicPartition = new StreamPartition<>(ioConfig.getId(), partition); + StreamPartition topicPartition = new StreamPartition<>(ioConfig.getId(), partition); if (!recordSupplier.getAssignment().contains(topicPartition)) { recordSupplier.assign(Collections.singleton(topicPartition)); } @@ -2381,12 +2381,12 @@ protected T2 getOffsetFromStreamForPartition(T1 partition, boolean useEarliestOf private void createTasksForGroup(int groupId, int replicas) throws JsonProcessingException { TaskGroup group = taskGroups.get(groupId); - Map startPartitions = group.startingSequences; - Map endPartitions = new HashMap<>(); - for (T1 partition : startPartitions.keySet()) { + Map startPartitions = group.startingSequences; + Map endPartitions = new HashMap<>(); + for (partitionType partition : startPartitions.keySet()) { endPartitions.put(partition, END_OF_PARTITION); } - Set exclusiveStartSequenceNumberPartitions = taskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; + Set exclusiveStartSequenceNumberPartitions = taskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; DateTime minimumMessageTime = taskGroups.get(groupId).minimumMessageTime.orNull(); DateTime maximumMessageTime = taskGroups.get(groupId).maximumMessageTime.orNull(); @@ -2403,7 +2403,7 @@ private void createTasksForGroup(int groupId, int replicas) throws JsonProcessin ); - List> taskList = createIndexTasks( + List> taskList = createIndexTasks( replicas, group.baseSequenceName, sortingMapper, @@ -2455,7 +2455,7 @@ private void updateCurrentOffsets() throws InterruptedException, ExecutionExcept ).map( task -> Futures.transform( taskClient.getCurrentOffsetsAsync(task.getKey(), false), - (Function, Void>) (currentSequences) -> { + (Function, Void>) (currentSequences) -> { if (currentSequences != null && !currentSequences.isEmpty()) { task.getValue().currentSequences = currentSequences; @@ -2472,7 +2472,7 @@ private void updateCurrentOffsets() throws InterruptedException, ExecutionExcept private void updateLatestOffsetsFromStream() { synchronized (recordSupplierLock) { - Set partitionIds = null; + Set partitionIds = null; try { partitionIds = recordSupplier.getPartitionIds(ioConfig.getId()); } @@ -2481,7 +2481,7 @@ private void updateLatestOffsetsFromStream() Throwables.propagate(e); } - Set> partitions = partitionIds + Set> partitions = partitionIds .stream() .map(e -> new StreamPartition<>(ioConfig.getId(), e)) .collect(Collectors.toSet()); @@ -2498,7 +2498,7 @@ private void updateLatestOffsetsFromStream() } - protected Map getHighestCurrentOffsets() + protected Map getHighestCurrentOffsets() { return taskGroups .values() @@ -2512,7 +2512,7 @@ protected Map getHighestCurrentOffsets() )); } - private SequenceNumber makeSequenceNumber(T2 seq) + private OrderedSequenceNumber makeSequenceNumber(sequenceType seq) { return makeSequenceNumber(seq, false, false); } @@ -2561,12 +2561,12 @@ public abstract void checkpoint( */ protected abstract SeekableStreamIOConfig createIoConfig( int groupId, - Map startPartitions, - Map endPartitions, + Map startPartitions, + Map endPartitions, String baseSequenceName, DateTime minimumMessageTime, DateTime maximumMessageTime, - Set exclusiveStartSequenceNumberPartitions, + Set exclusiveStartSequenceNumberPartitions, SeekableStreamSupervisorIOConfig ioConfig ); @@ -2578,11 +2578,11 @@ protected abstract SeekableStreamIOConfig createIoConfig( * * @throws JsonProcessingException */ - protected abstract List> createIndexTasks( + protected abstract List> createIndexTasks( int replicas, String baseSequenceName, ObjectMapper sortingMapper, - TreeMap> sequenceOffsets, + TreeMap> sequenceOffsets, SeekableStreamIOConfig taskIoConfig, SeekableStreamTuningConfig taskTuningConfig, RowIngestionMetersFactory rowIngestionMetersFactory @@ -2596,7 +2596,7 @@ protected abstract List> createIndexTasks( * * @return taskgroup id */ - protected abstract int getTaskGroupIdForPartition(T1 partition); + protected abstract int getTaskGroupIdForPartition(partitionType partition); /** * checks if the passed in DataSourceMetadata is a specific instance @@ -2626,18 +2626,18 @@ protected abstract List> createIndexTasks( * * @return specific instance of datasource metadata */ - protected abstract SeekableStreamDataSourceMetadata createDataSourceMetaData( + protected abstract SeekableStreamDataSourceMetadata createDataSourceMetaData( String - stream, Map map + stream, Map map ); /** - * wraps the passed in T2 sequence number into a {@link SequenceNumber} object + * wraps the passed in sequenceType sequence number into a {@link OrderedSequenceNumber} object * to facilitate comparison and accomodate exclusive starting sequennce in kinesis * - * @return specific instance of [Kafka/Kinesis]SequenceNumber + * @return specific instance of [Kafka/Kinesis]OrderedSequenceNumber */ - protected abstract SequenceNumber makeSequenceNumber(T2 seq, boolean useExclusive, boolean isExclusive); + protected abstract OrderedSequenceNumber makeSequenceNumber(sequenceType seq, boolean useExclusive, boolean isExclusive); /** * schedules periodic emitLag() reporting for Kafka, not yet implemented in Kinesis, @@ -2651,21 +2651,21 @@ protected abstract SeekableStreamDataSourceMetadata createDataSourceMeta * * @return map of partition id -> lag */ - protected abstract Map getLagPerPartition(Map currentOffsets); + protected abstract Map getLagPerPartition(Map currentOffsets); /** * returns an instance of a specific Kinesis/Kafka recordSupplier * * @return specific instance of Kafka/Kinesis RecordSupplier */ - protected abstract RecordSupplier setupRecordSupplier(); + protected abstract RecordSupplier setupRecordSupplier(); /** * creates a specific instance of Kafka/Kinesis Supervisor Report Payload * * @return specific instance of Kafka/Kinesis Supervisor Report Payload */ - protected abstract SeekableStreamSupervisorReportPayload createReportPayload( + protected abstract SeekableStreamSupervisorReportPayload createReportPayload( int numPartitions, boolean includeOffsets ); @@ -2677,6 +2677,6 @@ protected abstract SeekableStreamSupervisorReportPayload createReportPay * * @throws TimeoutException */ - protected abstract boolean checkSequenceAvailability(@NotNull T1 partition, @NotNull T2 sequenceFromMetadata) + protected abstract boolean checkSequenceAvailability(@NotNull partitionType partition, @NotNull sequenceType sequenceFromMetadata) throws TimeoutException; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java index 98083ae53663..7f8ffb860bc2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -29,7 +29,7 @@ import java.util.List; import java.util.Map; -public abstract class SeekableStreamSupervisorReportPayload +public abstract class SeekableStreamSupervisorReportPayload { private final String dataSource; private final String id; @@ -38,8 +38,8 @@ public abstract class SeekableStreamSupervisorReportPayload private final long durationSeconds; private final List activeTasks; private final List publishingTasks; - private final Map latestOffsets; - private final Map minimumLag; + private final Map latestOffsets; + private final Map minimumLag; private final Long aggregateLag; private final DateTime offsetsLastUpdated; private final boolean suspended; @@ -50,8 +50,8 @@ public SeekableStreamSupervisorReportPayload( int partitions, int replicas, long durationSeconds, - @Nullable Map latestOffsets, - @Nullable Map minimumLag, + @Nullable Map latestOffsets, + @Nullable Map minimumLag, @Nullable Long aggregateLag, @Nullable DateTime offsetsLastUpdated, boolean suspended @@ -131,14 +131,14 @@ public List getPublishingTasks() @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getLatestOffsets() + public Map getLatestOffsets() { return latestOffsets; } @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getMinimumLag() + public Map getMinimumLag() { return minimumLag; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java index ecc876321de0..8ca2e1c4a5b5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java @@ -26,24 +26,24 @@ import javax.annotation.Nullable; import java.util.Map; -public class TaskReportData +public class TaskReportData { private final String id; - private final Map startingOffsets; + private final Map startingOffsets; private final DateTime startTime; private final Long remainingSeconds; private final TaskType type; - private Map currentOffsets; - private final Map lag; + private Map currentOffsets; + private final Map lag; public TaskReportData( String id, - @Nullable Map startingOffsets, - @Nullable Map currentOffsets, + @Nullable Map startingOffsets, + @Nullable Map currentOffsets, @Nullable DateTime startTime, Long remainingSeconds, TaskType type, - @Nullable Map lag + @Nullable Map lag ) { this.id = id; @@ -63,14 +63,14 @@ public String getId() @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getStartingOffsets() + public Map getStartingOffsets() { return startingOffsets; } @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getCurrentOffsets() + public Map getCurrentOffsets() { return currentOffsets; } @@ -95,12 +95,12 @@ public TaskType getType() @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getLag() + public Map getLag() { return lag; } - public void setCurrentSequenceNumbers(Map currentOffsets) + public void setCurrentSequenceNumbers(Map currentOffsets) { this.currentOffsets = currentOffsets; } From 3aac350cc0958ea8360b4c5cdb34ade5fd6bdd36 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Fri, 12 Oct 2018 15:39:57 -0700 Subject: [PATCH 23/87] improve recordsupplier and add unit tests --- .../indexing/kafka/KafkaRecordSupplier.java | 39 +- .../kafka/supervisor/KafkaSupervisor.java | 4 +- .../kafka/KafkaRecordSupplierTest.java | 433 +++++++++++++++ .../kinesis/KinesisRecordSupplier.java | 4 +- .../kinesis/KinesisRecordSupplierTest.java | 491 ++++++++++++++++++ .../common/OrderedPartitionableRecord.java | 31 +- 6 files changed, 972 insertions(+), 30 deletions(-) create mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java create mode 100644 extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 4eb321c977fa..64daee07ca1a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; @@ -44,7 +43,6 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; public class KafkaRecordSupplier implements RecordSupplier @@ -53,16 +51,16 @@ public class KafkaRecordSupplier implements RecordSupplier private static final Random RANDOM = ThreadLocalRandom.current(); private final KafkaConsumer consumer; - private final KafkaSupervisorIOConfig ioConfig; + private final Map consumerProperties; private boolean closed; private final BlockingQueue> records; public KafkaRecordSupplier( - KafkaSupervisorIOConfig ioConfig + Map consumerProperties ) { - this.ioConfig = ioConfig; + this.consumerProperties = consumerProperties; this.consumer = getKafkaConsumer(); this.closed = false; this.records = new LinkedBlockingQueue<>(); @@ -120,25 +118,17 @@ public Set> getAssignment() @Override public OrderedPartitionableRecord poll(long timeout) { - if (records.isEmpty()) { - ConsumerRecords polledRecords = consumer.poll(timeout); - for (ConsumerRecord record : polledRecords) { - records.offer(new OrderedPartitionableRecord<>( - record.topic(), - record.partition(), - record.offset(), - ImmutableList.of(record.value()) - )); - } - } - - try { - return records.poll(timeout, TimeUnit.MILLISECONDS); - } - catch (InterruptedException e) { - log.warn(e, "InterruptedException"); - return null; + ConsumerRecords polledRecords = consumer.poll(timeout); + if (!polledRecords.isEmpty()) { + ConsumerRecord record = polledRecords.iterator().next(); + return new OrderedPartitionableRecord<>( + record.topic(), + record.partition(), + record.offset(), + record.value() == null ? null : ImmutableList.of(record.value()) + ); } + return null; } @Override @@ -189,8 +179,9 @@ private KafkaConsumer getKafkaConsumer() props.setProperty("metadata.max.age.ms", "10000"); props.setProperty("group.id", StringUtils.format("kafka-supervisor-%s", getRandomId())); + props.setProperty("max.poll.records", "1"); - props.putAll(ioConfig.getConsumerProperties()); + props.putAll(consumerProperties); props.setProperty("enable.auto.commit", "false"); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 4770577943b1..fb1a73ab4ccc 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -44,8 +44,8 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; -import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; @@ -158,7 +158,7 @@ public void checkpoint( @Override protected RecordSupplier setupRecordSupplier() { - return new KafkaRecordSupplier(spec.getIoConfig()); + return new KafkaRecordSupplier(spec.getIoConfig().getConsumerProperties()); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java new file mode 100644 index 000000000000..745fabb82898 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java @@ -0,0 +1,433 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kafka; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.curator.test.TestingCluster; +import org.apache.druid.indexing.kafka.test.TestBroker; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +public class KafkaRecordSupplierTest +{ + private static final Logger log = new Logger(KafkaRecordSupplierTest.class); + private static String topic = "topic"; + private static long poll_timeout_millis = 1000; + private static int topicPosFix = 0; + + private static TestingCluster zkServer; + private static TestBroker kafkaServer; + + private List> records; + + + private static List> generateRecords(String topic) + { + return ImmutableList.of( + new ProducerRecord<>(topic, 0, null, JB("2008", "a", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, JB("2009", "b", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, JB("2010", "c", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, JB("2011", "d", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, JB("2011", "e", "y", "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<>(topic, 0, null, StringUtils.toUtf8("unparseable")), + new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")), + new ProducerRecord<>(topic, 0, null, null), + new ProducerRecord<>(topic, 0, null, JB("2013", "f", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "notanumber", "20.0", "1.0")), + new ProducerRecord<>(topic, 1, null, JB("2049", "f", "y", "10", "notanumber", "1.0")), + new ProducerRecord<>(topic, 1, null, JB("2049", "f", "y", "10", "20.0", "notanumber")), + new ProducerRecord<>(topic, 1, null, JB("2012", "g", "y", "10", "20.0", "1.0")), + new ProducerRecord<>(topic, 1, null, JB("2011", "h", "y", "10", "20.0", "1.0")) + ); + } + + private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) + { + try { + return new ObjectMapper().writeValueAsBytes( + ImmutableMap.builder() + .put("timestamp", timestamp) + .put("dim1", dim1) + .put("dim2", dim2) + .put("dimLong", dimLong) + .put("dimFloat", dimFloat) + .put("met1", met1) + .build() + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + private static String getTopicName() + { + return "topic-" + topicPosFix++; + } + + private Set> createOrderedPartitionableRecords() + { + Map partitionToOffset = new HashMap<>(); + return records.stream().map(r -> { + long offset = 0; + if (partitionToOffset.containsKey(r.partition())) { + offset = partitionToOffset.get(r.partition()); + partitionToOffset.put(r.partition(), offset + 1); + } else { + partitionToOffset.put(r.partition(), 1L); + } + return new OrderedPartitionableRecord<>( + topic, + r.partition(), + offset, + r.value() == null ? null : ImmutableList.of(r.value()) + ); + }).collect(Collectors.toSet()); + } + + @BeforeClass + public static void setupClass() throws Exception + { + zkServer = new TestingCluster(1); + zkServer.start(); + + kafkaServer = new TestBroker( + zkServer.getConnectString(), + null, + 1, + ImmutableMap.of("num.partitions", "2") + ); + kafkaServer.start(); + + } + + @Before + public void setupTest() + { + topic = getTopicName(); + records = generateRecords(topic); + } + + @AfterClass + public static void tearDownClass() throws Exception + { + kafkaServer.close(); + kafkaServer = null; + + zkServer.stop(); + zkServer = null; + } + + @Test + public void testSupplierSetup() throws ExecutionException, InterruptedException + { + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : records) { + kafkaProducer.send(record).get(); + } + } + + Set> partitions = ImmutableSet.of( + StreamPartition.of(topic, 0), + StreamPartition.of(topic, 1) + ); + + KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( + kafkaServer.consumerProperties()); + + Assert.assertTrue(recordSupplier.getAssignment().isEmpty()); + + recordSupplier.assign(partitions); + + Assert.assertEquals(partitions, recordSupplier.getAssignment()); + Assert.assertEquals(ImmutableSet.of(0, 1), recordSupplier.getPartitionIds(topic)); + + recordSupplier.close(); + } + + @Test + public void testPoll() throws InterruptedException, ExecutionException + { + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : records) { + kafkaProducer.send(record).get(); + } + } + + Set> partitions = ImmutableSet.of( + StreamPartition.of(topic, 0), + StreamPartition.of(topic, 1) + ); + + KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( + kafkaServer.consumerProperties()); + + recordSupplier.assign(partitions); + recordSupplier.seekToEarliest(partitions); + + Set> supplierRecords = new HashSet<>(); + OrderedPartitionableRecord record = recordSupplier.poll(poll_timeout_millis); + + while (record != null) { + supplierRecords.add(record); + record = recordSupplier.poll(poll_timeout_millis); + } + + Set> initialRecords = createOrderedPartitionableRecords(); + + Assert.assertEquals(partitions, recordSupplier.getAssignment()); + Assert.assertEquals(records.size(), supplierRecords.size()); + Assert.assertTrue(initialRecords.containsAll(supplierRecords)); + + recordSupplier.close(); + } + + + @Test + public void testPollAfterMoreDataAdded() throws InterruptedException, ExecutionException + { + // Insert data + + KafkaProducer producer = kafkaServer.newProducer(); + for (ProducerRecord record : records.subList(0, 13)) { + producer.send(record).get(); + } + + Set> partitions = ImmutableSet.of( + StreamPartition.of(topic, 0), + StreamPartition.of(topic, 1) + ); + + + KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( + kafkaServer.consumerProperties()); + + recordSupplier.assign(partitions); + recordSupplier.seekToEarliest(partitions); + + Set> supplierRecords = new HashSet<>(); + OrderedPartitionableRecord record = recordSupplier.poll(poll_timeout_millis); + + while (record != null) { + supplierRecords.add(record); + record = recordSupplier.poll(poll_timeout_millis); + } + + // Insert data + for (ProducerRecord rec : records.subList(13, 15)) { + producer.send(rec).get(); + } + + + record = recordSupplier.poll(poll_timeout_millis); + while (record != null) { + supplierRecords.add(record); + record = recordSupplier.poll(poll_timeout_millis); + } + + Set> initialRecords = createOrderedPartitionableRecords(); + + Assert.assertEquals(records.size(), supplierRecords.size()); + Assert.assertTrue(initialRecords.containsAll(supplierRecords)); + + + recordSupplier.close(); + } + + @Test + public void testSeek() throws InterruptedException, ExecutionException + { + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : records) { + kafkaProducer.send(record).get(); + } + } + + StreamPartition partition0 = StreamPartition.of(topic, 0); + StreamPartition partition1 = StreamPartition.of(topic, 1); + + Set> partitions = ImmutableSet.of( + StreamPartition.of(topic, 0), + StreamPartition.of(topic, 1) + ); + + KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( + kafkaServer.consumerProperties()); + + recordSupplier.assign(partitions); + + Assert.assertEquals(0L, (long) recordSupplier.getEarliestSequenceNumber(partition0)); + Assert.assertEquals(0L, (long) recordSupplier.getEarliestSequenceNumber(partition1)); + + recordSupplier.seek(partition0, 2L); + recordSupplier.seek(partition1, 2L); + + + Set> supplierRecords = new HashSet<>(); + OrderedPartitionableRecord record = recordSupplier.poll(poll_timeout_millis); + + while (record != null) { + supplierRecords.add(record); + record = recordSupplier.poll(poll_timeout_millis); + } + + Set> initialRecords = createOrderedPartitionableRecords(); + + Assert.assertEquals(11, supplierRecords.size()); + Assert.assertTrue(initialRecords.containsAll(supplierRecords)); + + + recordSupplier.close(); + + } + + @Test + public void testSeekToLatest() throws InterruptedException, ExecutionException + { + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : records) { + kafkaProducer.send(record).get(); + } + } + + StreamPartition partition0 = StreamPartition.of(topic, 0); + StreamPartition partition1 = StreamPartition.of(topic, 1); + + Set> partitions = ImmutableSet.of( + StreamPartition.of(topic, 0), + StreamPartition.of(topic, 1) + ); + + KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( + kafkaServer.consumerProperties()); + + recordSupplier.assign(partitions); + + Assert.assertEquals(0L, (long) recordSupplier.getEarliestSequenceNumber(partition0)); + Assert.assertEquals(0L, (long) recordSupplier.getEarliestSequenceNumber(partition1)); + + recordSupplier.seekToLatest(partitions); + Assert.assertNull(recordSupplier.poll(poll_timeout_millis)); + + recordSupplier.close(); + } + + @Test(expected = IllegalStateException.class) + public void testSeekUnassigned() throws InterruptedException, ExecutionException + { + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : records) { + kafkaProducer.send(record).get(); + } + } + + StreamPartition partition0 = StreamPartition.of(topic, 0); + StreamPartition partition1 = StreamPartition.of(topic, 1); + + Set> partitions = ImmutableSet.of( + StreamPartition.of(topic, 0) + ); + + KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( + kafkaServer.consumerProperties()); + + recordSupplier.assign(partitions); + + Assert.assertEquals(0, (long) recordSupplier.getEarliestSequenceNumber(partition0)); + + recordSupplier.seekToEarliest(Collections.singleton(partition1)); + + recordSupplier.close(); + } + + @Test + public void testPosition() throws ExecutionException, InterruptedException + { + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : records) { + kafkaProducer.send(record).get(); + } + } + + StreamPartition partition0 = StreamPartition.of(topic, 0); + StreamPartition partition1 = StreamPartition.of(topic, 1); + + Set> partitions = ImmutableSet.of( + StreamPartition.of(topic, 0), + StreamPartition.of(topic, 1) + ); + + KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( + kafkaServer.consumerProperties()); + + recordSupplier.assign(partitions); + + Assert.assertEquals(0L, (long) recordSupplier.position(partition0)); + Assert.assertEquals(0L, (long) recordSupplier.position(partition1)); + + recordSupplier.seek(partition0, 4L); + recordSupplier.seek(partition1, 5L); + + Assert.assertEquals(4L, (long) recordSupplier.position(partition0)); + Assert.assertEquals(5L, (long) recordSupplier.position(partition1)); + + recordSupplier.poll(poll_timeout_millis); + Assert.assertTrue(recordSupplier.position(partition0) == 5L || recordSupplier.position(partition1) == 6L); + + recordSupplier.seekToEarliest(Collections.singleton(partition0)); + Assert.assertEquals(0L, (long) recordSupplier.position(partition0)); + + recordSupplier.seekToLatest(Collections.singleton(partition0)); + Assert.assertEquals(11L, (long) recordSupplier.position(partition0)); + + recordSupplier.close(); + } +} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 671529dac10b..3a2196c3b693 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -422,8 +422,8 @@ public String getEarliestSequenceNumber(StreamPartition partition) throw @Override public String position(StreamPartition partition) { - // only needed in calcLag, which is not supported in Kinesis - throw new UnsupportedOperationException("position in KinesisRecordSupplier not supported"); + checkIfClosed(); + return null; } @Override diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java new file mode 100644 index 000000000000..68386c215943 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -0,0 +1,491 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import cloud.localstack.Localstack; +import cloud.localstack.TestUtils; +import cloud.localstack.docker.LocalstackDockerTestRunner; +import cloud.localstack.docker.annotation.LocalstackDockerProperties; +import com.amazonaws.http.SdkHttpMetadata; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.model.PutRecordsRequest; +import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry; +import com.amazonaws.services.kinesis.model.PutRecordsResult; +import com.amazonaws.services.kinesis.model.PutRecordsResultEntry; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +@RunWith(LocalstackDockerTestRunner.class) +@LocalstackDockerProperties(services = {"kinesis"}) +public class KinesisRecordSupplierTest +{ + static { + TestUtils.setEnv("AWS_CBOR_DISABLE", "1"); + if (Localstack.useSSL()) { + TestUtils.disableSslCertChecking(); + } + } + + private static final Logger log = new Logger(KinesisRecordSupplierTest.class); + private static String stream = "streamm"; + private static long poll_timeout_millis = 1000; + private static String shardId1 = "shardId-000000000001"; + private static String shardId0 = "shardId-000000000000"; + private static int streamPosFix = 0; + private static final List records = ImmutableList.of( + generateRequestEntry( + "1", + JB("2011", "d", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2011", "e", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0") + ), + generateRequestEntry("1", StringUtils.toUtf8("unparseable")), + generateRequestEntry( + "1", + StringUtils.toUtf8("unparseable2") + ), + generateRequestEntry("1", "{}".getBytes()), + generateRequestEntry( + "1", + JB("2013", "f", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "1", + JB("2049", "f", "y", "notanumber", "20.0", "1.0") + ), + generateRequestEntry( + "123123", + JB("2012", "g", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "123123", + JB("2011", "h", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "123123", + JB("2008", "a", "y", "10", "20.0", "1.0") + ), + generateRequestEntry( + "123123", + JB("2009", "b", "y", "10", "20.0", "1.0") + ) + ); + + private static PutRecordsRequestEntry generateRequestEntry(String partition, byte[] data) + { + return new PutRecordsRequestEntry().withPartitionKey(partition) + .withData(ByteBuffer.wrap(data)); + } + + private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) + { + try { + return new ObjectMapper().writeValueAsBytes( + ImmutableMap.builder() + .put("timestamp", timestamp) + .put("dim1", dim1) + .put("dim2", dim2) + .put("dimLong", dimLong) + .put("dimFloat", dimFloat) + .put("met1", met1) + .build() + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + + private AmazonKinesis getKinesisClientInstance() throws InterruptedException + { + AmazonKinesis kinesis = TestUtils.getClientKinesis(); + SdkHttpMetadata createRes = kinesis.createStream(stream, 2).getSdkHttpMetadata(); + // sleep required because of kinesalite + Thread.sleep(500); + return kinesis; + } + + private static PutRecordsRequest generateRecordsRequests(String stream) + { + return new PutRecordsRequest() + .withStreamName(stream) + .withRecords(records); + } + + private static PutRecordsRequest generateRecordsRequests(String stream, int first, int last) + { + return new PutRecordsRequest() + .withStreamName(stream) + .withRecords(records.subList(first, last)); + } + + private static List insertData( + AmazonKinesis kinesis, + PutRecordsRequest req + ) + { + PutRecordsResult res = kinesis.putRecords(req); + Assert.assertEquals((int) res.getFailedRecordCount(), 0); + return res.getRecords(); + } + + private static String getStreamName() + { + return "stream-" + streamPosFix++; + } + + @Before + public void setupTest() throws IOException, InterruptedException + { + stream = getStreamName(); + } + + @Test + public void testSupplierSetup() throws InterruptedException + { + + getKinesisClientInstance(); + + Set> partitions = ImmutableSet.of( + StreamPartition.of(stream, shardId0), + StreamPartition.of(stream, shardId1) + ); + + KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( + Localstack.getEndpointKinesis(), + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + 1, + 0, + 2, + null, + null, + false, + 100, + 5000, + 5000, + 60000 + ); + + Assert.assertTrue(recordSupplier.getAssignment().isEmpty()); + + recordSupplier.assign(partitions); + + Assert.assertEquals(partitions, recordSupplier.getAssignment()); + Assert.assertEquals(ImmutableSet.of(shardId1, shardId0), recordSupplier.getPartitionIds(stream)); + Assert.assertNull(recordSupplier.poll(100)); + + recordSupplier.close(); + } + + @Test + public void testPoll() throws InterruptedException + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); + Set> initialRecords = insertDataResults.stream() + .map(r -> new OrderedPartitionableRecord<>( + stream, + r.getShardId(), + r.getSequenceNumber(), + null + )) + .collect(Collectors.toSet()); + + Set> partitions = ImmutableSet.of( + StreamPartition.of(stream, shardId0), + StreamPartition.of(stream, shardId1) + ); + + KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( + Localstack.getEndpointKinesis(), + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + 1, + 0, + 2, + null, + null, + false, + 100, + 5000, + 5000, + 60000 + ); + recordSupplier.assign(partitions); + recordSupplier.seekToEarliest(partitions); + + Set> supplierRecords = new HashSet<>(); + OrderedPartitionableRecord record = recordSupplier.poll(poll_timeout_millis); + + while (record != null) { + supplierRecords.add(record); + record = recordSupplier.poll(poll_timeout_millis); + } + + Assert.assertEquals(partitions, recordSupplier.getAssignment()); + Assert.assertEquals(initialRecords.size(), supplierRecords.size()); + Assert.assertTrue(supplierRecords.containsAll(initialRecords)); + + recordSupplier.close(); + } + + @Test + public void testPollAfterMoreDataAdded() throws InterruptedException + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List insertDataResults1 = insertData(kinesis, generateRecordsRequests(stream, 0, 5)); + Set> initialRecords = insertDataResults1.stream() + .map(r -> new OrderedPartitionableRecord<>( + stream, + r.getShardId(), + r.getSequenceNumber(), + null + )) + .collect(Collectors.toSet()); + + Set> partitions = ImmutableSet.of( + StreamPartition.of(stream, shardId0), + StreamPartition.of(stream, shardId1) + ); + + KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( + Localstack.getEndpointKinesis(), + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + 1, + 0, + 2, + null, + null, + false, + 100, + 5000, + 5000, + 60000 + ); + + recordSupplier.assign(partitions); + recordSupplier.seekToEarliest(partitions); + + Set> supplierRecords = new HashSet<>(); + OrderedPartitionableRecord record = recordSupplier.poll(poll_timeout_millis); + + while (record != null) { + supplierRecords.add(record); + record = recordSupplier.poll(poll_timeout_millis); + } + + List insertDataResults2 = insertData(kinesis, generateRecordsRequests(stream, 5, 12)); + insertDataResults2.forEach(entry -> initialRecords.add(new OrderedPartitionableRecord<>( + stream, + entry.getShardId(), + entry.getSequenceNumber(), + null + ))); + + record = recordSupplier.poll(poll_timeout_millis); + while (record != null) { + supplierRecords.add(record); + record = recordSupplier.poll(poll_timeout_millis); + } + + Assert.assertEquals(initialRecords.size(), supplierRecords.size()); + Assert.assertTrue(supplierRecords.containsAll(initialRecords)); + + recordSupplier.close(); + } + + @Test + public void testSeek() throws InterruptedException, TimeoutException + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); + + StreamPartition shard0 = StreamPartition.of(stream, shardId0); + StreamPartition shard1 = StreamPartition.of(stream, shardId1); + Set> partitions = ImmutableSet.of( + shard0, + shard1 + ); + + KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( + Localstack.getEndpointKinesis(), + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + 1, + 0, + 2, + null, + null, + false, + 100, + 5000, + 5000, + 60000 + ); + + recordSupplier.assign(partitions); + + Assert.assertEquals(insertDataResults.get(0).getSequenceNumber(), recordSupplier.getEarliestSequenceNumber(shard1)); + Assert.assertEquals(insertDataResults.get(8).getSequenceNumber(), recordSupplier.getEarliestSequenceNumber(shard0)); + + recordSupplier.seek(shard1, insertDataResults.get(2).getSequenceNumber()); + recordSupplier.seek(shard0, insertDataResults.get(10).getSequenceNumber()); + + Set> initialRecords1 = insertDataResults.subList(2, 8).stream() + .map(r -> new OrderedPartitionableRecord<>( + stream, + r.getShardId(), + r.getSequenceNumber(), + null + )) + .collect(Collectors.toSet()); + + Set> initialRecords2 = insertDataResults.subList(10, 12).stream() + .map(r -> new OrderedPartitionableRecord<>( + stream, + r.getShardId(), + r.getSequenceNumber(), + null + )) + .collect(Collectors.toSet()); + + Set> supplierRecords = new HashSet<>(); + OrderedPartitionableRecord record = recordSupplier.poll(poll_timeout_millis); + + while (record != null) { + supplierRecords.add(record); + record = recordSupplier.poll(poll_timeout_millis); + } + + Assert.assertEquals(8, supplierRecords.size()); + Assert.assertTrue(supplierRecords.containsAll(initialRecords1)); + Assert.assertTrue(supplierRecords.containsAll(initialRecords2)); + + recordSupplier.close(); + + } + + @Test + public void testSeekToLatest() throws InterruptedException, TimeoutException + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); + + StreamPartition shard0 = StreamPartition.of(stream, shardId0); + StreamPartition shard1 = StreamPartition.of(stream, shardId1); + Set> partitions = ImmutableSet.of( + shard0, + shard1 + ); + + KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( + Localstack.getEndpointKinesis(), + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + 1, + 0, + 2, + null, + null, + false, + 100, + 5000, + 5000, + 60000 + ); + + recordSupplier.assign(partitions); + + Assert.assertEquals(insertDataResults.get(0).getSequenceNumber(), recordSupplier.getEarliestSequenceNumber(shard1)); + Assert.assertEquals(insertDataResults.get(8).getSequenceNumber(), recordSupplier.getEarliestSequenceNumber(shard0)); + + recordSupplier.seekToLatest(partitions); + Assert.assertNull(recordSupplier.poll(poll_timeout_millis)); + + recordSupplier.close(); + } + + @Test(expected = ISE.class) + public void testSeekUnassigned() throws InterruptedException, TimeoutException + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); + + StreamPartition shard0 = StreamPartition.of(stream, shardId0); + StreamPartition shard1 = StreamPartition.of(stream, shardId1); + Set> partitions = ImmutableSet.of( + shard1 + ); + + KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( + Localstack.getEndpointKinesis(), + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + 1, + 0, + 2, + null, + null, + false, + 100, + 5000, + 5000, + 60000 + ); + + recordSupplier.assign(partitions); + + Assert.assertEquals(insertDataResults.get(0).getSequenceNumber(), recordSupplier.getEarliestSequenceNumber(shard1)); + + recordSupplier.seekToEarliest(Collections.singleton(shard0)); + + recordSupplier.close(); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java index 1dc4769d5bd4..6d98864274a6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java @@ -20,13 +20,14 @@ package org.apache.druid.indexing.seekablestream.common; import java.util.List; +import java.util.Objects; /** * Represents a generic record with a partitionType (partition id) and sequenceType (sequence number) and data * from a Kafka/Kinesis stream * * @param partition id - * @param sequence number + * @param sequence number */ public class OrderedPartitionableRecord { @@ -37,7 +38,12 @@ public class OrderedPartitionableRecord private final sequenceType sequenceNumber; private final List data; - public OrderedPartitionableRecord(String stream, partitionType partitionId, sequenceType sequenceNumber, List data) + public OrderedPartitionableRecord( + String stream, + partitionType partitionId, + sequenceType sequenceNumber, + List data + ) { this.stream = stream; this.partitionId = partitionId; @@ -69,4 +75,25 @@ public StreamPartition getStreamPartition() { return StreamPartition.of(stream, partitionId); } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + OrderedPartitionableRecord that = (OrderedPartitionableRecord) o; + return Objects.equals(stream, that.stream) && + Objects.equals(partitionId, that.partitionId) && + Objects.equals(sequenceNumber, that.sequenceNumber); + } + + @Override + public int hashCode() + { + return Objects.hash(stream, partitionId, sequenceNumber); + } } From 32c163ad9e1abfaa9c53906b0a767d89758d5150 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Fri, 12 Oct 2018 16:33:12 -0700 Subject: [PATCH 24/87] migrated to aws-java-sdk-kinesis --- .../kinesis-indexing-service/pom.xml | 36 +------- .../kinesis/KinesisRecordSupplier.java | 89 ++++++++++--------- 2 files changed, 50 insertions(+), 75 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index c25c2616e0f9..5192df7ee360 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -39,7 +39,7 @@ 1.6.3 - 1.10.61 + 1.11.42 @@ -76,38 +76,8 @@ com.amazonaws - amazon-kinesis-client - ${kinesis.version} - - - com.fasterxml.jackson.core - jackson-databind - - - com.fasterxml.jackson.core - jackson-annotations - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.dataformat - jackson-dataformat-cbor - - - commons-codec - commons-codec - - - com.amazonaws - aws-java-sdk-cloudwatch - - - com.amazonaws - aws-java-sdk-dynamodb - - + aws-java-sdk-kinesis + ${aws.version} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 3a2196c3b693..83d6eb9207c8 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -25,14 +25,12 @@ import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.services.kinesis.AmazonKinesis; import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder; -import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; -import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxyFactory; -import com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisProxyFactory; -import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; +import com.amazonaws.services.kinesis.model.GetRecordsRequest; import com.amazonaws.services.kinesis.model.GetRecordsResult; import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; import com.amazonaws.services.kinesis.model.Record; import com.amazonaws.services.kinesis.model.ResourceNotFoundException; +import com.amazonaws.services.kinesis.model.Shard; import com.amazonaws.services.kinesis.model.ShardIteratorType; import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder; import com.amazonaws.util.AwsHostNameUtils; @@ -48,7 +46,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Iterator; @@ -74,7 +71,7 @@ public class KinesisRecordSupplier implements RecordSupplier private class PartitionResource { private final StreamPartition streamPartition; - private final IKinesisProxy kinesisProxy; + private final AmazonKinesis kinesisProxy; private final ScheduledExecutorService scheduledExec; private final Object startLock = new Object(); @@ -84,7 +81,7 @@ private class PartitionResource public PartitionResource( StreamPartition streamPartition, - IKinesisProxy kinesisProxy, + AmazonKinesis kinesisProxy, ScheduledExecutorService scheduledExec ) { @@ -156,22 +153,15 @@ private Runnable getRecordRunnable() return; } - GetRecordsResult recordsResult = kinesisProxy.get(shardIterator, recordsPerFetch); + GetRecordsResult recordsResult = kinesisProxy.getRecords(new GetRecordsRequest().withShardIterator( + shardIterator).withLimit(recordsPerFetch)); // list will come back empty if there are no records for (Record kinesisRecord : recordsResult.getRecords()) { final List data; - if (deaggregate) { - data = new ArrayList<>(); - final List userRecords = UserRecord.deaggregate(Collections.singletonList(kinesisRecord)); - for (UserRecord userRecord : userRecords) { - data.add(toByteArray(userRecord.getData())); - } - } else { - data = Collections.singletonList(toByteArray(kinesisRecord.getData())); - } + data = Collections.singletonList(toByteArray(kinesisRecord.getData())); final OrderedPartitionableRecord record = new OrderedPartitionableRecord<>( streamPartition.getStream(), @@ -200,9 +190,12 @@ private Runnable getRecordRunnable() recordBufferFullWait ); - shardIterator = kinesisProxy.getIterator( - record.getPartitionId(), ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), record.getSequenceNumber() - ); + shardIterator = kinesisProxy.getShardIterator( + record.getStream(), + record.getPartitionId(), + ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), + record.getSequenceNumber() + ).getShardIterator(); rescheduleRunnable(recordBufferFullWait); return; @@ -242,10 +235,10 @@ private void rescheduleRunnable(long delayMillis) private final int recordBufferFullWait; private final int fetchSequenceNumberTimeout; - private final IKinesisProxyFactory kinesisProxyFactory; + private final AmazonKinesisClientBuilder kinesisBuilder; private final ScheduledExecutorService scheduledExec; - private final Map kinesisProxies = new ConcurrentHashMap<>(); + private final Map kinesisProxies = new ConcurrentHashMap<>(); private final Map, PartitionResource> partitionResources = new ConcurrentHashMap<>(); private final BlockingQueue> records; @@ -294,16 +287,18 @@ public KinesisRecordSupplier( awsCredentialsProvider = builder.build(); } - AmazonKinesis kinesisClient = AmazonKinesisClientBuilder.standard() - .withCredentials(awsCredentialsProvider) - .withClientConfiguration(new ClientConfiguration()) - .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration( - endpoint, - AwsHostNameUtils.parseRegion(endpoint, null) - )) - .build(); - - kinesisProxyFactory = new KinesisProxyFactory(awsCredentialsProvider, kinesisClient); + kinesisBuilder = AmazonKinesisClientBuilder.standard() + .withCredentials(awsCredentialsProvider) + .withClientConfiguration(new ClientConfiguration()) + .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration( + endpoint, + AwsHostNameUtils.parseRegion( + endpoint, + null + ) + )); + + records = new LinkedBlockingQueue<>(recordBufferSize); log.info( @@ -430,8 +425,11 @@ public String position(StreamPartition partition) public Set getPartitionIds(String stream) { checkIfClosed(); - Set shardList = getKinesisProxy(stream).getAllShardIds(); - return shardList != null ? shardList : ImmutableSet.of(); + return getKinesisProxy(stream).describeStream(stream) + .getStreamDescription() + .getShards() + .stream() + .map(Shard::getShardId).collect(Collectors.toSet()); } @Override @@ -457,10 +455,10 @@ public void close() this.closed = true; } - private IKinesisProxy getKinesisProxy(String streamName) + private AmazonKinesis getKinesisProxy(String streamName) { if (!kinesisProxies.containsKey(streamName)) { - kinesisProxies.put(streamName, kinesisProxyFactory.getProxy(streamName)); + kinesisProxies.put(streamName, kinesisBuilder.build()); } return kinesisProxies.get(streamName); @@ -479,9 +477,12 @@ private void seekInternal(StreamPartition partition, String sequenceNumb sequenceNumber != null ? sequenceNumber : iteratorEnum.toString() ); - resource.shardIterator = getKinesisProxy(partition.getStream()).getIterator( - partition.getPartitionId(), iteratorEnum.toString(), sequenceNumber - ); + resource.shardIterator = getKinesisProxy(partition.getStream()).getShardIterator( + partition.getStream(), + partition.getPartitionId(), + iteratorEnum.toString(), + sequenceNumber + ).getShardIterator(); checkPartitionsStarted = true; } @@ -490,11 +491,15 @@ private String getSequenceNumberInternal(StreamPartition partition, Shar throws TimeoutException { long timeoutMillis = System.currentTimeMillis() + fetchSequenceNumberTimeout; - IKinesisProxy kinesisProxy = getKinesisProxy(partition.getStream()); + AmazonKinesis kinesis = getKinesisProxy(partition.getStream()); String shardIterator = null; try { - shardIterator = kinesisProxy.getIterator(partition.getPartitionId(), iteratorEnum.toString(), null); + shardIterator = kinesis.getShardIterator( + partition.getStream(), + partition.getPartitionId(), + iteratorEnum.toString() + ).getShardIterator(); } catch (ResourceNotFoundException e) { log.warn("Caught ResourceNotFoundException: %s", e.getMessage()); @@ -509,7 +514,7 @@ private String getSequenceNumberInternal(StreamPartition partition, Shar GetRecordsResult recordsResult; try { - recordsResult = kinesisProxy.get(shardIterator, 1); + recordsResult = kinesis.getRecords(new GetRecordsRequest().withShardIterator(shardIterator).withLimit(1)); } catch (ProvisionedThroughputExceededException e) { log.warn("Exceeded provisioned throughput, retrying in [%,dms]", PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS); From 919a4e224e941817de3f9ffc2d709454739b3850 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 15 Oct 2018 16:28:35 -0700 Subject: [PATCH 25/87] merge changes from master --- .../druid/indexing/kafka/KafkaIOConfig.java | 6 +- .../druid/indexing/kafka/KafkaIndexTask.java | 36 ++++++++- .../indexing/kafka/KafkaRecordSupplier.java | 10 ++- .../kafka/supervisor/KafkaSupervisor.java | 6 +- .../supervisor/KafkaSupervisorIOConfig.java | 9 ++- .../indexing/kafka/KafkaIndexTaskTest.java | 4 +- .../kafka/KafkaRecordSupplierTest.java | 16 ++-- .../KafkaSupervisorIOConfigTest.java | 1 + .../druid/indexing/kafka/test/TestBroker.java | 4 +- .../kinesis-indexing-service/pom.xml | 4 +- .../indexing/kinesis/KinesisIndexTask.java | 4 +- .../supervisor/SeekableStreamSupervisor.java | 80 ++++++++++++------- 12 files changed, 122 insertions(+), 58 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java index 6c6ac36da88e..9f31e1ef046f 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java @@ -35,7 +35,7 @@ public class KafkaIOConfig extends SeekableStreamIOConfig private static final boolean DEFAULT_SKIP_OFFSET_GAPS = false; @Nullable - private final Map consumerProperties; + private final Map consumerProperties; private final boolean skipOffsetGaps; @JsonCreator @@ -44,7 +44,7 @@ public KafkaIOConfig( @JsonProperty("baseSequenceName") String baseSequenceName, @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, - @JsonProperty("consumerProperties") Map consumerProperties, + @JsonProperty("consumerProperties") Map consumerProperties, @JsonProperty("useTransaction") Boolean useTransaction, @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, @@ -84,7 +84,7 @@ public Set getExclusiveStartSequenceNumberPartitions() @Nullable @JsonProperty - public Map getConsumerProperties() + public Map getConsumerProperties() { return consumerProperties; } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 827e5b6908d6..b2369c5da650 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import org.apache.druid.data.input.InputRow; @@ -32,6 +33,7 @@ import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; @@ -39,6 +41,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.metadata.PasswordProvider; import org.apache.druid.query.NoopQueryRunner; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; @@ -67,6 +70,7 @@ public class KafkaIndexTask extends SeekableStreamIndexTask static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15; private final SeekableStreamIndexTaskRunner runner; + private final ObjectMapper configMapper; // This value can be tuned in some tests private long pollRetryMs = 30000; @@ -81,7 +85,8 @@ public KafkaIndexTask( @JsonProperty("context") Map context, @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, - @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, + @JacksonInject ObjectMapper configMapper ) { super( @@ -96,6 +101,7 @@ public KafkaIndexTask( rowIngestionMetersFactory, "index_kafka" ); + this.configMapper = configMapper; if (context != null && context.get(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED) != null && ((boolean) context.get(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED))) { runner = new IncrementalPublishingKafkaIndexTaskRunner( @@ -209,9 +215,7 @@ KafkaConsumer newConsumer() final Properties props = new Properties(); - for (Map.Entry entry : ((KafkaIOConfig) ioConfig).getConsumerProperties().entrySet()) { - props.setProperty(entry.getKey(), entry.getValue()); - } + addConsumerPropertiesFromConfig(props, configMapper, ((KafkaIOConfig) ioConfig).getConsumerProperties()); props.setProperty("enable.auto.commit", "false"); props.setProperty("auto.offset.reset", "none"); @@ -225,6 +229,30 @@ KafkaConsumer newConsumer() } } + public static void addConsumerPropertiesFromConfig( + Properties properties, + ObjectMapper configMapper, + Map consumerProperties + ) + { + // Extract passwords before SSL connection to Kafka + for (Map.Entry entry : consumerProperties.entrySet()) { + String propertyKey = entry.getKey(); + if (propertyKey.equals(KafkaSupervisorIOConfig.TRUST_STORE_PASSWORD_KEY) + || propertyKey.equals(KafkaSupervisorIOConfig.KEY_STORE_PASSWORD_KEY) + || propertyKey.equals(KafkaSupervisorIOConfig.KEY_PASSWORD_KEY)) { + PasswordProvider configPasswordProvider = configMapper.convertValue( + entry.getValue(), + PasswordProvider.class + ); + properties.setProperty(propertyKey, configPasswordProvider.getPassword()); + } else { + properties.setProperty(propertyKey, String.valueOf(entry.getValue())); + } + } + } + + static void assignPartitions( final KafkaConsumer consumer, final String topic, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 64daee07ca1a..e06afb2d1859 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.kafka; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; @@ -51,16 +52,19 @@ public class KafkaRecordSupplier implements RecordSupplier private static final Random RANDOM = ThreadLocalRandom.current(); private final KafkaConsumer consumer; - private final Map consumerProperties; + private final Map consumerProperties; + private final ObjectMapper sortingMapper; private boolean closed; private final BlockingQueue> records; public KafkaRecordSupplier( - Map consumerProperties + Map consumerProperties, + ObjectMapper sortingMapper ) { this.consumerProperties = consumerProperties; + this.sortingMapper = sortingMapper; this.consumer = getKafkaConsumer(); this.closed = false; this.records = new LinkedBlockingQueue<>(); @@ -181,7 +185,7 @@ private KafkaConsumer getKafkaConsumer() props.setProperty("group.id", StringUtils.format("kafka-supervisor-%s", getRandomId())); props.setProperty("max.poll.records", "1"); - props.putAll(consumerProperties); + KafkaIndexTask.addConsumerPropertiesFromConfig(props, sortingMapper, consumerProperties); props.setProperty("enable.auto.commit", "false"); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index fb1a73ab4ccc..a4ec03a99e93 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -158,8 +158,7 @@ public void checkpoint( @Override protected RecordSupplier setupRecordSupplier() { - return new KafkaRecordSupplier(spec.getIoConfig().getConsumerProperties()); - + return new KafkaRecordSupplier(spec.getIoConfig().getConsumerProperties(), sortingMapper); } @Override @@ -290,7 +289,8 @@ protected List> createIndexTasks( context, null, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + sortingMapper )); } return taskList; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java index 90ff600e3dfd..da6411e5deee 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java @@ -31,8 +31,11 @@ public class KafkaSupervisorIOConfig extends SeekableStreamSupervisorIOConfig { public static final String BOOTSTRAP_SERVERS_KEY = "bootstrap.servers"; + public static final String TRUST_STORE_PASSWORD_KEY = "ssl.truststore.password"; + public static final String KEY_STORE_PASSWORD_KEY = "ssl.keystore.password"; + public static final String KEY_PASSWORD_KEY = "ssl.key.password"; - private final Map consumerProperties; + private final Map consumerProperties; private final boolean skipOffsetGaps; @JsonCreator @@ -41,7 +44,7 @@ public KafkaSupervisorIOConfig( @JsonProperty("replicas") Integer replicas, @JsonProperty("taskCount") Integer taskCount, @JsonProperty("taskDuration") Period taskDuration, - @JsonProperty("consumerProperties") Map consumerProperties, + @JsonProperty("consumerProperties") Map consumerProperties, @JsonProperty("startDelay") Period startDelay, @JsonProperty("period") Period period, @JsonProperty("useEarliestOffset") Boolean useEarliestOffset, @@ -79,7 +82,7 @@ public String getTopic() } @JsonProperty - public Map getConsumerProperties() + public Map getConsumerProperties() { return consumerProperties; } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index d1dc35d5cea3..2aa8946e1c91 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -420,6 +420,7 @@ public void testRunAfterDataInserted() throws Exception @Test(timeout = 60_000L) public void testRunBeforeDataInserted() throws Exception { + Map consumerProps = kafkaServer.consumerProperties(); final KafkaIndexTask task = createTask( null, new KafkaIOConfig( @@ -2423,7 +2424,8 @@ private List readSegmentColumn(final String column, final SegmentDescrip ); IndexIO indexIO = new TestUtils().getTestIndexIO(); QueryableIndex index = indexIO.loadIndex(outputLocation); - DictionaryEncodedColumn theColumn = (DictionaryEncodedColumn) index.getColumnHolder(column).getColumn(); + DictionaryEncodedColumn theColumn = (DictionaryEncodedColumn) index.getColumnHolder(column) + .getColumn(); List values = Lists.newArrayList(); for (int i = 0; i < theColumn.length(); i++) { int id = theColumn.getSingleValueRow(i); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java index 745fabb82898..0edf45928b5c 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java @@ -30,6 +30,7 @@ import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.TestHelper; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.junit.AfterClass; @@ -53,6 +54,7 @@ public class KafkaRecordSupplierTest private static String topic = "topic"; private static long poll_timeout_millis = 1000; private static int topicPosFix = 0; + private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); private static TestingCluster zkServer; private static TestBroker kafkaServer; @@ -175,7 +177,7 @@ public void testSupplierSetup() throws ExecutionException, InterruptedException ); KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( - kafkaServer.consumerProperties()); + kafkaServer.consumerProperties(), objectMapper); Assert.assertTrue(recordSupplier.getAssignment().isEmpty()); @@ -204,7 +206,7 @@ public void testPoll() throws InterruptedException, ExecutionException ); KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( - kafkaServer.consumerProperties()); + kafkaServer.consumerProperties(), objectMapper); recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); @@ -244,7 +246,7 @@ public void testPollAfterMoreDataAdded() throws InterruptedException, ExecutionE KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( - kafkaServer.consumerProperties()); + kafkaServer.consumerProperties(), objectMapper); recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); @@ -297,7 +299,7 @@ public void testSeek() throws InterruptedException, ExecutionException ); KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( - kafkaServer.consumerProperties()); + kafkaServer.consumerProperties(), objectMapper); recordSupplier.assign(partitions); @@ -345,7 +347,7 @@ public void testSeekToLatest() throws InterruptedException, ExecutionException ); KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( - kafkaServer.consumerProperties()); + kafkaServer.consumerProperties(), objectMapper); recordSupplier.assign(partitions); @@ -376,7 +378,7 @@ public void testSeekUnassigned() throws InterruptedException, ExecutionException ); KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( - kafkaServer.consumerProperties()); + kafkaServer.consumerProperties(), objectMapper); recordSupplier.assign(partitions); @@ -406,7 +408,7 @@ public void testPosition() throws ExecutionException, InterruptedException ); KafkaRecordSupplier recordSupplier = new KafkaRecordSupplier( - kafkaServer.consumerProperties()); + kafkaServer.consumerProperties(), objectMapper); recordSupplier.assign(partitions); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java index e64015adba58..243b8eb3335b 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java @@ -187,4 +187,5 @@ public void testBootstrapServersRequired() throws Exception exception.expectMessage(CoreMatchers.containsString("bootstrap.servers")); mapper.readValue(jsonStr, KafkaSupervisorIOConfig.class); } + } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java index 05c164201d18..c4800aef2353 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java @@ -49,7 +49,7 @@ public class TestBroker implements Closeable private final File directory; private final boolean directoryCleanup; private final int id; - private final Map brokerProps; + private final Map brokerProps; private volatile KafkaServer server; @@ -57,7 +57,7 @@ public TestBroker( String zookeeperConnect, @Nullable File directory, int id, - Map brokerProps + Map brokerProps ) { this.zookeeperConnect = zookeeperConnect; diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index 5192df7ee360..b6db46f226c2 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -32,7 +32,7 @@ org.apache.druid druid - 0.13.0-SNAPSHOT + 0.13.0-incubating-SNAPSHOT ../../pom.xml @@ -45,7 +45,7 @@ org.apache.druid - druid-api + druid-core ${project.parent.version} provided diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 1aaeef919c0b..0be6bdacd707 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -35,8 +35,8 @@ import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.InputRow; import org.apache.druid.discovery.DiscoveryDruidNode; -import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.discovery.LookupNodeService; +import org.apache.druid.discovery.NodeType; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; @@ -250,7 +250,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception final DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( toolbox.getDruidNode(), - DruidNodeDiscoveryProvider.NODE_TYPE_PEON, + NodeType.PEON, ImmutableMap.of( toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), lookupNodeService.getName(), lookupNodeService diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 22a73ca38530..021f97754dbe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -124,7 +124,7 @@ *

* * @param partition id type - * @param sequence number type + * @param sequence number type */ public abstract class SeekableStreamSupervisor implements Supervisor @@ -450,16 +450,17 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) // Map<{group ID}, Map<{partition ID}, {startingOffset}>> private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); + protected final ObjectMapper sortingMapper; protected final List partitionIds = new CopyOnWriteArrayList<>(); protected volatile Map latestSequenceFromStream; protected volatile DateTime sequenceLastUpdated; + private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); private final TaskStorage taskStorage; private final TaskMaster taskMaster; private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; private final SeekableStreamIndexTaskClient taskClient; - private final ObjectMapper sortingMapper; private final SeekableStreamSupervisorSpec spec; private final String dataSource; private final SeekableStreamSupervisorIOConfig ioConfig; @@ -806,7 +807,10 @@ private SupervisorReport payload = createReportPayload(numPartitions, includeOffsets); + final SeekableStreamSupervisorReportPayload payload = createReportPayload( + numPartitions, + includeOffsets + ); SupervisorReport> report = new SupervisorReport<>( dataSource, @@ -1083,13 +1087,13 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) // as well as the case where the metadata store do not have an entry for the reset partitions boolean doReset = false; for (Entry resetPartitionOffset : resetMetadata.getSeekableStreamPartitions() - .getMap() - .entrySet()) { + .getMap() + .entrySet()) { final sequenceType partitionOffsetInMetadataStore = currentMetadata == null - ? null - : currentMetadata.getSeekableStreamPartitions() - .getMap() - .get(resetPartitionOffset.getKey()); + ? null + : currentMetadata.getSeekableStreamPartitions() + .getMap() + .get(resetPartitionOffset.getKey()); final TaskGroup partitionTaskGroup = taskGroups.get( getTaskGroupIdForPartition(resetPartitionOffset.getKey()) ); @@ -1210,10 +1214,10 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti // seamless schema migration. Iterator it = seekableStreamIndexTask.getIOConfig() - .getStartPartitions() - .getMap() - .keySet() - .iterator(); + .getStartPartitions() + .getMap() + .keySet() + .iterator(); final Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null); if (taskGroupId != null) { @@ -1269,9 +1273,9 @@ public Boolean apply(SeekableStreamIndexTask.Status status) } } else { for (partitionType partition : seekableStreamIndexTask.getIOConfig() - .getStartPartitions() - .getMap() - .keySet()) { + .getStartPartitions() + .getMap() + .keySet()) { if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { log.warn( "Stopping task [%s] which does not match the expected partition allocation", @@ -1603,7 +1607,8 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) } @SuppressWarnings("unchecked") - SeekableStreamIndexTask task = (SeekableStreamIndexTask) taskOptional.get(); + SeekableStreamIndexTask task = (SeekableStreamIndexTask) taskOptional + .get(); String taskSequenceName = task.getIOConfig().getBaseSequenceName(); if (taskGroups.get(taskGroupId) != null) { @@ -1814,7 +1819,8 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } } - List> results = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); + List> results = Futures.successfulAsList(futures) + .get(futureTimeoutInSeconds, TimeUnit.SECONDS); for (int j = 0; j < results.size(); j++) { Integer groupId = futureGroupIds.get(j); TaskGroup group = taskGroups.get(groupId); @@ -1848,7 +1854,10 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } } - private ListenableFuture> checkpointTaskGroup(final TaskGroup taskGroup, final boolean finalize) + private ListenableFuture> checkpointTaskGroup( + final TaskGroup taskGroup, + final boolean finalize + ) { if (finalize) { // 1) Check if any task completed (in which case we're done) and kill unassigned tasks @@ -1897,7 +1906,8 @@ public Map apply(@Nullable Object input) } return Futures.transform( - Futures.successfulAsList(pauseFutures), new Function>, Map>() + Futures.successfulAsList(pauseFutures), + new Function>, Map>() { @Nullable @Override @@ -1981,7 +1991,8 @@ public Map apply(List> startingOffsets = generateStartingSequencesForPartitionGroup(groupId); + Map> startingOffsets = generateStartingSequencesForPartitionGroup( + groupId); ImmutableMap simpleStartingOffsets = startingOffsets .entrySet() @@ -2261,7 +2273,9 @@ protected int getNoticesQueueSize() return notices.size(); } - private ImmutableMap> generateStartingSequencesForPartitionGroup(int groupId) + private ImmutableMap> generateStartingSequencesForPartitionGroup( + int groupId + ) throws TimeoutException { ImmutableMap.Builder> builder = ImmutableMap.builder(); @@ -2291,7 +2305,8 @@ private ImmutableMap> generat * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't * find any data, it will retrieve the latest or earliest Kafka/Kinesis offset depending on the useEarliestOffset config. */ - private OrderedSequenceNumber getOffsetFromStorageForPartition(partitionType partition) throws TimeoutException + private OrderedSequenceNumber getOffsetFromStorageForPartition(partitionType partition) + throws TimeoutException { final Map metadataOffsets = getOffsetsFromMetadataStorage(); sequenceType offset = metadataOffsets.get(partition); @@ -2346,7 +2361,8 @@ private Map getOffsetsFromMetadataStorage() if (dataSourceMetadata instanceof SeekableStreamDataSourceMetadata && checkSourceMetadataMatch(dataSourceMetadata)) { @SuppressWarnings("unchecked") - SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata).getSeekableStreamPartitions(); + SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata) + .getSeekableStreamPartitions(); if (partitions != null) { if (!ioConfig.getId().equals(partitions.getName())) { log.warn( @@ -2364,7 +2380,8 @@ && checkSourceMetadataMatch(dataSourceMetadata)) { return Collections.emptyMap(); } - protected sequenceType getOffsetFromStreamForPartition(partitionType partition, boolean useEarliestOffset) throws TimeoutException + protected sequenceType getOffsetFromStreamForPartition(partitionType partition, boolean useEarliestOffset) + throws TimeoutException { synchronized (recordSupplierLock) { StreamPartition topicPartition = new StreamPartition<>(ioConfig.getId(), partition); @@ -2637,7 +2654,11 @@ protected abstract SeekableStreamDataSourceMetadata * * @return specific instance of [Kafka/Kinesis]OrderedSequenceNumber */ - protected abstract OrderedSequenceNumber makeSequenceNumber(sequenceType seq, boolean useExclusive, boolean isExclusive); + protected abstract OrderedSequenceNumber makeSequenceNumber( + sequenceType seq, + boolean useExclusive, + boolean isExclusive + ); /** * schedules periodic emitLag() reporting for Kafka, not yet implemented in Kinesis, @@ -2677,6 +2698,9 @@ protected abstract SeekableStreamSupervisorReportPayload Date: Wed, 17 Oct 2018 16:26:39 -0700 Subject: [PATCH 26/87] fix pom files and forbiddenapi checks --- extensions-core/kinesis-indexing-service/pom.xml | 2 -- .../druid/indexing/kinesis/KinesisIndexTask.java | 11 ++++++----- .../druid/indexing/kinesis/KinesisRecordSupplier.java | 7 ++++--- .../druid/indexing/kinesis/KinesisIndexTaskTest.java | 2 +- .../indexing/kinesis/KinesisRecordSupplierTest.java | 2 +- .../kinesis/supervisor/KinesisSupervisorTest.java | 2 +- 6 files changed, 13 insertions(+), 13 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index b6db46f226c2..86215ddc4638 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -27,8 +27,6 @@ druid-kinesis-indexing-service druid-kinesis-indexing-service - pom - org.apache.druid druid diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 0be6bdacd707..e13802f90b5e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -59,6 +59,7 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -221,7 +222,7 @@ public KinesisIndexTask( public TaskStatus run(final TaskToolbox toolbox) throws Exception { log.info("Starting up!"); - startTime = DateTime.now(); + startTime = DateTimes.nowUtc(); mapper = toolbox.getObjectMapper(); status = Status.STARTING; @@ -321,7 +322,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception // Set up sequenceNames. final Map sequenceNames = Maps.newHashMap(); for (String partitionNum : lastOffsets.keySet()) { - sequenceNames.put(partitionNum, String.format("%s_%s", ioConfig.getBaseSequenceName(), partitionNum)); + sequenceNames.put(partitionNum, StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), partitionNum)); } // Set up committer. @@ -993,7 +994,7 @@ public Response pause(final long timeout) throws InterruptedException { if (!(status == Status.PAUSED || status == Status.READING)) { return Response.status(Response.Status.BAD_REQUEST) - .entity(String.format("Can't pause, task is not in a pausable state (state: [%s])", status)) + .entity(StringUtils.format("Can't pause, task is not in a pausable state (state: [%s])", status)) .build(); } @@ -1199,7 +1200,7 @@ public Response setEndOffsets(Map offsets, final boolean resume) } else if (!endOffsets.keySet().containsAll(offsets.keySet())) { return Response.status(Response.Status.BAD_REQUEST) .entity( - String.format( + StringUtils.format( "Request contains partitions not being handled by this task, my partitions: %s", endOffsets.keySet() ) @@ -1219,7 +1220,7 @@ public Response setEndOffsets(Map offsets, final boolean resume) if (entry.getValue().compareTo(lastOffsets.get(entry.getKey())) < 0) { return Response.status(Response.Status.BAD_REQUEST) .entity( - String.format( + StringUtils.format( "End offset must be >= current offset for partition [%s] (current: %s)", entry.getKey(), lastOffsets.get(entry.getKey()) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 83d6eb9207c8..a5dc6b134c0e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -41,6 +41,7 @@ import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -178,7 +179,7 @@ private Runnable getRecordRunnable() record.getPartitionId(), record.getSequenceNumber(), records.remainingCapacity(), - record.getData().stream().map(String::new).collect(Collectors.toList()) + record.getData().stream().map(StringUtils::fromUtf8).collect(Collectors.toList()) ); } @@ -276,7 +277,7 @@ public KinesisRecordSupplier( log.info("Assuming role [%s] with externalId [%s]", awsAssumedRoleArn, awsExternalId); STSAssumeRoleSessionCredentialsProvider.Builder builder = new STSAssumeRoleSessionCredentialsProvider - .Builder(awsAssumedRoleArn, String.format("druid-kinesis-%s", UUID.randomUUID().toString())) + .Builder(awsAssumedRoleArn, StringUtils.format("druid-kinesis-%s", UUID.randomUUID().toString())) .withStsClient(AWSSecurityTokenServiceClientBuilder.standard() .withCredentials(awsCredentialsProvider) .build()); @@ -544,7 +545,7 @@ private String getSequenceNumberInternal(StreamPartition partition, Shar } throw new TimeoutException( - String.format( + StringUtils.format( "Timeout while retrieving sequence number for partition[%s]", partition.getPartitionId() ) diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 847e90ad965f..22a69167346d 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -229,7 +229,7 @@ public class KinesisIndexTaskTest "1", StringUtils.toUtf8("unparseable2") ), - generateRequestEntry("1", "{}".getBytes()), + generateRequestEntry("1", StringUtils.toUtf8("{}")), generateRequestEntry( "1", JB("2013", "f", "y", "10", "20.0", "1.0") diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index 68386c215943..b9a4ba72e941 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -88,7 +88,7 @@ public class KinesisRecordSupplierTest "1", StringUtils.toUtf8("unparseable2") ), - generateRequestEntry("1", "{}".getBytes()), + generateRequestEntry("1", StringUtils.toUtf8("{}")), generateRequestEntry( "1", JB("2013", "f", "y", "10", "20.0", "1.0") diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index dbc61a776aaa..f1af7bbf6a6b 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -151,7 +151,7 @@ public class KinesisSupervisorTest extends EasyMockSupport "1", StringUtils.toUtf8("unparseable2") ), - generateRequestEntry("1", "{}".getBytes()), + generateRequestEntry("1", StringUtils.toUtf8("{}")), generateRequestEntry( "1", JB("2013", "f", "y", "10", "20.0", "1.0") From d3fae3e6b7024777dea6bb131d02dda08983f07d Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 18 Oct 2018 17:32:14 -0700 Subject: [PATCH 27/87] checkpoint JavaType bug fix --- .../indexing/kafka/KafkaIndexTaskClient.java | 3 +-- .../kinesis/KinesisIndexTaskClient.java | 2 +- .../SeekableStreamIndexTaskClient.java | 23 ++++++++++++++----- 3 files changed, 19 insertions(+), 9 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java index 3733d190c1e5..1162a251968c 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -55,9 +55,8 @@ public KafkaIndexTaskClient( } @Override - protected JavaType constructMapType(Class mapType) + protected JavaType constructPartitionOffsetMapType(Class mapType) { return mapper.getTypeFactory().constructMapType(mapType, Integer.class, Long.class); } - } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java index f73a77aef2de..4812fc1c1c14 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java @@ -54,7 +54,7 @@ public KinesisIndexTaskClient( } @Override - protected JavaType constructMapType(Class mapType) + protected JavaType constructPartitionOffsetMapType(Class mapType) { return mapper.getTypeFactory().constructMapType(mapType, String.class, String.class); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index c54405f2034a..1d18326d68f7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -26,6 +26,7 @@ import org.apache.druid.indexing.common.IndexTaskClient; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -112,7 +113,7 @@ public Map pause(final String id) if (response.getStatus().equals(HttpResponseStatus.OK)) { log.info("Task [%s] paused successfully", id); - return deserialize(response.getContent(), constructMapType(Map.class)); + return deserialize(response.getContent(), constructPartitionOffsetMapType(Map.class)); } while (true) { @@ -140,7 +141,7 @@ public Map pause(final String id) return ImmutableMap.of(); } catch (IOException | InterruptedException e) { - throw new RuntimeException( + throw new RE( StringUtils.format("Exception [%s] while pausing Task [%s]", e.getMessage(), id), e ); @@ -219,7 +220,7 @@ public Map getCurrentOffsets(final String id, final null, retry ); - return deserialize(response.getContent(), constructMapType(Map.class)); + return deserialize(response.getContent(), constructPartitionOffsetMapType(Map.class)); } catch (NoTaskLocationException e) { return ImmutableMap.of(); @@ -236,7 +237,7 @@ public TreeMap> getCheckpoints(final S final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "checkpoints", null, retry); return deserialize( response.getContent(), - constructMapType(TreeMap.class) + constructCheckpointMapType() ); } catch (NoTaskLocationException e) { @@ -261,7 +262,7 @@ public Map getEndOffsets(final String id) try { final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "offsets/end", null, true); - return deserialize(response.getContent(), constructMapType(Map.class)); + return deserialize(response.getContent(), constructPartitionOffsetMapType(Map.class)); } catch (NoTaskLocationException e) { return ImmutableMap.of(); @@ -349,7 +350,17 @@ public ListenableFuture getStatusAsync(final Str return doAsync(() -> getStatus(id)); } - protected abstract JavaType constructMapType(Class mapType); + private JavaType constructCheckpointMapType() + { + ObjectMapper mapper = new ObjectMapper(); + return mapper.getTypeFactory() + .constructMapType( + TreeMap.class, + mapper.getTypeFactory().constructType(Integer.class), + constructPartitionOffsetMapType(TreeMap.class) + ); + } + protected abstract JavaType constructPartitionOffsetMapType(Class mapType); } From 7aebece843c8846a9c513172bf47f3f799b996da Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 18 Oct 2018 18:02:36 -0700 Subject: [PATCH 28/87] fix pom and stuff --- .../seekablestream/supervisor/SeekableStreamSupervisor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 021f97754dbe..1362b8fe81d1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1357,7 +1357,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) killTask(taskId); } } - log.debug("Found [%d] seekable stream indexing tasks for dataSource [%s]", taskCount, dataSource); + log.debug("Found [%d] seekablestream indexing tasks for dataSource [%s]", taskCount, dataSource); // make sure the checkpoints are consistent with each other and with the metadata store verifyAndMergeCheckpoints(taskGroupsToVerify.values()); From d0c23a55be7d1c4f93982120fee0b9f6f0c3814c Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 23 Oct 2018 14:26:19 -0700 Subject: [PATCH 29/87] disable checkpointing in kinesis --- .../kafka/supervisor/KafkaSupervisor.java | 3 +- .../kinesis/supervisor/KinesisSupervisor.java | 5 +-- .../supervisor/KinesisSupervisorTest.java | 34 +++++++++++++++++++ .../supervisor/SeekableStreamSupervisor.java | 23 ++++++++----- 4 files changed, 54 insertions(+), 11 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index a4ec03a99e93..2e91cebb8592 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -114,7 +114,8 @@ public KafkaSupervisor( rowIngestionMetersFactory, NOT_SET, Long.MAX_VALUE, - false + false, + true ); this.spec = spec; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 54fa02e7f1ca..34931978fa22 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -43,8 +43,8 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; -import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; @@ -98,7 +98,8 @@ public KinesisSupervisor( rowIngestionMetersFactory, NOT_SET, SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - true + true, + false ); this.spec = spec; diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index f1af7bbf6a6b..18c6a01e6640 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -731,9 +731,11 @@ public void testKillIncompatibleTasks() throws Exception getSequenceNumber(res, shardId1, 0) )); + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(2); + */ replayAll(); supervisor.start(); @@ -864,12 +866,15 @@ public void testKillBadPartitionAssignment() throws Exception checkpoints1.put(0, ImmutableMap.of(shardId1, getSequenceNumber(res, shardId1, 0))); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(1); + */ + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); taskQueue.shutdown("id4"); @@ -919,12 +924,14 @@ public void testRequeueTaskWhenFailed() throws Exception shardId0, getSequenceNumber(res, shardId0, 0) )); + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .anyTimes(); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .anyTimes(); + */ taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1035,9 +1042,11 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception shardId0, getSequenceNumber(res, shardId0, 0) )); + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(2); + */ taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1058,12 +1067,15 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception EasyMock.reset(taskClient); // for the newly created replica task + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); + */ + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes(); EasyMock.expect(taskStorage.getStatus(iHaveFailed.getId())) @@ -1158,12 +1170,15 @@ public void testQueueNextTasksOnSuccess() throws Exception getSequenceNumber(res, shardId1, 0) )); // there would be 4 tasks, 2 for each task group + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); + */ + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { @@ -1300,12 +1315,14 @@ public void testBeginPublishAndQueueNextTasks() throws Exception shardId0, getSequenceNumber(res, shardId0, 0) )); + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); + */ EasyMock.replay(taskStorage, taskRunner, taskClient, taskQueue); @@ -1713,9 +1730,11 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception shardId0, getSequenceNumber(res, shardId0, 1) )); + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); + */ replayAll(); @@ -1811,12 +1830,15 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception )); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); + */ + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { @@ -1878,12 +1900,14 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception )); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); + */ captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); @@ -1968,12 +1992,14 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception )); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); + */ captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); @@ -2165,12 +2191,14 @@ public void testStopGracefully() throws Exception shardId0, getSequenceNumber(res, shardId0, 1) )); + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); + */ taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2455,12 +2483,14 @@ public void testResetRunningTasks() throws Exception getSequenceNumber(res, shardId0, 1) )); + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); + */ taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2590,6 +2620,7 @@ public void testNoDataIngestionTasks() throws Exception getSequenceNumber(res, shardId0, 1) )); + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); @@ -2599,6 +2630,7 @@ public void testNoDataIngestionTasks() throws Exception EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); + */ taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2759,12 +2791,14 @@ public void testSuspendedRunningTasks() throws Exception shardId0, getSequenceNumber(res, shardId0, 1) )); + /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); + */ taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 1362b8fe81d1..ba25eb37c94f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -481,6 +481,7 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) private final sequenceType END_OF_PARTITION; private final sequenceType NOT_SET; private final boolean useExclusiveStartingSequence; + private final boolean isCheckpointSupported; private boolean listenerRegistered = false; private long lastRunTime; private int initRetryCounter = 0; @@ -502,7 +503,8 @@ public SeekableStreamSupervisor( final RowIngestionMetersFactory rowIngestionMetersFactory, final sequenceType NOT_SET, final sequenceType END_OF_PARTITION, - final boolean useExclusiveStartingSequence + final boolean useExclusiveStartingSequence, + final boolean isCheckpointSupported ) { this.taskStorage = taskStorage; @@ -514,6 +516,7 @@ public SeekableStreamSupervisor( this.NOT_SET = NOT_SET; this.END_OF_PARTITION = END_OF_PARTITION; this.useExclusiveStartingSequence = useExclusiveStartingSequence; + this.isCheckpointSupported = isCheckpointSupported; this.dataSource = spec.getDataSchema().getDataSource(); this.ioConfig = spec.getIoConfig(); @@ -1360,7 +1363,9 @@ public Boolean apply(SeekableStreamIndexTask.Status status) log.debug("Found [%d] seekablestream indexing tasks for dataSource [%s]", taskCount, dataSource); // make sure the checkpoints are consistent with each other and with the metadata store - verifyAndMergeCheckpoints(taskGroupsToVerify.values()); + if (isCheckpointSupported) { + verifyAndMergeCheckpoints(taskGroupsToVerify.values()); + } } @@ -2156,12 +2161,14 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep private void createNewTasks() throws JsonProcessingException { // update the checkpoints in the taskGroup to latest ones so that new tasks do not read what is already published - verifyAndMergeCheckpoints( - taskGroups.values() - .stream() - .filter(taskGroup -> taskGroup.tasks.size() < ioConfig.getReplicas()) - .collect(Collectors.toList()) - ); + if (isCheckpointSupported) { + verifyAndMergeCheckpoints( + taskGroups.values() + .stream() + .filter(taskGroup -> taskGroup.tasks.size() < ioConfig.getReplicas()) + .collect(Collectors.toList()) + ); + } // check that there is a current task group for each group of partitions in [partitionGroups] for (Integer groupId : partitionGroups.keySet()) { From 2bab3c346ab0bb298f1a13068d2afb43dd1a9dcf Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 24 Oct 2018 18:07:54 -0700 Subject: [PATCH 30/87] fix kinesis sequence number null in closed shard --- .../apache/druid/indexing/kinesis/KinesisRecordSupplier.java | 4 ++-- .../apache/druid/indexing/kinesis/KinesisSequenceNumber.java | 5 ++++- .../seekablestream/supervisor/SeekableStreamSupervisor.java | 2 +- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index a5dc6b134c0e..2030cf5f3424 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -540,8 +540,8 @@ private String getSequenceNumberInternal(StreamPartition partition, Shar } if (shardIterator == null) { - log.info("Partition[%s] returned a null shard iterator", partition.getPartitionId()); - return null; + log.info("Partition[%s] returned a null shard iterator, is the shard closed?", partition.getPartitionId()); + return OrderedPartitionableRecord.END_OF_SHARD_MARKER; } throw new TimeoutException( diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java index 3d27dae42c4d..d0cb952efa3a 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.kinesis; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import javax.validation.constraints.NotNull; @@ -34,7 +35,9 @@ public class KinesisSequenceNumber extends OrderedSequenceNumber private KinesisSequenceNumber(@NotNull String sequenceNumber, boolean useExclusive, boolean isExclusive) { super(sequenceNumber, useExclusive, isExclusive); - this.intSequence = "".equals(sequenceNumber) ? new BigInteger("-1") : new BigInteger(sequenceNumber); + this.intSequence = OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(sequenceNumber) + ? new BigInteger("-1") + : new BigInteger(sequenceNumber); } public static KinesisSequenceNumber of(String sequenceNumber) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index ba25eb37c94f..aa97571a03f1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -2300,7 +2300,7 @@ private ImmutableMap> generat // get the offset from metadata storage (if available) or Kafka/Kinesis (otherwise) OrderedSequenceNumber offsetFromStorage = getOffsetFromStorageForPartition(partition); - if (!OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(offset)) { + if (offsetFromStorage != null) { builder.put(partition, offsetFromStorage); } } From 73986ab3b5e63489b23f021dff12d79209bb38c7 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 24 Oct 2018 18:08:42 -0700 Subject: [PATCH 31/87] merge changes from master --- distribution/pom.xml | 530 ++++---- .../kafka/KafkaDataSourceMetadata.java | 3 +- .../indexing/kafka/KafkaRecordSupplier.java | 9 +- .../kafka/supervisor/KafkaSupervisor.java | 53 +- .../kafka/supervisor/KafkaSupervisorTest.java | 1147 +++++++++-------- .../kinesis-indexing-service/pom.xml | 2 +- .../kinesis/KinesisRecordSupplier.java | 9 +- .../kinesis/supervisor/KinesisSupervisor.java | 52 +- .../kinesis/KinesisIndexTaskTest.java | 7 +- .../supervisor/KinesisSupervisorTest.java | 48 + .../SeekableStreamIndexTaskClient.java | 6 +- .../supervisor/SeekableStreamSupervisor.java | 85 +- 12 files changed, 1078 insertions(+), 873 deletions(-) diff --git a/distribution/pom.xml b/distribution/pom.xml index 5e1106b8983a..fcd191b65d48 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -31,275 +31,275 @@ druid org.apache.druid - 0.13.0-SNAPSHOT + 0.13.0-incubating-SNAPSHOT - - - org.apache.druid - druid-services - ${project.parent.version} - - - - - --clean - + + + org.apache.druid + druid-services + ${project.parent.version} + + + + + --clean + - + + + + net.nicoulaj.maven.plugins + checksum-maven-plugin + 1.7 + + + dist-checksum + + files + + + + + + SHA-512 + + false + + + ${project.build.directory} + + *-src.tar.gz + *-bin.tar.gz + + + + false + + + + + + + + dist + + false + + tar + + + + + + org.codehaus.mojo + exec-maven-plugin + + + pull-deps + package + + exec + + + java + + -classpath + + -Ddruid.extensions.loadList=[] + -Ddruid.extensions.directory=${project.build.directory}/extensions + + + -Ddruid.extensions.hadoopDependenciesDir=${project.build.directory}/hadoop-dependencies + + org.apache.druid.cli.Main + tools + pull-deps + --clean + --defaultVersion + ${project.parent.version} + -l + ${settings.localRepository} + -h + org.apache.hadoop:hadoop-client:${hadoop.compile.version} + -c + org.apache.druid.extensions:druid-avro-extensions + -c + org.apache.druid.extensions:druid-bloom-filter + -c + org.apache.druid.extensions:druid-datasketches + -c + org.apache.druid.extensions:druid-hdfs-storage + -c + org.apache.druid.extensions:druid-histogram + -c + org.apache.druid.extensions:druid-kafka-eight + -c + org.apache.druid.extensions:druid-kafka-extraction-namespace + -c + org.apache.druid.extensions:druid-kafka-indexing-service + -c + org.apache.druid.extensions:druid-kinesis-indexing-service + -c + org.apache.druid.extensions:druid-lookups-cached-global + -c + org.apache.druid.extensions:druid-lookups-cached-single + -c + org.apache.druid.extensions:druid-protobuf-extensions + -c + org.apache.druid.extensions:mysql-metadata-storage + -c + org.apache.druid.extensions:postgresql-metadata-storage + -c + org.apache.druid.extensions:druid-kerberos + -c + org.apache.druid.extensions:druid-s3-extensions + -c + org.apache.druid.extensions:druid-stats + -c + org.apache.druid.extensions:druid-examples + -c + org.apache.druid.extensions:simple-client-sslcontext + -c + org.apache.druid.extensions:druid-basic-security + ${druid.distribution.pulldeps.opts} + + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + distro-assembly + package + + single + + + apache-druid-${project.parent.version} + posix + + src/assembly/assembly.xml + + + + + source-release-assembly-druid + package + + single + + + apache-druid-${project.version}-src + posix + + src/assembly/source-assembly.xml + + false + + + + + + org.codehaus.mojo + license-maven-plugin + + + download-licenses + + download-licenses + + + + + + + + + bundle-contrib-exts + - - net.nicoulaj.maven.plugins - checksum-maven-plugin - 1.7 - - - dist-checksum - - files - - - + + org.codehaus.mojo + exec-maven-plugin + + + pull-deps-contrib-exts + package + + exec + - - SHA-512 - - false - - - ${project.build.directory} - - *-src.tar.gz - *-bin.tar.gz - - - - false + java + + -classpath + + -Ddruid.extensions.loadList=[] + -Ddruid.extensions.directory=${project.build.directory}/extensions + + + -Ddruid.extensions.hadoopDependenciesDir=${project.build.directory}/hadoop-dependencies + + org.apache.druid.cli.Main + tools + pull-deps + --defaultVersion + ${project.parent.version} + -l + ${settings.localRepository} + --no-default-hadoop + -c + org.apache.druid.extensions.contrib:ambari-metrics-emitter + -c + org.apache.druid.extensions.contrib:druid-azure-extensions + -c + org.apache.druid.extensions.contrib:druid-cassandra-storage + -c + org.apache.druid.extensions.contrib:druid-cloudfiles-extensions + -c + org.apache.druid.extensions.contrib:druid-distinctcount + -c + org.apache.druid.extensions.contrib:druid-rocketmq + -c + org.apache.druid.extensions.contrib:druid-google-extensions + -c + org.apache.druid.extensions.contrib:druid-kafka-eight-simple-consumer + -c + org.apache.druid.extensions.contrib:graphite-emitter + -c + org.apache.druid.extensions.contrib:druid-opentsdb-emitter + -c + org.apache.druid.extensions.contrib:druid-orc-extensions + -c + org.apache.druid.extensions.contrib:druid-parquet-extensions + -c + org.apache.druid.extensions.contrib:druid-rabbitmq + -c + org.apache.druid.extensions.contrib:druid-redis-cache + -c + org.apache.druid.extensions.contrib:sqlserver-metadata-storage + -c + org.apache.druid.extensions.contrib:statsd-emitter + -c + org.apache.druid.extensions.contrib:druid-thrift-extensions + -c + org.apache.druid.extensions.contrib:druid-time-min-max + -c + org.apache.druid.extensions.contrib:druid-virtual-columns + -c + org.apache.druid.extensions.contrib:materialized-view-maintenance + -c + org.apache.druid.extensions.contrib:materialized-view-selection + - + + + - - - - - dist - - false - - tar - - - - - - org.codehaus.mojo - exec-maven-plugin - - - pull-deps - package - - exec - - - java - - -classpath - - -Ddruid.extensions.loadList=[] - -Ddruid.extensions.directory=${project.build.directory}/extensions - - - -Ddruid.extensions.hadoopDependenciesDir=${project.build.directory}/hadoop-dependencies - - org.apache.druid.cli.Main - tools - pull-deps - --clean - --defaultVersion - ${project.parent.version} - -l - ${settings.localRepository} - -h - org.apache.hadoop:hadoop-client:${hadoop.compile.version} - -c - org.apache.druid.extensions:druid-avro-extensions - -c - org.apache.druid.extensions:druid-bloom-filter - -c - org.apache.druid.extensions:druid-datasketches - -c - org.apache.druid.extensions:druid-hdfs-storage - -c - org.apache.druid.extensions:druid-histogram - -c - org.apache.druid.extensions:druid-kafka-eight - -c - org.apache.druid.extensions:druid-kafka-extraction-namespace - -c - org.apache.druid.extensions:druid-kafka-indexing-service - -c - org.apache.druid.extensions:druid-kinesis-indexing-service - -c - org.apache.druid.extensions:druid-lookups-cached-global - -c - org.apache.druid.extensions:druid-lookups-cached-single - -c - org.apache.druid.extensions:druid-protobuf-extensions - -c - org.apache.druid.extensions:mysql-metadata-storage - -c - org.apache.druid.extensions:postgresql-metadata-storage - -c - org.apache.druid.extensions:druid-kerberos - -c - org.apache.druid.extensions:druid-s3-extensions - -c - org.apache.druid.extensions:druid-stats - -c - org.apache.druid.extensions:druid-examples - -c - org.apache.druid.extensions:simple-client-sslcontext - -c - org.apache.druid.extensions:druid-basic-security - ${druid.distribution.pulldeps.opts} - - - - - - - org.apache.maven.plugins - maven-assembly-plugin - - - distro-assembly - package - - single - - - apache-druid-${project.parent.version} - posix - - src/assembly/assembly.xml - - - - - source-release-assembly-druid - package - - single - - - apache-druid-${project.version}-src - posix - - src/assembly/source-assembly.xml - - false - - - - - - org.codehaus.mojo - license-maven-plugin - - - download-licenses - - download-licenses - - - - - - - - - bundle-contrib-exts - - - - org.codehaus.mojo - exec-maven-plugin - - - pull-deps-contrib-exts - package - - exec - - - java - - -classpath - - -Ddruid.extensions.loadList=[] - -Ddruid.extensions.directory=${project.build.directory}/extensions - - - -Ddruid.extensions.hadoopDependenciesDir=${project.build.directory}/hadoop-dependencies - - org.apache.druid.cli.Main - tools - pull-deps - --defaultVersion - ${project.parent.version} - -l - ${settings.localRepository} - --no-default-hadoop - -c - org.apache.druid.extensions.contrib:ambari-metrics-emitter - -c - org.apache.druid.extensions.contrib:druid-azure-extensions - -c - org.apache.druid.extensions.contrib:druid-cassandra-storage - -c - org.apache.druid.extensions.contrib:druid-cloudfiles-extensions - -c - org.apache.druid.extensions.contrib:druid-distinctcount - -c - org.apache.druid.extensions.contrib:druid-rocketmq - -c - org.apache.druid.extensions.contrib:druid-google-extensions - -c - org.apache.druid.extensions.contrib:druid-kafka-eight-simple-consumer - -c - org.apache.druid.extensions.contrib:graphite-emitter - -c - org.apache.druid.extensions.contrib:druid-opentsdb-emitter - -c - org.apache.druid.extensions.contrib:druid-orc-extensions - -c - org.apache.druid.extensions.contrib:druid-parquet-extensions - -c - org.apache.druid.extensions.contrib:druid-rabbitmq - -c - org.apache.druid.extensions.contrib:druid-redis-cache - -c - org.apache.druid.extensions.contrib:sqlserver-metadata-storage - -c - org.apache.druid.extensions.contrib:statsd-emitter - -c - org.apache.druid.extensions.contrib:druid-thrift-extensions - -c - org.apache.druid.extensions.contrib:druid-time-min-max - -c - org.apache.druid.extensions.contrib:druid-virtual-columns - -c - org.apache.druid.extensions.contrib:materialized-view-maintenance - -c - org.apache.druid.extensions.contrib:materialized-view-selection - - - - - - - - - - + + + + \ No newline at end of file diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java index cfb3908ad9a1..a37941f4f62b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java @@ -45,7 +45,8 @@ public SeekableStreamPartitions getKafkaPartitions() @Override protected SeekableStreamDataSourceMetadata createConcretDataSourceMetaData( - String streamId, Map newMap + String streamId, + Map newMap ) { return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(streamId, newMap)); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index e06afb2d1859..82efd2a956c1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; @@ -158,13 +157,11 @@ public Long position(StreamPartition partition) @Override public Set getPartitionIds(String stream) { - final Map> topics = consumer.listTopics(); - if (!topics.containsKey(stream)) { + List partitions = consumer.partitionsFor(stream); + if (partitions == null) { throw new ISE("Topic [%s] is not found in KafkaConsumer's list of topics", stream); } - return topics == null - ? ImmutableSet.of() - : topics.get(stream).stream().map(PartitionInfo::partition).collect(Collectors.toSet()); + return partitions.stream().map(PartitionInfo::partition).collect(Collectors.toSet()); } @Override diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 2e91cebb8592..adde2bc85d4a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; +import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -318,7 +319,8 @@ protected Map getLagPerPartition(Map currentOffset @Override protected KafkaDataSourceMetadata createDataSourceMetaData( - String topic, Map map + String topic, + Map map ) { return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, map)); @@ -326,7 +328,9 @@ protected KafkaDataSourceMetadata createDataSourceMetaData( @Override protected OrderedSequenceNumber makeSequenceNumber( - Long seq, boolean useExclusive, boolean isExclusive + Long seq, + boolean useExclusive, + boolean isExclusive ) { return KafkaSequenceNumber.of(seq); @@ -370,7 +374,8 @@ private Runnable emitLag() @Override protected boolean checkSequenceAvailability( - @NotNull Integer partition, @NotNull Long sequenceFromMetadata + @NotNull Integer partition, + @NotNull Long sequenceFromMetadata ) throws TimeoutException { Long latestOffset = getOffsetFromStreamForPartition(partition, false); @@ -436,4 +441,46 @@ protected void tryInit() { super.tryInit(); } + + @Override + @VisibleForTesting + protected void addTaskGroupToActivelyReadingTaskGroup( + int taskGroupId, + ImmutableMap partitionOffsets, + Optional minMsgTime, + Optional maxMsgTime, + Set tasks, + Set exclusiveStartingSequencePartitions + ) + { + super.addTaskGroupToActivelyReadingTaskGroup( + taskGroupId, + partitionOffsets, + minMsgTime, + maxMsgTime, + tasks, + exclusiveStartingSequencePartitions + ); + } + + @Override + @VisibleForTesting + protected void addTaskGroupToPendingCompletionTaskGroup( + int taskGroupId, + ImmutableMap partitionOffsets, + Optional minMsgTime, + Optional maxMsgTime, + Set tasks, + Set exclusiveStartingSequencePartitions + ) + { + super.addTaskGroupToPendingCompletionTaskGroup( + taskGroupId, + partitionOffsets, + minMsgTime, + maxMsgTime, + tasks, + exclusiveStartingSequencePartitions + ); + } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 10030633b176..bc7cc612aa19 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -114,6 +114,8 @@ import java.util.concurrent.Executor; import java.util.concurrent.TimeoutException; +import static org.easymock.EasyMock.expect; + @RunWith(Parameterized.class) public class KafkaSupervisorTest extends EasyMockSupport { @@ -133,6 +135,7 @@ public class KafkaSupervisorTest extends EasyMockSupport private static int topicPostfix; private static ZkUtils zkUtils; + private final int numThreads; private KafkaSupervisor supervisor; @@ -256,15 +259,15 @@ public void testNoInitialState() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -303,15 +306,15 @@ public void testSkipOffsetGaps() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -332,15 +335,15 @@ public void testMultiTask() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); replayAll(); supervisor.start(); @@ -369,15 +372,15 @@ public void testReplicas() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); replayAll(); supervisor.start(); @@ -406,15 +409,15 @@ public void testLateMessageRejectionPeriod() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); replayAll(); supervisor.start(); @@ -445,15 +448,15 @@ public void testEarlyMessageRejectionPeriod() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); replayAll(); supervisor.start(); @@ -487,15 +490,15 @@ public void testLatestOffset() throws Exception addSomeEvents(1100); Capture captured = Capture.newInstance(); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); replayAll(); supervisor.start(); @@ -519,15 +522,15 @@ public void testDatasourceMetadata() throws Exception addSomeEvents(100); Capture captured = Capture.newInstance(); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); replayAll(); supervisor.start(); @@ -548,9 +551,9 @@ public void testBadMetadataOffsets() throws Exception supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); addSomeEvents(1); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)) ) @@ -625,39 +628,39 @@ public void testKillIncompatibleTasks() throws Exception List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) - .anyTimes(); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); - EasyMock.expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(false)); + expect(taskClient.stopAsync("id1", false)).andReturn(Futures.immediateFuture(true)); + expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(false)); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); taskQueue.shutdown("id3"); - EasyMock.expect(taskQueue.add(EasyMock.anyObject(Task.class))).andReturn(true); + expect(taskQueue.add(EasyMock.anyObject(Task.class))).andReturn(true); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(2); replayAll(); @@ -723,45 +726,45 @@ public void testKillBadPartitionAssignment() throws Exception List existingTasks = ImmutableList.of(id1, id2, id3, id4, id5); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) - .anyTimes(); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(existingTasks).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.getStatus("id4")).andReturn(Optional.of(TaskStatus.running("id4"))).anyTimes(); + expect(taskStorage.getStatus("id5")).andReturn(Optional.of(TaskStatus.running("id5"))).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(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); + expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); + expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); - EasyMock.expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); - EasyMock.expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); + expect(taskClient.stopAsync("id3", false)).andReturn(Futures.immediateFuture(true)); + expect(taskClient.stopAsync("id4", false)).andReturn(Futures.immediateFuture(false)); + expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(1); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(1); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); taskQueue.shutdown("id4"); @@ -780,33 +783,33 @@ public void testRequeueTaskWhenFailed() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) - .anyTimes(); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .anyTimes(); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .anyTimes(); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .anyTimes(); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -820,12 +823,12 @@ public void testRequeueTaskWhenFailed() throws Exception // test that running the main loop again checks the status of the tasks that were created and does nothing if they // are all still running EasyMock.reset(taskStorage); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - EasyMock.expect(taskStorage.getStatus(task.getId())) - .andReturn(Optional.of(TaskStatus.running(task.getId()))) - .anyTimes(); - EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } EasyMock.replay(taskStorage); @@ -838,17 +841,17 @@ public void testRequeueTaskWhenFailed() throws Exception KafkaIndexTask iHaveFailed = (KafkaIndexTask) tasks.get(3); EasyMock.reset(taskStorage); EasyMock.reset(taskQueue); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(imStillAlive).anyTimes(); for (Task task : imStillAlive) { - EasyMock.expect(taskStorage.getStatus(task.getId())) - .andReturn(Optional.of(TaskStatus.running(task.getId()))) - .anyTimes(); - EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - EasyMock.expect(taskStorage.getStatus(iHaveFailed.getId())) - .andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); - EasyMock.expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(aNewTaskCapture))).andReturn(true); + expect(taskStorage.getStatus(iHaveFailed.getId())) + .andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); + expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); + expect(taskQueue.add(EasyMock.capture(aNewTaskCapture))).andReturn(true); EasyMock.replay(taskStorage); EasyMock.replay(taskQueue); @@ -883,16 +886,16 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception List existingTasks = ImmutableList.of(id1); Capture captured = Capture.newInstance(); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); - EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(existingTasks).anyTimes(); + expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes(); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) @@ -900,9 +903,9 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L, 2, 0L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(2); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -923,24 +926,24 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception EasyMock.reset(taskClient); // for the newly created replica task - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(2); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); - - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes(); - EasyMock.expect(taskStorage.getStatus(iHaveFailed.getId())) - .andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); - EasyMock.expect(taskStorage.getStatus(runningTaskId)) - .andReturn(Optional.of(TaskStatus.running(runningTaskId))) - .anyTimes(); - EasyMock.expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); - EasyMock.expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(Status.READING)); - EasyMock.expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(aNewTaskCapture))).andReturn(true); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes(); + expect(taskStorage.getStatus(iHaveFailed.getId())) + .andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId()))); + expect(taskStorage.getStatus(runningTaskId)) + .andReturn(Optional.of(TaskStatus.running(runningTaskId))) + .anyTimes(); + expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); + expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); + expect(taskClient.getStatusAsync(runningTaskId)).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes(); + expect(taskQueue.add(EasyMock.capture(aNewTaskCapture))).andReturn(true); EasyMock.replay(taskStorage); EasyMock.replay(taskQueue); EasyMock.replay(taskClient); @@ -970,22 +973,22 @@ public void testQueueNextTasksOnSuccess() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) - .anyTimes(); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -999,30 +1002,30 @@ public void testQueueNextTasksOnSuccess() throws Exception EasyMock.reset(taskStorage); EasyMock.reset(taskClient); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) - .anyTimes(); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .anyTimes(); + expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); // there would be 4 tasks, 2 for each task group - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); - - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); + + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - EasyMock.expect(taskStorage.getStatus(task.getId())) - .andReturn(Optional.of(TaskStatus.running(task.getId()))) - .anyTimes(); - EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } EasyMock.replay(taskStorage); EasyMock.replay(taskClient); @@ -1039,19 +1042,19 @@ public void testQueueNextTasksOnSuccess() throws Exception EasyMock.reset(taskStorage); EasyMock.reset(taskQueue); EasyMock.reset(taskClient); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(imStillRunning).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(imStillRunning).anyTimes(); for (Task task : imStillRunning) { - EasyMock.expect(taskStorage.getStatus(task.getId())) - .andReturn(Optional.of(TaskStatus.running(task.getId()))) - .anyTimes(); - EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - EasyMock.expect(taskStorage.getStatus(iAmSuccess.getId())) - .andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId()))); - EasyMock.expect(taskStorage.getTask(iAmSuccess.getId())).andReturn(Optional.of((Task) iAmSuccess)).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(newTasksCapture))).andReturn(true).times(2); - EasyMock.expect(taskClient.stopAsync(EasyMock.capture(shutdownTaskIdCapture), EasyMock.eq(false))) - .andReturn(Futures.immediateFuture(true)); + expect(taskStorage.getStatus(iAmSuccess.getId())) + .andReturn(Optional.of(TaskStatus.success(iAmSuccess.getId()))); + expect(taskStorage.getTask(iAmSuccess.getId())).andReturn(Optional.of((Task) iAmSuccess)).anyTimes(); + expect(taskQueue.add(EasyMock.capture(newTasksCapture))).andReturn(true).times(2); + expect(taskClient.stopAsync(EasyMock.capture(shutdownTaskIdCapture), EasyMock.eq(false))) + .andReturn(Futures.immediateFuture(true)); EasyMock.replay(taskStorage); EasyMock.replay(taskQueue); EasyMock.replay(taskClient); @@ -1072,16 +1075,16 @@ public void testBeginPublishAndQueueNextTasks() throws Exception addSomeEvents(100); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1097,45 +1100,45 @@ public void testBeginPublishAndQueueNextTasks() throws Exception EasyMock.reset(taskStorage, taskRunner, taskClient, taskQueue); captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - EasyMock.expect(taskStorage.getStatus(task.getId())) - .andReturn(Optional.of(TaskStatus.running(task.getId()))) - .anyTimes(); - EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.READING)) - .anyTimes(); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .times(2); - EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); - EasyMock.expect( + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.READING)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .times(2); + expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); + expect( taskClient.setEndOffsetsAsync( EasyMock.contains("sequenceName-0"), EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), EasyMock.eq(true) ) ).andReturn(Futures.immediateFuture(true)).times(2); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); EasyMock.replay(taskStorage, taskRunner, taskClient, taskQueue); @@ -1183,28 +1186,28 @@ public void testDiscoverExistingPublishingTask() throws Exception workItems.add(new TestTaskRunnerWorkItem(task, null, location)); Capture captured = Capture.newInstance(); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); + expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); - EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); - EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); + expect(taskClient.getCurrentOffsetsAsync("id1", false)) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)); - EasyMock.expect(taskClient.getCheckpoints(EasyMock.anyString(), EasyMock.anyBoolean())) - .andReturn(checkpoints) - .anyTimes(); + expect(taskClient.getCheckpoints(EasyMock.anyString(), EasyMock.anyBoolean())) + .andReturn(checkpoints) + .anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1277,22 +1280,22 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() workItems.add(new TestTaskRunnerWorkItem(task, null, location)); Capture captured = Capture.newInstance(); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(task)).anyTimes(); + expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); + expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); - EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 2, 30L))); - EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); + expect(taskClient.getCurrentOffsetsAsync("id1", false)) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 2, 30L))); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 2, 30L)); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1383,36 +1386,36 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2)).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2)).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.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); + expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); - EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); - EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 1L, 1, 2L, 2, 3L))); - EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); - EasyMock.expect(taskClient.getCurrentOffsetsAsync("id2", false)) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 4L, 1, 5L, 2, 6L))); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getCurrentOffsetsAsync("id1", false)) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 1L, 1, 2L, 2, 3L))); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); + expect(taskClient.getCurrentOffsetsAsync("id2", false)) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 4L, 1, 5L, 2, 6L))); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); // since id1 is publishing, so getCheckpoints wouldn't be called for it TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); replayAll(); @@ -1461,16 +1464,16 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception addSomeEvents(1); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1486,23 +1489,23 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); - - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); + + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - EasyMock.expect(taskStorage.getStatus(task.getId())) - .andReturn(Optional.of(TaskStatus.running(task.getId()))) - .anyTimes(); - EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(task.getId())) - .andReturn(Futures.immediateFuture(Status.NOT_STARTED)); - EasyMock.expect(taskClient.getStartTimeAsync(task.getId())) - .andReturn(Futures.immediateFailedFuture(new RuntimeException())); + expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + expect(taskClient.getStatusAsync(task.getId())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)); + expect(taskClient.getStartTimeAsync(task.getId())) + .andReturn(Futures.immediateFailedFuture(new RuntimeException())); taskQueue.shutdown(task.getId()); } EasyMock.replay(taskStorage, taskClient, taskQueue); @@ -1520,16 +1523,16 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception addSomeEvents(100); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1549,36 +1552,36 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - EasyMock.expect(taskStorage.getStatus(task.getId())) - .andReturn(Optional.of(TaskStatus.running(task.getId()))) - .anyTimes(); - EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.READING)) - .anyTimes(); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .times(2); - EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.READING)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .times(2); + expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); taskQueue.shutdown(EasyMock.contains("sequenceName-0")); EasyMock.expectLastCall().times(2); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); EasyMock.replay(taskStorage, taskRunner, taskClient, taskQueue); @@ -1601,16 +1604,16 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception addSomeEvents(100); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(4); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1630,35 +1633,35 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L)); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(1, 0L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints1)) - .times(2); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints2)) - .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints1)) + .times(2); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints2)) + .times(2); captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); for (Task task : tasks) { - EasyMock.expect(taskStorage.getStatus(task.getId())) - .andReturn(Optional.of(TaskStatus.running(task.getId()))) - .anyTimes(); - EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); + expect(taskStorage.getStatus(task.getId())) + .andReturn(Optional.of(TaskStatus.running(task.getId()))) + .anyTimes(); + expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); } - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.READING)) - .anyTimes(); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .times(2); - EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); - EasyMock.expect( + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.READING)) + .anyTimes(); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())); + expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-1"))) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .times(2); + expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 20L, 2, 30L))) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 10L, 1, 15L, 2, 35L))); + expect( taskClient.setEndOffsetsAsync( EasyMock.contains("sequenceName-0"), EasyMock.eq(ImmutableMap.of(0, 10L, 1, 20L, 2, 35L)), @@ -1667,7 +1670,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception ).andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); taskQueue.shutdown(EasyMock.contains("sequenceName-0")); EasyMock.expectLastCall().times(2); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); EasyMock.replay(taskStorage, taskRunner, taskClient, taskQueue); @@ -1691,7 +1694,7 @@ public void testStopNotStarted() @Test public void testStop() { - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); taskClient.close(); taskRunner.unregisterListener(StringUtils.format("KafkaSupervisor-%s", DATASOURCE)); replayAll(); @@ -1756,37 +1759,37 @@ public void testStopGracefully() throws Exception workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).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(); - EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); - EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); - EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); - EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); - EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); // getCheckpoints will not be called for id1 as it is in publishing state TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1796,11 +1799,11 @@ public void testStopGracefully() throws Exception verifyAll(); EasyMock.reset(taskRunner, taskClient, taskQueue); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - EasyMock.expect(taskClient.pauseAsync("id2")) - .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); - EasyMock.expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) - .andReturn(Futures.immediateFuture(true)); + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + expect(taskClient.pauseAsync("id2")) + .andReturn(Futures.immediateFuture((Map) ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); + expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) + .andReturn(Futures.immediateFuture(true)); taskQueue.shutdown("id3"); EasyMock.expectLastCall().times(2); @@ -1813,10 +1816,10 @@ public void testStopGracefully() throws Exception @Test public void testResetNoTasks() throws Exception { - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1826,7 +1829,7 @@ public void testResetNoTasks() throws Exception verifyAll(); EasyMock.reset(indexerMetadataStorageCoordinator); - EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); EasyMock.replay(indexerMetadataStorageCoordinator); supervisor.resetInternal(null); @@ -1838,10 +1841,10 @@ public void testResetNoTasks() throws Exception public void testResetDataSourceMetadata() throws Exception { supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1868,9 +1871,9 @@ public void testResetDataSourceMetadata() throws Exception )); EasyMock.reset(indexerMetadataStorageCoordinator); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)) - .andReturn(kafkaDataSourceMetadata); - EasyMock.expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata( + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)) + .andReturn(kafkaDataSourceMetadata); + expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata( EasyMock.capture(captureDataSource), EasyMock.capture(captureDataSourceMetadata) )).andReturn(true); @@ -1894,10 +1897,10 @@ public void testResetDataSourceMetadata() throws Exception public void testResetNoDataSourceMetadata() throws Exception { supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1912,7 +1915,7 @@ public void testResetNoDataSourceMetadata() throws Exception EasyMock.reset(indexerMetadataStorageCoordinator); // no DataSourceMetadata in metadata store - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null); EasyMock.replay(indexerMetadataStorageCoordinator); supervisor.resetInternal(resetMetadata); @@ -1972,36 +1975,36 @@ public void testResetRunningTasks() throws Exception workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).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(); - EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); - EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); - EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); - EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); - EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2011,7 +2014,7 @@ public void testResetRunningTasks() throws Exception verifyAll(); EasyMock.reset(taskQueue, indexerMetadataStorageCoordinator); - EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); taskQueue.shutdown("id2"); taskQueue.shutdown("id3"); EasyMock.replay(taskQueue, indexerMetadataStorageCoordinator); @@ -2065,38 +2068,38 @@ public void testNoDataIngestionTasks() throws Exception null ); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + 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(); - EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); - EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); - EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); - EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); - EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2106,7 +2109,7 @@ public void testNoDataIngestionTasks() throws Exception verifyAll(); EasyMock.reset(taskQueue, indexerMetadataStorageCoordinator); - EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); taskQueue.shutdown("id1"); taskQueue.shutdown("id2"); taskQueue.shutdown("id3"); @@ -2159,39 +2162,39 @@ public void testCheckpointForInactiveTaskGroup() workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect( + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + 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(); - EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); - EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); - EasyMock.expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(Status.READING)); final DateTime startTime = DateTimes.nowUtc(); - EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); - EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + 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> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2254,16 +2257,16 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException null ); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect( + 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(); @@ -2333,40 +2336,40 @@ public void testCheckpointWithNullTaskGroupId() null ); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect( + 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(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); - EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(Status.READING)) - .anyTimes(); + expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.READING)) + .anyTimes(); final TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 0L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.anyString(), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(3); - EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) - .anyTimes(); - EasyMock.expect(taskClient.pauseAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L))) - .anyTimes(); - EasyMock.expect(taskClient.setEndOffsetsAsync( + expect(taskClient.getCheckpointsAsync(EasyMock.anyString(), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(3); + expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + expect(taskClient.pauseAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 10L))) + .anyTimes(); + expect(taskClient.setEndOffsetsAsync( EasyMock.anyString(), EasyMock.eq(ImmutableMap.of(0, 10L)), EasyMock.anyBoolean() )) - .andReturn(Futures.immediateFuture(true)) - .anyTimes(); + .andReturn(Futures.immediateFuture(true)) + .anyTimes(); replayAll(); @@ -2396,16 +2399,16 @@ public void testSuspendedNoRunningTasks() throws Exception supervisor = getSupervisor(1, 1, true, "PT1H", null, null, false, true, kafkaHost); addSomeEvents(1); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); // this asserts that taskQueue.add does not in fact get called because supervisor should be suspended - EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andAnswer((IAnswer) () -> { + expect(taskQueue.add(EasyMock.anyObject())).andAnswer((IAnswer) () -> { Assert.fail(); return null; }).anyTimes(); @@ -2472,47 +2475,47 @@ public void testSuspendedRunningTasks() throws Exception workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); - EasyMock.expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes(); - EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); - EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).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(); - EasyMock.expect(taskClient.getStatusAsync("id1")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); - EasyMock.expect(taskClient.getStatusAsync("id2")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - EasyMock.expect(taskClient.getStatusAsync("id3")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); - EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); - EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); - EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); + expect(taskClient.getStatusAsync("id1")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + expect(taskClient.getStatusAsync("id2")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStatusAsync("id3")) + .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); + expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); // getCheckpoints will not be called for id1 as it is in publishing state TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); - EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) - .andReturn(Futures.immediateFuture(checkpoints)) - .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); - EasyMock.expect(taskClient.pauseAsync("id2")) - .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); - EasyMock.expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) - .andReturn(Futures.immediateFuture(true)); + expect(taskClient.pauseAsync("id2")) + .andReturn(Futures.immediateFuture(ImmutableMap.of(0, 15L, 1, 25L, 2, 30L))); + expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of(0, 15L, 1, 25L, 2, 30L), true)) + .andReturn(Futures.immediateFuture(true)); taskQueue.shutdown("id3"); EasyMock.expectLastCall().times(2); @@ -2525,10 +2528,10 @@ public void testSuspendedRunningTasks() throws Exception @Test public void testResetSuspended() throws Exception { - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2538,7 +2541,7 @@ public void testResetSuspended() throws Exception verifyAll(); EasyMock.reset(indexerMetadataStorageCoordinator); - EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); + expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(DATASOURCE)).andReturn(true); EasyMock.replay(indexerMetadataStorageCoordinator); supervisor.resetInternal(null); @@ -2562,10 +2565,10 @@ public void testFailedInitializationAndRecovery() throws Exception ); addSomeEvents(1); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) @@ -2588,15 +2591,15 @@ public void testFailedInitializationAndRecovery() throws Exception resetAll(); Capture captured = Capture.newInstance(); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2645,7 +2648,8 @@ public void testGetCurrentTotalStats() throws Exception ImmutableMap.of(0, 0L), Optional.absent(), Optional.absent(), - ImmutableSet.of("task1") + ImmutableSet.of("task1"), + ImmutableSet.of() ); supervisor.addTaskGroupToPendingCompletionTaskGroup( @@ -2653,7 +2657,8 @@ public void testGetCurrentTotalStats() throws Exception ImmutableMap.of(0, 0L), Optional.absent(), Optional.absent(), - ImmutableSet.of("task2") + ImmutableSet.of("task2"), + ImmutableSet.of() ); expect(taskClient.getMovingAveragesAsync("task1")).andReturn(Futures.immediateFuture(ImmutableMap.of( diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index 86215ddc4638..060eadb1fb27 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -37,7 +37,7 @@ 1.6.3 - 1.11.42 + 1.11.437 diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 2030cf5f3424..35d7136b8b48 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -68,6 +68,7 @@ public class KinesisRecordSupplier implements RecordSupplier private static final EmittingLogger log = new EmittingLogger(KinesisRecordSupplier.class); private static final long PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS = 3000; private static final long EXCEPTION_RETRY_DELAY_MS = 10000; + private final String endpoint; private class PartitionResource { @@ -268,6 +269,7 @@ public KinesisRecordSupplier( this.recordBufferOfferTimeout = recordBufferOfferTimeout; this.recordBufferFullWait = recordBufferFullWait; this.fetchSequenceNumberTimeout = fetchSequenceNumberTimeout; + this.endpoint = endpoint; AWSCredentialsProvider awsCredentialsProvider = AWSCredentialsUtils.defaultAWSCredentialsProviderChain( new ConstructibleAWSCredentialsConfig(awsAccessKeyId, awsSecretAccessKey) @@ -299,7 +301,6 @@ public KinesisRecordSupplier( ) )); - records = new LinkedBlockingQueue<>(recordBufferSize); log.info( @@ -309,7 +310,8 @@ public KinesisRecordSupplier( ); scheduledExec = Executors.newScheduledThreadPool( - fetchThreads, Execs.makeThreadFactory("KinesisRecordSupplier-Worker-%d") + fetchThreads, + Execs.makeThreadFactory("KinesisRecordSupplier-Worker-%d") ); } @@ -459,7 +461,8 @@ public void close() private AmazonKinesis getKinesisProxy(String streamName) { if (!kinesisProxies.containsKey(streamName)) { - kinesisProxies.put(streamName, kinesisBuilder.build()); + AmazonKinesis kinesis = kinesisBuilder.build(); + kinesisProxies.put(streamName, kinesis); } return kinesisProxies.get(streamName); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 34931978fa22..c3a39d06f602 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; +import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.Task; @@ -241,7 +242,8 @@ protected boolean checkTaskInstance(Task task) @Override protected SeekableStreamSupervisorReportPayload createReportPayload( - int numPartitions, boolean includeOffsets + int numPartitions, + boolean includeOffsets ) { KinesisSupervisorIOConfig ioConfig = spec.getIoConfig(); @@ -264,7 +266,8 @@ protected Map getLagPerPartition(Map currentOffs @Override protected SeekableStreamDataSourceMetadata createDataSourceMetaData( - String stream, Map map + String stream, + Map map ) { return new KinesisDataSourceMetadata( @@ -274,7 +277,9 @@ protected SeekableStreamDataSourceMetadata createDataSourceMetaD @Override protected OrderedSequenceNumber makeSequenceNumber( - String seq, boolean useExclusive, boolean isExclusive + String seq, + boolean useExclusive, + boolean isExclusive ) { return KinesisSequenceNumber.of(seq, useExclusive, isExclusive); @@ -333,6 +338,47 @@ protected boolean checkSequenceAvailability(@NotNull String partition, @NotNull && KinesisSequenceNumber.of(earliestSequence).compareTo(KinesisSequenceNumber.of(sequenceFromMetadata)) <= 0; } + @Override + @VisibleForTesting + protected void addTaskGroupToActivelyReadingTaskGroup( + int taskGroupId, + ImmutableMap partitionOffsets, + Optional minMsgTime, + Optional maxMsgTime, + Set tasks, + Set exclusiveStartingSequencePartitions + ) + { + super.addTaskGroupToActivelyReadingTaskGroup( + taskGroupId, + partitionOffsets, + minMsgTime, + maxMsgTime, + tasks, + exclusiveStartingSequencePartitions + ); + } + + @Override + @VisibleForTesting + protected void addTaskGroupToPendingCompletionTaskGroup( + int taskGroupId, + ImmutableMap partitionOffsets, + Optional minMsgTime, + Optional maxMsgTime, + Set tasks, + Set exclusiveStartingSequencePartitions + ) + { + super.addTaskGroupToPendingCompletionTaskGroup( + taskGroupId, + partitionOffsets, + minMsgTime, + maxMsgTime, + tasks, + exclusiveStartingSequencePartitions + ); + } // Implement this for Kinesis which uses approximate time from latest instead of offset lag /* diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 22a69167346d..02f5d7406d38 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -1851,7 +1851,8 @@ private QueryRunnerFactoryConglomerate makeTimeseriesOnlyConglomerate() { @Override public QueryRunner decorate( - QueryRunner delegate, QueryToolChest> toolChest + QueryRunner delegate, + QueryToolChest> toolChest ) { return delegate; @@ -1953,7 +1954,9 @@ public boolean checkPointDataSourceMetadata( { @Override public boolean registerSegmentHandoffCallback( - SegmentDescriptor descriptor, Executor exec, Runnable handOffRunnable + SegmentDescriptor descriptor, + Executor exec, + Runnable handOffRunnable ) { if (doHandoff) { diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 18c6a01e6640..3b6c5ed5efdd 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -33,6 +33,7 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.data.input.impl.DimensionSchema; @@ -110,6 +111,8 @@ import java.util.concurrent.Executor; import java.util.stream.Collectors; +import static org.easymock.EasyMock.expect; + @RunWith(LocalstackDockerTestRunner.class) @LocalstackDockerProperties(services = {"kinesis"}) public class KinesisSupervisorTest extends EasyMockSupport @@ -2849,6 +2852,51 @@ public void testResetSuspended() throws Exception } + @Test + public void testGetCurrentTotalStats() + { + supervisor = getSupervisor(1, 2, true, "PT1H", null, null, false); + supervisor.addTaskGroupToActivelyReadingTaskGroup( + supervisor.getTaskGroupIdForPartition("0"), + ImmutableMap.of("0", "0"), + Optional.absent(), + Optional.absent(), + ImmutableSet.of("task1"), + ImmutableSet.of() + ); + + supervisor.addTaskGroupToPendingCompletionTaskGroup( + supervisor.getTaskGroupIdForPartition("1"), + ImmutableMap.of("0", "0"), + Optional.absent(), + Optional.absent(), + ImmutableSet.of("task2"), + ImmutableSet.of() + ); + + expect(taskClient.getMovingAveragesAsync("task1")).andReturn(Futures.immediateFuture(ImmutableMap.of( + "prop1", + "val1" + ))).times(1); + + expect(taskClient.getMovingAveragesAsync("task2")).andReturn(Futures.immediateFuture(ImmutableMap.of( + "prop2", + "val2" + ))).times(1); + + replayAll(); + + Map> stats = supervisor.getStats(); + + verifyAll(); + + Assert.assertEquals(2, stats.size()); + Assert.assertEquals(ImmutableSet.of("0", "1"), stats.keySet()); + Assert.assertEquals(ImmutableMap.of("task1", ImmutableMap.of("prop1", "val1")), stats.get("0")); + Assert.assertEquals(ImmutableMap.of("task2", ImmutableMap.of("prop2", "val2")), stats.get("1")); + } + + private static List insertData( AmazonKinesis kinesis, PutRecordsRequest req diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 1d18326d68f7..5032d161ff35 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -66,7 +66,11 @@ public boolean stop(final String id, final boolean publish) try { final FullResponseHolder response = submitRequestWithEmptyContent( - id, HttpMethod.POST, "stop", publish ? "publish=true" : null, true + id, + HttpMethod.POST, + "stop", + publish ? "publish=true" : null, + true ); return isSuccess(response); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index aa97571a03f1..7350236b252e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -832,9 +832,9 @@ private SupervisorReport currentOffsets = entry.getValue().currentSequences; Long remainingSeconds = null; if (startTime != null) { - remainingSeconds = Math.max( - 0, ioConfig.getTaskDuration().getMillis() - (System.currentTimeMillis() - startTime.getMillis()) - ) / 1000; + long elapsedMillis = System.currentTimeMillis() - startTime.getMillis(); + long remainingMillis = Math.max(0, ioConfig.getTaskDuration().getMillis() - elapsedMillis); + remainingSeconds = TimeUnit.MILLISECONDS.toSeconds(remainingMillis); } taskReports.add( @@ -939,19 +939,21 @@ private Map> getCurrentTotalStats() } for (int groupId : pendingCompletionTaskGroups.keySet()) { - TaskGroup group = taskGroups.get(groupId); - for (String taskId : group.taskIds()) { - futures.add( - Futures.transform( - taskClient.getMovingAveragesAsync(taskId), - (Function, StatsFromTaskResult>) (currentStats) -> new StatsFromTaskResult( - groupId, - taskId, - currentStats - ) - ) - ); - groupAndTaskIds.add(new Pair<>(groupId, taskId)); + List pendingGroups = pendingCompletionTaskGroups.get(groupId); + for (TaskGroup pendingGroup : pendingGroups) { + for (String taskId : pendingGroup.taskIds()) { + futures.add( + Futures.transform( + taskClient.getMovingAveragesAsync(taskId), + (Function, StatsFromTaskResult>) (currentStats) -> new StatsFromTaskResult( + groupId, + taskId, + currentStats + ) + ) + ); + groupAndTaskIds.add(new Pair<>(groupId, taskId)); + } } } @@ -971,6 +973,55 @@ private Map> getCurrentTotalStats() return allStats; } + + @VisibleForTesting + protected void addTaskGroupToActivelyReadingTaskGroup( + int taskGroupId, + ImmutableMap partitionOffsets, + Optional minMsgTime, + Optional maxMsgTime, + Set tasks, + Set exclusiveStartingSequencePartitions + ) + { + TaskGroup group = new TaskGroup( + taskGroupId, + partitionOffsets, + minMsgTime, + maxMsgTime, + exclusiveStartingSequencePartitions + ); + group.tasks.putAll(tasks.stream().collect(Collectors.toMap(x -> x, x -> new TaskData()))); + if (taskGroups.putIfAbsent(taskGroupId, group) != null) { + throw new ISE( + "trying to add taskGroup with ID [%s] to actively reading task groups, but group already exists.", + taskGroupId + ); + } + } + + @VisibleForTesting + protected void addTaskGroupToPendingCompletionTaskGroup( + int taskGroupId, + ImmutableMap partitionOffsets, + Optional minMsgTime, + Optional maxMsgTime, + Set tasks, + Set exclusiveStartingSequencePartitions + ) + { + TaskGroup group = new TaskGroup( + taskGroupId, + partitionOffsets, + minMsgTime, + maxMsgTime, + exclusiveStartingSequencePartitions + ); + group.tasks.putAll(tasks.stream().collect(Collectors.toMap(x -> x, x -> new TaskData()))); + pendingCompletionTaskGroups.computeIfAbsent(taskGroupId, x -> new CopyOnWriteArrayList<>()) + .add(group); + } + @VisibleForTesting protected void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException @@ -1680,7 +1731,7 @@ private void updatePartitionDataFromStream() return; } - if (partitionIds == null) { + if (partitionIds == null || partitionIds.size() == 0) { log.warn("No partitions found for stream[%s]", ioConfig.getId()); return; } From b2eaf74a7ad25d012c3035c688d6933bf38f5dd1 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 29 Oct 2018 13:02:35 -0700 Subject: [PATCH 32/87] fixes for kinesis tasks --- .../kinesis-indexing-service/pom.xml | 12 +- .../indexing/kinesis/KinesisIndexTask.java | 34 +- .../kinesis/KinesisIndexTaskTest.java | 44 +- .../kinesis/KinesisRecordSupplierTest.java | 20 +- .../supervisor/KinesisSupervisorTest.java | 14 +- .../supervisor/SeekableStreamSupervisor.java | 45 +- pom.xml | 2874 +++++++++-------- 7 files changed, 1517 insertions(+), 1526 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index 060eadb1fb27..ec949136b23f 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -34,12 +34,6 @@ ../../pom.xml - - - 1.6.3 - 1.11.437 - - org.apache.druid @@ -75,13 +69,13 @@ com.amazonaws aws-java-sdk-kinesis - ${aws.version} + ${aws.sdk.version} com.amazonaws aws-java-sdk-sts - ${aws.version} + ${aws.sdk.version} @@ -119,7 +113,7 @@ cloud.localstack localstack-utils - 0.1.14 + 0.1.13 diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index e13802f90b5e..622b20630cd7 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -274,8 +274,14 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception // Start up, set up initial offsets. final Object restoredMetadata = driver.startJob(); if (restoredMetadata == null) { + log.info( + "no restored metadata found for [%s], using starting sequences[%s] from ioConfig", + topic, + ioConfig.getStartPartitions() + ); lastOffsets.putAll(ioConfig.getStartPartitions().getMap()); } else { + log.info("found meatadata [%s] for [%s]", restoredMetadata, topic); @SuppressWarnings("unchecked") final Map restoredMetadataMap = (Map) restoredMetadata; final SeekableStreamPartitions restoredNextPartitions = toolbox @@ -541,7 +547,11 @@ public void run() // Sanity check, we should only be publishing things that match our desired end state. if (!endOffsets.equals(finalPartitions.getMap())) { - throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata); + throw new ISE( + "WTF?! Driver attempted to publish invalid metadata[%s], final sequences are [%s]", + commitMetadata, + endOffsets + ); } final SegmentTransactionalInsertAction action; @@ -806,27 +816,13 @@ private boolean possiblyPause(Set assignment) throws InterruptedExceptio pauseRequested = true; } */ - if (pauseRequested) { status = Status.PAUSED; - long nanos = 0; hasPaused.signalAll(); while (pauseRequested) { - if (pauseMillis == PAUSE_FOREVER) { - log.info("Pausing ingestion until resumed"); - shouldResume.await(); - } else { - if (pauseMillis > 0) { - log.info("Pausing ingestion for [%,d] ms", pauseMillis); - nanos = TimeUnit.MILLISECONDS.toNanos(pauseMillis); - pauseMillis = 0; - } - if (nanos <= 0L) { - pauseRequested = false; // timeout elapsed - } - nanos = shouldResume.awaitNanos(nanos); - } + log.info("Pausing ingestion until resumed"); + shouldResume.await(); } status = Status.READING; @@ -1237,9 +1233,7 @@ public Response setEndOffsets(Map offsets, final boolean resume) pauseLock.unlock(); } - if (resume) { - resume(); - } + resume(); return Response.ok(endOffsets).build(); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 02f5d7406d38..71f2d13b1047 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -19,7 +19,7 @@ package org.apache.druid.indexing.kinesis; -import cloud.localstack.Localstack; +import cloud.localstack.LocalstackTestRunner; import cloud.localstack.TestUtils; import cloud.localstack.docker.LocalstackDockerTestRunner; import cloud.localstack.docker.annotation.LocalstackDockerProperties; @@ -188,7 +188,7 @@ public class KinesisIndexTaskTest { static { TestUtils.setEnv("AWS_CBOR_DISABLE", "1"); - if (Localstack.useSSL()) { + if (LocalstackTestRunner.useSSL()) { TestUtils.disableSslCertChecking(); } } @@ -430,7 +430,7 @@ public void testRunAfterDataInserted() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -508,7 +508,7 @@ public void testRunBeforeDataInserted() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -588,7 +588,7 @@ public void testRunWithMinimumMessageTime() throws Exception null, DateTimes.of("2010"), null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -658,7 +658,7 @@ public void testRunWithMaximumMessageTime() throws Exception null, null, DateTimes.of("2010"), - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -738,7 +738,7 @@ public void testRunWithTransformSpec() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -807,7 +807,7 @@ public void testRunOnNothing() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -861,7 +861,7 @@ public void testReportParseExceptions() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -915,7 +915,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -1014,7 +1014,7 @@ public void testMultipleParseExceptionsFailure() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -1091,7 +1091,7 @@ public void testRunReplicas() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -1118,7 +1118,7 @@ public void testRunReplicas() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -1189,7 +1189,7 @@ public void testRunConflicting() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -1216,7 +1216,7 @@ public void testRunConflicting() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -1288,7 +1288,7 @@ public void testRunConflictingWithoutTransactions() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -1315,7 +1315,7 @@ public void testRunConflictingWithoutTransactions() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -1389,7 +1389,7 @@ public void testRunOneTaskTwoPartitions() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -1469,7 +1469,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -1496,7 +1496,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -1578,7 +1578,7 @@ public void testRestore() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, @@ -1623,7 +1623,7 @@ public void testRestore() throws Exception null, null, null, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, TestUtils.TEST_ACCESS_KEY, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index b9a4ba72e941..3a39396b4099 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -19,7 +19,7 @@ package org.apache.druid.indexing.kinesis; -import cloud.localstack.Localstack; +import cloud.localstack.LocalstackTestRunner; import cloud.localstack.TestUtils; import cloud.localstack.docker.LocalstackDockerTestRunner; import cloud.localstack.docker.annotation.LocalstackDockerProperties; @@ -44,7 +44,6 @@ import org.junit.Test; import org.junit.runner.RunWith; -import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashSet; @@ -59,9 +58,6 @@ public class KinesisRecordSupplierTest { static { TestUtils.setEnv("AWS_CBOR_DISABLE", "1"); - if (Localstack.useSSL()) { - TestUtils.disableSslCertChecking(); - } } private static final Logger log = new Logger(KinesisRecordSupplierTest.class); @@ -180,7 +176,7 @@ private static String getStreamName() } @Before - public void setupTest() throws IOException, InterruptedException + public void setupTest() { stream = getStreamName(); } @@ -197,7 +193,7 @@ public void testSupplierSetup() throws InterruptedException ); KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), TestUtils.TEST_ACCESS_KEY, TestUtils.TEST_SECRET_KEY, 1, @@ -243,7 +239,7 @@ public void testPoll() throws InterruptedException ); KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), TestUtils.TEST_ACCESS_KEY, TestUtils.TEST_SECRET_KEY, 1, @@ -295,7 +291,7 @@ public void testPollAfterMoreDataAdded() throws InterruptedException ); KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), TestUtils.TEST_ACCESS_KEY, TestUtils.TEST_SECRET_KEY, 1, @@ -355,7 +351,7 @@ public void testSeek() throws InterruptedException, TimeoutException ); KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), TestUtils.TEST_ACCESS_KEY, TestUtils.TEST_SECRET_KEY, 1, @@ -426,7 +422,7 @@ public void testSeekToLatest() throws InterruptedException, TimeoutException ); KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), TestUtils.TEST_ACCESS_KEY, TestUtils.TEST_SECRET_KEY, 1, @@ -465,7 +461,7 @@ public void testSeekUnassigned() throws InterruptedException, TimeoutException ); KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), TestUtils.TEST_ACCESS_KEY, TestUtils.TEST_SECRET_KEY, 1, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 3b6c5ed5efdd..d761fcc9495e 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -19,7 +19,7 @@ package org.apache.druid.indexing.kinesis.supervisor; -import cloud.localstack.Localstack; +import cloud.localstack.LocalstackTestRunner; import cloud.localstack.docker.LocalstackDockerTestRunner; import cloud.localstack.docker.annotation.LocalstackDockerProperties; import com.amazonaws.http.SdkHttpMetadata; @@ -216,7 +216,7 @@ public static void setupClass() { cloud.localstack.TestUtils.setEnv("AWS_CBOR_DISABLE", "1"); /* Disable SSL certificate checks for local testing */ - if (Localstack.useSSL()) { + if (LocalstackTestRunner.useSSL()) { cloud.localstack.TestUtils.disableSslCertChecking(); } @@ -1469,7 +1469,7 @@ public void testDiscoverExistingPublishingTask() throws Exception KinesisIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(cloud.localstack.TestUtils.TEST_ACCESS_KEY, capturedTaskConfig.getAwsAccessKeyId()); Assert.assertEquals(cloud.localstack.TestUtils.TEST_SECRET_KEY, capturedTaskConfig.getAwsSecretAccessKey()); - Assert.assertEquals(Localstack.getEndpointKinesis(), capturedTaskConfig.getEndpoint()); + Assert.assertEquals(LocalstackTestRunner.getEndpointKinesis(), capturedTaskConfig.getEndpoint()); Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); @@ -1600,7 +1600,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() KinesisIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(cloud.localstack.TestUtils.TEST_ACCESS_KEY, capturedTaskConfig.getAwsAccessKeyId()); Assert.assertEquals(cloud.localstack.TestUtils.TEST_SECRET_KEY, capturedTaskConfig.getAwsSecretAccessKey()); - Assert.assertEquals(Localstack.getEndpointKinesis(), capturedTaskConfig.getEndpoint()); + Assert.assertEquals(LocalstackTestRunner.getEndpointKinesis(), capturedTaskConfig.getEndpoint()); Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); @@ -2960,7 +2960,7 @@ private KinesisSupervisor getSupervisor( { KinesisSupervisorIOConfig KinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( stream, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, replicas, taskCount, @@ -3132,7 +3132,7 @@ private KinesisIndexTask createKinesisIndexTask( null, minimumMessageTime, maximumMessageTime, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, null, cloud.localstack.TestUtils.TEST_ACCESS_KEY, @@ -3161,7 +3161,7 @@ private KinesisSupervisor getSupervisor( { KinesisSupervisorIOConfig KinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( stream, - Localstack.getEndpointKinesis(), + LocalstackTestRunner.getEndpointKinesis(), null, replicas, taskCount, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 7350236b252e..a39dba760815 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1058,29 +1058,33 @@ private void possiblyRegisterListener() Optional taskRunner = taskMaster.getTaskRunner(); if (taskRunner.isPresent()) { - taskRunner.get().registerListener( - new TaskRunnerListener() - { - @Override - public String getListenerId() - { - return supervisorId; - } - - @Override - public void locationChanged(final String taskId, final TaskLocation newLocation) + try { + taskRunner.get().registerListener( + new TaskRunnerListener() { - // do nothing - } + @Override + public String getListenerId() + { + return supervisorId; + } - @Override - public void statusChanged(String taskId, TaskStatus status) - { - notices.add(new RunNotice()); - } - }, MoreExecutors.sameThreadExecutor() - ); + @Override + public void locationChanged(final String taskId, final TaskLocation newLocation) + { + // do nothing + } + @Override + public void statusChanged(String taskId, TaskStatus status) + { + notices.add(new RunNotice()); + } + }, MoreExecutors.sameThreadExecutor() + ); + } + catch (ISE e) { + log.info("listener already registered with taskrunner."); + } listenerRegistered = true; } } @@ -1748,6 +1752,7 @@ private void updatePartitionDataFromStream() boolean initialPartitionDiscovery = this.partitionIds.isEmpty(); for (partitionType partitionId : partitionIds) { if (closedPartitions.contains(partitionId)) { + log.info("partition [%s] is closed and has no more data, skipping.", partitionId); continue; } diff --git a/pom.xml b/pom.xml index 809fd010e1e6..145d9a545539 100644 --- a/pom.xml +++ b/pom.xml @@ -20,1460 +20,1462 @@ - 4.0.0 + 4.0.0 - - org.apache - apache - 21 - + + org.apache + apache + 21 + - org.apache.druid - druid - 0.13.0-incubating-SNAPSHOT - pom + org.apache.druid + druid + 0.13.0-incubating-SNAPSHOT + pom - Druid - Druid - A Distributed Column Store - https://druid.apache.org/ + Druid + Druid - A Distributed Column Store + https://druid.apache.org/ - - - Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 - - + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0 + + - - - Druid Committers - http://druid.io/community/index.html#committers - - + + + Druid Committers + http://druid.io/community/index.html#committers + + - - - Apache Druid (incubating) developers list - dev-subscribe@druid.apache.org - dev-unsubscribe@druid.apache.org - dev@druid.apache.org - https://mail-archives.apache.org/mod_mbox/druid-dev - - + + + Apache Druid (incubating) developers list + dev-subscribe@druid.apache.org + dev-unsubscribe@druid.apache.org + dev@druid.apache.org + https://mail-archives.apache.org/mod_mbox/druid-dev + + - 2011 + 2011 - - scm:git:ssh://git@github.com/apache/incubator-druid.git - scm:git:ssh://git@github.com/apache/incubator-druid.git - https://github.com/apache/incubator-druid.git - 0.13.0-incubating-SNAPSHOT - + + scm:git:ssh://git@github.com/apache/incubator-druid.git + scm:git:ssh://git@github.com/apache/incubator-druid.git + https://github.com/apache/incubator-druid.git + 0.13.0-incubating-SNAPSHOT + - - 1.8 - 1.8 - UTF-8 - 4.0.0 - 2.12.0 - 1.10.0 - 1.17.0 - 4.0.0 - 8.1.0 - 16.0.1 - 4.1.0 - 9.4.10.v20180503 - 1.19.3 - - 2.6.7 - 2.5 - - 3.10.6.Final - - 4.1.30.Final - 1.7.12 - - 2.8.3 - 2.0.0 - 1.6.6 - 1.11.199 - 2.5.5 - - 3.4.11 - apache.snapshots - Apache Snapshot Repository - https://repository.apache.org/snapshots + + 1.8 + 1.8 + UTF-8 + 4.0.0 + 2.12.0 + 1.10.0 + 1.17.0 + 4.0.0 + 8.1.0 + 16.0.1 + 4.1.0 + 9.4.10.v20180503 + 1.19.3 + + 2.6.7 + 2.5 + + 3.10.6.Final + + 4.1.30.Final + 1.7.12 + + 2.8.3 + 2.0.0 + 1.6.6 + 1.11.199 + 2.5.5 + + 3.4.11 + apache.snapshots + Apache Snapshot Repository + https://repository.apache.org/snapshots - - 3 - + + 3 + - - core - examples - indexing-hadoop - indexing-service - processing - server - sql - services - integration-tests - benchmarks - aws-common - extendedset - hll - - extensions-core/avro-extensions - extensions-core/datasketches - extensions-core/druid-bloom-filter - extensions-core/druid-kerberos - extensions-core/hdfs-storage - extensions-core/histogram - extensions-core/stats - extensions-core/kafka-eight - extensions-core/kafka-extraction-namespace - extensions-core/kafka-indexing-service - extensions-core/kinesis-indexing-service - extensions-core/mysql-metadata-storage - extensions-core/postgresql-metadata-storage - extensions-core/protobuf-extensions - extensions-core/lookups-cached-global - extensions-core/lookups-cached-single - extensions-core/s3-extensions - extensions-core/simple-client-sslcontext - extensions-core/druid-basic-security - - extensions-contrib/influx-extensions - extensions-contrib/azure-extensions - extensions-contrib/cassandra-storage - extensions-contrib/druid-rocketmq - extensions-contrib/cloudfiles-extensions - extensions-contrib/graphite-emitter - extensions-contrib/kafka-eight-simpleConsumer - extensions-contrib/rabbitmq - extensions-contrib/distinctcount - extensions-contrib/parquet-extensions - extensions-contrib/statsd-emitter - extensions-contrib/orc-extensions - extensions-contrib/time-min-max - extensions-contrib/google-extensions - extensions-contrib/virtual-columns - extensions-contrib/thrift-extensions - extensions-contrib/ambari-metrics-emitter - extensions-contrib/sqlserver-metadata-storage - extensions-contrib/kafka-emitter - extensions-contrib/redis-cache - extensions-contrib/opentsdb-emitter - extensions-contrib/materialized-view-maintenance - extensions-contrib/materialized-view-selection - - distribution - + + core + examples + indexing-hadoop + indexing-service + processing + server + sql + services + integration-tests + benchmarks + aws-common + extendedset + hll + + extensions-core/avro-extensions + extensions-core/datasketches + extensions-core/druid-bloom-filter + extensions-core/druid-kerberos + extensions-core/hdfs-storage + extensions-core/histogram + extensions-core/stats + extensions-core/kafka-eight + extensions-core/kafka-extraction-namespace + extensions-core/kafka-indexing-service + extensions-core/kinesis-indexing-service + extensions-core/mysql-metadata-storage + extensions-core/postgresql-metadata-storage + extensions-core/protobuf-extensions + extensions-core/lookups-cached-global + extensions-core/lookups-cached-single + extensions-core/s3-extensions + extensions-core/simple-client-sslcontext + extensions-core/druid-basic-security + + extensions-contrib/influx-extensions + extensions-contrib/azure-extensions + extensions-contrib/cassandra-storage + extensions-contrib/druid-rocketmq + extensions-contrib/cloudfiles-extensions + extensions-contrib/graphite-emitter + extensions-contrib/kafka-eight-simpleConsumer + extensions-contrib/rabbitmq + extensions-contrib/distinctcount + extensions-contrib/parquet-extensions + extensions-contrib/statsd-emitter + extensions-contrib/orc-extensions + extensions-contrib/time-min-max + extensions-contrib/google-extensions + extensions-contrib/virtual-columns + extensions-contrib/thrift-extensions + extensions-contrib/ambari-metrics-emitter + extensions-contrib/sqlserver-metadata-storage + extensions-contrib/kafka-emitter + extensions-contrib/redis-cache + extensions-contrib/opentsdb-emitter + extensions-contrib/materialized-view-maintenance + extensions-contrib/materialized-view-selection + + distribution + - - - ${repoOrgId} - ${repoOrgName} - ${repoOrgUrl} - - + + + ${repoOrgId} + ${repoOrgName} + ${repoOrgUrl} + + - - - - - commons-codec - commons-codec - 1.7 - - - commons-httpclient - commons-httpclient - 3.1 - - - commons-io - commons-io - 2.5 - - - commons-logging - commons-logging - 1.1.1 - - - commons-lang - commons-lang - 2.6 - - - com.amazonaws - aws-java-sdk-ec2 - ${aws.sdk.version} - - - com.amazonaws - aws-java-sdk-s3 - ${aws.sdk.version} - - - com.ning - compress-lzf - 1.0.4 - - - io.airlift - airline - 0.7 - - - - com.google.code.findbugs - annotations - - - - - org.skife.config - config-magic - 0.9 - - - org.slf4j - slf4j-api - - - - - org.apache.zookeeper - zookeeper - ${zookeeper.version} - - - org.slf4j - slf4j-log4j12 - - - log4j - log4j - - - io.netty - netty - - - - - org.apache.curator - curator-client - ${apache.curator.version} - - - org.apache.curator - curator-framework - ${apache.curator.version} - - - org.jboss.netty - netty - - - - - org.apache.curator - curator-recipes - ${apache.curator.version} - - - org.apache.zookeeper - zookeeper - - - - - org.apache.curator - curator-x-discovery - ${apache.curator.version} - - - org.apache.calcite - calcite-core - ${calcite.version} - - - org.apache.calcite - calcite-linq4j - ${calcite.version} - - - org.apache.calcite.avatica - avatica-core - ${avatica.version} - - - org.apache.calcite.avatica - avatica-server - ${avatica.version} - - - com.google.guava - guava - ${guava.version} - - - com.google.inject - guice - ${guice.version} - - - com.google.inject.extensions - guice-servlet - ${guice.version} - - - com.google.inject.extensions - guice-multibindings - ${guice.version} - - - com.google.errorprone - error_prone_annotations - 2.2.0 - - - com.ibm.icu - icu4j - 54.1.1 - - - org.mozilla - rhino - 1.7R5 - - - org.apache.commons - commons-compress - 1.16 - - - org.tukaani - xz - 1.8 - - - com.github.luben - zstd-jni - 1.3.3-1 - - - com.fasterxml.jackson.core - jackson-annotations - ${jackson.version} - - - com.fasterxml.jackson.core - jackson-core - ${jackson.version} - - - com.fasterxml.jackson.core - jackson-databind - ${jackson.version} - - - com.fasterxml.jackson.datatype - jackson-datatype-guava - ${jackson.version} - - - com.fasterxml.jackson.datatype - jackson-datatype-joda - ${jackson.version} - - - com.fasterxml.jackson.dataformat - jackson-dataformat-smile - ${jackson.version} - - - com.fasterxml.jackson.jaxrs - jackson-jaxrs-json-provider - ${jackson.version} - - - com.fasterxml.jackson.jaxrs - jackson-jaxrs-smile-provider - ${jackson.version} - - - org.hibernate - hibernate-validator - 5.1.3.Final - - - javax.validation - validation-api - 1.1.0.Final - - - javax.inject - javax.inject - 1 - - - javax.el - javax.el-api - 3.0.0 - - - org.glassfish - javax.el - 3.0.0 - - - org.jdbi - jdbi - 2.63.1 - - - com.sun.jersey - jersey-core - ${jersey.version} - - - com.sun.jersey.contribs - jersey-guice - ${jersey.version} - - - com.google.inject - guice - - - com.google.inject.extensions - guice-servlet - - - - - com.sun.jersey - jersey-server - ${jersey.version} - - - com.sun.jersey - jersey-json - ${jersey.version} - - - org.eclipse.jetty - jetty-server - ${jetty.version} - - - org.eclipse.jetty - jetty-servlet - ${jetty.version} - - - org.eclipse.jetty - jetty-servlets - ${jetty.version} - - - org.eclipse.jetty - jetty-proxy - ${jetty.version} - - - org.eclipse.jetty - jetty-util - ${jetty.version} - - - org.eclipse.jetty - jetty-security - ${jetty.version} - - - io.netty - netty - ${netty3.version} - - - io.netty - netty-all - ${netty4.version} - - - joda-time - joda-time - 2.9.9 - - - com.google.code.findbugs - jsr305 - 2.0.1 - - - org.apache.logging.log4j - log4j-api - ${log4j.version} - - - org.apache.logging.log4j - log4j-core - ${log4j.version} - - - org.apache.logging.log4j - log4j-slf4j-impl - ${log4j.version} - - - org.apache.logging.log4j - log4j-1.2-api - ${log4j.version} - - - - org.apache.logging.log4j - log4j-jul - ${log4j.version} - - - org.slf4j - jcl-over-slf4j - ${slf4j.version} - - - com.lmax - disruptor - 3.3.6 - - - net.spy - spymemcached - 2.12.3 - - - org.antlr - antlr4-runtime - 4.5.1 - - - org.antlr - antlr4-coordinator - 4.5.1 - - - commons-cli - commons-cli - 1.2 - - - org.apache.commons - commons-dbcp2 - 2.0.1 - - - org.lz4 - lz4-java - 1.5.0 - - - com.google.protobuf - protobuf-java - 3.1.0 - - - io.tesla.aether - tesla-aether - 0.0.5 - - - org.slf4j - slf4j-api - - - - - org.eclipse.aether - aether-api - 0.9.0.M2 - - - org.apache.httpcomponents - httpclient - 4.5.3 - - - org.apache.httpcomponents - httpcore - 4.4.4 - - - org.apache.hadoop - hadoop-client - ${hadoop.compile.version} - provided - - - org.apache.avro - avro - - - - - org.mapdb - mapdb - 1.0.8 - - - org.apache.derby - derbynet - 10.11.1.1 - - - org.apache.derby - derbyclient - 10.11.1.1 - - - org.apache.commons - commons-math3 - 3.6.1 - - - it.unimi.dsi - fastutil - 8.1.0 - - - com.opencsv - opencsv - 4.2 - - - com.jayway.jsonpath - json-path - 2.3.0 - - - net.thisptr - jackson-jq - 0.0.7 - - - org.slf4j - slf4j-api - 1.6.4 - - - org.roaringbitmap - RoaringBitmap - 0.5.18 - - - org.ow2.asm - asm - 5.2 - - - org.ow2.asm - asm-commons - 5.2 - - - org.asynchttpclient - async-http-client - - 2.5.3 - - - org.gridkit.lab - jvm-attach-api - 1.2 - - - net.java.dev.jna - jna - 4.5.1 - - - io.dropwizard.metrics - metrics-core - ${dropwizard.metrics.version} - - - org.codehaus.jackson - jackson-core-asl - 1.9.13 - - - org.codehaus.jackson - jackson-mapper-asl - 1.9.13 - - - javax.servlet - javax.servlet-api - 3.1.0 - - - javax.activation - activation - 1.1.1 - - - commons-pool - commons-pool - 1.6 - - - org.codehaus.plexus - plexus-utils - 3.0.15 - - - com.github.ben-manes.caffeine - caffeine - ${caffeine.version} - + + + + + commons-codec + commons-codec + 1.7 + + + commons-httpclient + commons-httpclient + 3.1 + + + commons-io + commons-io + 2.5 + + + commons-logging + commons-logging + 1.1.1 + + + commons-lang + commons-lang + 2.6 + + + com.amazonaws + aws-java-sdk-ec2 + ${aws.sdk.version} + + + com.amazonaws + aws-java-sdk-s3 + ${aws.sdk.version} + + + com.ning + compress-lzf + 1.0.4 + + + io.airlift + airline + 0.7 + + + + com.google.code.findbugs + annotations + + + + + org.skife.config + config-magic + 0.9 + + + org.slf4j + slf4j-api + + + + + org.apache.zookeeper + zookeeper + ${zookeeper.version} + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + io.netty + netty + + + + + org.apache.curator + curator-client + ${apache.curator.version} + + + org.apache.curator + curator-framework + ${apache.curator.version} + + + org.jboss.netty + netty + + + + + org.apache.curator + curator-recipes + ${apache.curator.version} + + + org.apache.zookeeper + zookeeper + + + + + org.apache.curator + curator-x-discovery + ${apache.curator.version} + + + org.apache.calcite + calcite-core + ${calcite.version} + + + org.apache.calcite + calcite-linq4j + ${calcite.version} + + + org.apache.calcite.avatica + avatica-core + ${avatica.version} + + + org.apache.calcite.avatica + avatica-server + ${avatica.version} + + + com.google.guava + guava + ${guava.version} + + + com.google.inject + guice + ${guice.version} + + + com.google.inject.extensions + guice-servlet + ${guice.version} + + + com.google.inject.extensions + guice-multibindings + ${guice.version} + + + com.google.errorprone + error_prone_annotations + 2.2.0 + + + com.ibm.icu + icu4j + 54.1.1 + + + org.mozilla + rhino + 1.7R5 + + + org.apache.commons + commons-compress + 1.16 + + + org.tukaani + xz + 1.8 + + + com.github.luben + zstd-jni + 1.3.3-1 + + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + com.fasterxml.jackson.datatype + jackson-datatype-guava + ${jackson.version} + + + com.fasterxml.jackson.datatype + jackson-datatype-joda + ${jackson.version} + + + com.fasterxml.jackson.dataformat + jackson-dataformat-smile + ${jackson.version} + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + ${jackson.version} + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-smile-provider + ${jackson.version} + + + org.hibernate + hibernate-validator + 5.1.3.Final + + + javax.validation + validation-api + 1.1.0.Final + + + javax.inject + javax.inject + 1 + + + javax.el + javax.el-api + 3.0.0 + + + org.glassfish + javax.el + 3.0.0 + + + org.jdbi + jdbi + 2.63.1 + + + com.sun.jersey + jersey-core + ${jersey.version} + + + com.sun.jersey.contribs + jersey-guice + ${jersey.version} + + + com.google.inject + guice + + + com.google.inject.extensions + guice-servlet + + + + + com.sun.jersey + jersey-server + ${jersey.version} + + + com.sun.jersey + jersey-json + ${jersey.version} + + + org.eclipse.jetty + jetty-server + ${jetty.version} + + + org.eclipse.jetty + jetty-servlet + ${jetty.version} + + + org.eclipse.jetty + jetty-servlets + ${jetty.version} + + + org.eclipse.jetty + jetty-proxy + ${jetty.version} + + + org.eclipse.jetty + jetty-util + ${jetty.version} + + + org.eclipse.jetty + jetty-security + ${jetty.version} + + + io.netty + netty + ${netty3.version} + + + io.netty + netty-all + ${netty4.version} + + + joda-time + joda-time + 2.9.9 + + + com.google.code.findbugs + jsr305 + 2.0.1 + + + org.apache.logging.log4j + log4j-api + ${log4j.version} + + + org.apache.logging.log4j + log4j-core + ${log4j.version} + + + org.apache.logging.log4j + log4j-slf4j-impl + ${log4j.version} + + + org.apache.logging.log4j + log4j-1.2-api + ${log4j.version} + + + + org.apache.logging.log4j + log4j-jul + ${log4j.version} + + + org.slf4j + jcl-over-slf4j + ${slf4j.version} + + + com.lmax + disruptor + 3.3.6 + + + net.spy + spymemcached + 2.12.3 + + + org.antlr + antlr4-runtime + 4.5.1 + + + org.antlr + antlr4-coordinator + 4.5.1 + + + commons-cli + commons-cli + 1.2 + + + org.apache.commons + commons-dbcp2 + 2.0.1 + + + org.lz4 + lz4-java + 1.5.0 + + + com.google.protobuf + protobuf-java + 3.1.0 + + + io.tesla.aether + tesla-aether + 0.0.5 + + + org.slf4j + slf4j-api + + + + + org.eclipse.aether + aether-api + 0.9.0.M2 + + + org.apache.httpcomponents + httpclient + 4.5.3 + + + org.apache.httpcomponents + httpcore + 4.4.4 + + + org.apache.hadoop + hadoop-client + ${hadoop.compile.version} + provided + + + org.apache.avro + avro + + + + + org.mapdb + mapdb + 1.0.8 + + + org.apache.derby + derbynet + 10.11.1.1 + + + org.apache.derby + derbyclient + 10.11.1.1 + + + org.apache.commons + commons-math3 + 3.6.1 + + + it.unimi.dsi + fastutil + 8.1.0 + + + com.opencsv + opencsv + 4.2 + + + com.jayway.jsonpath + json-path + 2.3.0 + + + net.thisptr + jackson-jq + 0.0.7 + + + org.slf4j + slf4j-api + 1.6.4 + + + org.roaringbitmap + RoaringBitmap + 0.5.18 + + + org.ow2.asm + asm + 5.2 + + + org.ow2.asm + asm-commons + 5.2 + + + org.asynchttpclient + async-http-client + + 2.5.3 + + + org.gridkit.lab + jvm-attach-api + 1.2 + + + net.java.dev.jna + jna + 4.5.1 + + + io.dropwizard.metrics + metrics-core + ${dropwizard.metrics.version} + + + org.codehaus.jackson + jackson-core-asl + 1.9.13 + + + org.codehaus.jackson + jackson-mapper-asl + 1.9.13 + + + javax.servlet + javax.servlet-api + 3.1.0 + + + javax.activation + activation + 1.1.1 + + + commons-pool + commons-pool + 1.6 + + + org.codehaus.plexus + plexus-utils + 3.0.15 + + + com.github.ben-manes.caffeine + caffeine + ${caffeine.version} + - - org.apache.calcite - calcite-core - ${calcite.version} - test-jar - test - - - org.easymock - easymock - 3.4 - test - - - junit - junit - 4.12 - test - - - org.powermock - powermock-module-junit4 - ${powermock.version} - test - - - org.powermock - powermock-api-easymock - ${powermock.version} - test - - - org.slf4j - slf4j-simple - ${slf4j.version} - - - com.carrotsearch - junit-benchmarks - 0.7.2 - test - - - com.google.caliper - caliper - 0.5-rc1 - test - - - org.apache.curator - curator-test - ${apache.curator.test.version} - - - org.jboss.netty - netty - - - test - - - com.ircclouds.irc - irc-api - 1.0-0014 - - - org.slf4j - slf4j-api - - - - - com.maxmind.geoip2 - geoip2 - 0.4.0 - - - com.google.http-client - google-http-client - - - com.google.http-client - google-http-client-jackson2 - - - com.fasterxml.jackson.core - jackson-databind - - - - - org.testng - testng - 6.8.7 - - - org.hamcrest - hamcrest-all - 1.3 - test - - - pl.pragmatists - JUnitParams - 1.0.4 - test - - - com.google.guava - guava-testlib - ${guava.version} - test - - - + + org.apache.calcite + calcite-core + ${calcite.version} + test-jar + test + + + org.easymock + easymock + 3.4 + test + + + junit + junit + 4.12 + test + + + org.powermock + powermock-module-junit4 + ${powermock.version} + test + + + org.powermock + powermock-api-easymock + ${powermock.version} + test + + + org.slf4j + slf4j-simple + ${slf4j.version} + + + com.carrotsearch + junit-benchmarks + 0.7.2 + test + + + com.google.caliper + caliper + 0.5-rc1 + test + + + org.apache.curator + curator-test + ${apache.curator.test.version} + + + org.jboss.netty + netty + + + test + + + com.ircclouds.irc + irc-api + 1.0-0014 + + + org.slf4j + slf4j-api + + + + + com.maxmind.geoip2 + geoip2 + 0.4.0 + + + com.google.http-client + google-http-client + + + com.google.http-client + google-http-client-jackson2 + + + com.fasterxml.jackson.core + jackson-databind + + + + + org.testng + testng + 6.8.7 + + + org.hamcrest + hamcrest-all + 1.3 + test + + + pl.pragmatists + JUnitParams + 1.0.4 + test + + + com.google.guava + guava-testlib + ${guava.version} + test + + + - - - - org.codehaus.mojo - cobertura-maven-plugin - - - org.eluder.coveralls - coveralls-maven-plugin - 4.0.0 - - - org.apache.maven.plugins - maven-checkstyle-plugin - 2.17 - - ${project.build.sourceDirectory} - true - codestyle/checkstyle.xml - codestyle/checkstyle-suppressions.xml - checkstyle.suppressions.file - UTF-8 - codestyle/LICENSE.txt - true - true - - - - com.puppycrawl.tools - checkstyle - 8.0 - - - - - validate - validate - - check - - - - - - org.apache.maven.plugins - maven-pmd-plugin - 3.8 - - true - - /rulesets/java/imports.xml - - - target/generated-sources/ - - - - - validate - validate - - check - - - - - - de.thetaphi - forbiddenapis - 2.3 - - false - - - jdk-unsafe - - - ${project.parent.basedir}/codestyle/joda-time-forbidden-apis.txt - ${project.parent.basedir}/codestyle/druid-forbidden-apis.txt - - **.SuppressForbidden - - - - compile - compile - - check - - - - - jdk-unsafe - jdk-system-out - - - - - testCompile - test-compile - - testCheck - - - - - - org.codehaus.mojo - animal-sniffer-maven-plugin - 1.15 - - - check-java-api - test - - check - - - - org.codehaus.mojo.signature - - java18 - 1.0 - - - - sun.nio.ch.DirectBuffer - sun.misc.Cleaner - sun.misc.Unsafe - - - - - - - org.apache.maven.plugins - maven-enforcer-plugin - 1.4.1 - - - enforce-banned-dependencies - - enforce - - - - - 1.8.0 - - - - - com.google.code.findbugs:annotations - - - - true - - - - - - org.apache.maven.plugins - maven-jar-plugin - - - - ${java.version} (${java.vendor} ${java.vm.version}) - ${os.name} ${os.arch} ${os.version} - ${git.build.time} - ${git.build.version} - ${git.commit.id} - ${git.commit.id.describe} - - - - - - pl.project13.maven - git-commit-id-plugin - 2.2.5 - - - - revision - - - - - ${project.basedir}/.git - Etc/UTC - false - json - true - ${project.build.directory}/git.version - false - - git.build.user.email - git.build.host - git.commit.id.describe-short - git.commit.user.* - git.commit.message.* - git.closest.tag.* - git.commit.id.abbrev - git.dirty - - - false - true - 7 - -dirty - true - - - - - - - - org.apache.maven.plugins - maven-surefire-plugin - 2.19.1 - - - - - -Xmx1500m - -XX:MaxDirectMemorySize=512m - -Duser.language=en - -Duser.GroupByQueryRunnerTest.javacountry=US - -Dfile.encoding=UTF-8 - -Duser.timezone=UTC - -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager - - -Ddruid.indexing.doubleStorage=double - - false - - true - - - - org.apache.maven.plugins - maven-release-plugin - - true - - - - org.apache.maven.plugins - maven-clean-plugin - 2.5 - - - org.apache.maven.plugins - maven-dependency-plugin - 2.8 - - - org.apache.maven.plugins - maven-deploy-plugin - 2.7 - - - org.apache.maven.plugins - maven-help-plugin - 2.1.1 - - - org.apache.maven.plugins - maven-install-plugin - 2.3.1 - - - package - - install - - - - - - org.apache.maven.plugins - maven-resources-plugin - 2.5 - - - org.apache.maven.plugins - maven-shade-plugin - 2.2 - - - org.apache.maven.plugins - maven-site-plugin - 3.1 - - - org.scala-tools - maven-scala-plugin - 2.15.2 - - - org.antlr - antlr4-maven-plugin - 4.5.1 - - - org.apache.maven.plugins - maven-assembly-plugin - 3.1.0 - - - org.codehaus.mojo - license-maven-plugin - 1.8 - - - org.codehaus.mojo - exec-maven-plugin - 1.2.1 - - - org.apache.maven.plugins - maven-javadoc-plugin - 2.10.1 - - - -Xdoclint:none + + + + org.codehaus.mojo + cobertura-maven-plugin + + + org.eluder.coveralls + coveralls-maven-plugin + 4.0.0 + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.17 + + ${project.build.sourceDirectory} + true + codestyle/checkstyle.xml + codestyle/checkstyle-suppressions.xml + checkstyle.suppressions.file + UTF-8 + codestyle/LICENSE.txt + true + true + + + + com.puppycrawl.tools + checkstyle + 8.0 + + + + + validate + validate + + check + + + + + + org.apache.maven.plugins + maven-pmd-plugin + 3.8 + + true + + /rulesets/java/imports.xml + + + target/generated-sources/ + + + + + validate + validate + + check + + + + + + de.thetaphi + forbiddenapis + 2.3 + + false + + + jdk-unsafe + + + ${project.parent.basedir}/codestyle/joda-time-forbidden-apis.txt + ${project.parent.basedir}/codestyle/druid-forbidden-apis.txt + + + **.SuppressForbidden + + + + + compile + compile + + check + + + + + jdk-unsafe + jdk-system-out + + + + + testCompile + test-compile + + testCheck + + + + + + org.codehaus.mojo + animal-sniffer-maven-plugin + 1.15 + + + check-java-api + test + + check + + + + org.codehaus.mojo.signature + + java18 + 1.0 + + + + sun.nio.ch.DirectBuffer + sun.misc.Cleaner + sun.misc.Unsafe + + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + 1.4.1 + + + enforce-banned-dependencies + + enforce + + + + + 1.8.0 + + + + + com.google.code.findbugs:annotations + + + + true + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + ${java.version} (${java.vendor} ${java.vm.version}) + ${os.name} ${os.arch} ${os.version} + ${git.build.time} + ${git.build.version} + ${git.commit.id} + ${git.commit.id.describe} + + + + + + pl.project13.maven + git-commit-id-plugin + 2.2.5 + + + + revision + + + + + ${project.basedir}/.git + Etc/UTC + false + json + true + ${project.build.directory}/git.version + false + + git.build.user.email + git.build.host + git.commit.id.describe-short + git.commit.user.* + git.commit.message.* + git.closest.tag.* + git.commit.id.abbrev + git.dirty + + + false + true + 7 + -dirty + true + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.19.1 + + + + + -Xmx1500m + -XX:MaxDirectMemorySize=512m + -Duser.language=en + -Duser.GroupByQueryRunnerTest.javacountry=US + -Dfile.encoding=UTF-8 + -Duser.timezone=UTC + -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager + + -Ddruid.indexing.doubleStorage=double + + false + + true + + + + org.apache.maven.plugins + maven-release-plugin + + true + + + + org.apache.maven.plugins + maven-clean-plugin + 2.5 + + + org.apache.maven.plugins + maven-dependency-plugin + 2.8 + + + org.apache.maven.plugins + maven-deploy-plugin + 2.7 + + + org.apache.maven.plugins + maven-help-plugin + 2.1.1 + + + org.apache.maven.plugins + maven-install-plugin + 2.3.1 + + + package + + install + + + + + + org.apache.maven.plugins + maven-resources-plugin + 2.5 + + + org.apache.maven.plugins + maven-shade-plugin + 2.2 + + + org.apache.maven.plugins + maven-site-plugin + 3.1 + + + org.scala-tools + maven-scala-plugin + 2.15.2 + + + org.antlr + antlr4-maven-plugin + 4.5.1 + + + org.apache.maven.plugins + maven-assembly-plugin + 3.1.0 + + + org.codehaus.mojo + license-maven-plugin + 1.8 + + + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.10.1 + + + -Xdoclint:none - - org.apache.hadoop.fs - - - - org.codehaus.mojo - cobertura-maven-plugin - 2.7 - - - - org.apache.druid.sql.antlr4.* - - - org/apache/druid/sql/antlr4/**/*.class - - - xml - - true - - - - org.apache.maven.plugins - maven-compiler-plugin - - ${maven.compiler.source} - ${maven.compiler.target} - - - - - maven-remote-resources-plugin - - - process-resource-bundles - none - - - - - - + + org.apache.hadoop.fs + + + + org.codehaus.mojo + cobertura-maven-plugin + 2.7 + + + + org.apache.druid.sql.antlr4.* + + + org/apache/druid/sql/antlr4/**/*.class + + + xml + + true + + + + org.apache.maven.plugins + maven-compiler-plugin + + ${maven.compiler.source} + ${maven.compiler.target} + + + + + maven-remote-resources-plugin + + + process-resource-bundles + none + + + + + + - - - strict - - - - org.apache.maven.plugins - maven-compiler-plugin - - javac-with-errorprone - true - true - 1024m - 3000m - ${maven.compiler.source} - ${maven.compiler.target} - false - - -XepDisableWarningsInGeneratedCode + + + strict + + + + org.apache.maven.plugins + maven-compiler-plugin + + javac-with-errorprone + true + true + 1024m + 3000m + ${maven.compiler.source} + ${maven.compiler.target} + false + + -XepDisableWarningsInGeneratedCode - -Xep:ClassCanBeStatic:ERROR - -Xep:PreconditionsInvalidPlaceholder:ERROR - -Xep:MissingOverride:ERROR - -Xep:DefaultCharset:ERROR + -Xep:ClassCanBeStatic:ERROR + -Xep:PreconditionsInvalidPlaceholder:ERROR + -Xep:MissingOverride:ERROR + -Xep:DefaultCharset:ERROR - -Xep:ArgumentParameterSwap - -Xep:AssistedInjectAndInjectOnSameConstructor - -Xep:AutoFactoryAtInject - -Xep:ClassName - -Xep:ComparisonContractViolated - -Xep:DepAnn - -Xep:DivZero - -Xep:EmptyIf - -Xep:InjectInvalidTargetingOnScopingAnnotation - -Xep:InjectMoreThanOneQualifier - -Xep:InjectScopeAnnotationOnInterfaceOrAbstractClass - -Xep:InjectScopeOrQualifierAnnotationRetention - -Xep:InjectedConstructorAnnotations - -Xep:InsecureCryptoUsage - -Xep:JMockTestWithoutRunWithOrRuleAnnotation - -Xep:JavaxInjectOnFinalField - -Xep:LockMethodChecker - -Xep:LongLiteralLowerCaseSuffix - -Xep:NoAllocation - -Xep:NonRuntimeAnnotation - -Xep:NumericEquality - -Xep:ParameterPackage - -Xep:ProtoStringFieldReferenceEquality - -Xep:QualifierOnMethodWithoutProvides - -Xep:UnlockMethod - - - - - org.codehaus.plexus - plexus-compiler-javac-errorprone - 2.8.1 - - - - com.google.errorprone - error_prone_core - 2.0.19 - - - - - - - - parallel-test - - false - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - test - - test - - - - - ${maven.fork.count} - true - false - - - -Xmx768m -Duser.language=en -Duser.country=US -Dfile.encoding=UTF-8 - -Duser.timezone=UTC -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager - - -Ddruid.indexing.doubleStorage=double - - - true - - - - - - - - rat - - false - - - - - org.apache.rat - apache-rat-plugin - 0.12 - - - verify - - check - - - - - ${project.basedir}/rat - - - MIT - MIT JQuery - - - Copyright 2012 jQuery Foundation and other contributors; Licensed MIT - jQuery Foundation, Inc. | jquery.org/license - - - - Underscore - Underscore - - - Underscore is freely distributable under the MIT license - - - - Allan Jardine - Allan Jardine - - - Copyright 2009 Allan Jardine. All Rights Reserved - - - - Allan Jardine - Allan Jardine - - - Copyright 2009 Allan Jardine. All Rights Reserved - Copyright 2008-2011 Allan Jardine - GPL v2 or BSD 3 point style - - - + -Xep:ArgumentParameterSwap + -Xep:AssistedInjectAndInjectOnSameConstructor + -Xep:AutoFactoryAtInject + -Xep:ClassName + -Xep:ComparisonContractViolated + -Xep:DepAnn + -Xep:DivZero + -Xep:EmptyIf + -Xep:InjectInvalidTargetingOnScopingAnnotation + -Xep:InjectMoreThanOneQualifier + -Xep:InjectScopeAnnotationOnInterfaceOrAbstractClass + -Xep:InjectScopeOrQualifierAnnotationRetention + -Xep:InjectedConstructorAnnotations + -Xep:InsecureCryptoUsage + -Xep:JMockTestWithoutRunWithOrRuleAnnotation + -Xep:JavaxInjectOnFinalField + -Xep:LockMethodChecker + -Xep:LongLiteralLowerCaseSuffix + -Xep:NoAllocation + -Xep:NonRuntimeAnnotation + -Xep:NumericEquality + -Xep:ParameterPackage + -Xep:ProtoStringFieldReferenceEquality + -Xep:QualifierOnMethodWithoutProvides + -Xep:UnlockMethod + + + + + org.codehaus.plexus + plexus-compiler-javac-errorprone + 2.8.1 + + + + com.google.errorprone + error_prone_core + 2.0.19 + + + + + + + + parallel-test + + false + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + test + + test + + + + + ${maven.fork.count} + true + false + + + -Xmx768m -Duser.language=en -Duser.country=US -Dfile.encoding=UTF-8 + -Duser.timezone=UTC -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager + + -Ddruid.indexing.doubleStorage=double + + + true + + + + + + + + rat + + false + + + + + org.apache.rat + apache-rat-plugin + 0.12 + + + verify + + check + + + + + ${project.basedir}/rat + + + MIT + MIT JQuery + + + Copyright 2012 jQuery Foundation and other contributors; Licensed MIT + jQuery Foundation, Inc. | jquery.org/license + + + + Underscore + Underscore + + + Underscore is freely distributable under the MIT license + + + + Allan Jardine + Allan Jardine + + + Copyright 2009 Allan Jardine. All Rights Reserved + + + + Allan Jardine + Allan Jardine + + + Copyright 2009 Allan Jardine. All Rights Reserved + Copyright 2008-2011 Allan Jardine + GPL v2 or BSD 3 point style + + + - - - MIT JQuery - - - Underscore - - - Allan Jardine - - - - - **/*.md - publications/** - docs/** - - codestyle/* - eclipse.importorder - - **/javax.annotation.processing.Processor - **/org.apache.druid.initialization.DruidModule - **/org/apache/druid/math/expr/antlr/Expr.g4 - **/dependency-reduced-pom.xml - - **/target/** - **/build/** - **/test/resources/** - **/src/test/avro/** - **/src/test/thrift/** - .travis.yml - - **/*.json - **/jvm.config - **/quickstart/protobuf/** - **/tutorial/conf/** - **/derby.log - **/docker/** - **/client_tls/** - - **/*.iml - - **/hs_err_pid*.log - - - - - - - - - apache-release - - - - maven-assembly-plugin - - - source-release-assembly - none - - - - - - - + + + MIT JQuery + + + Underscore + + + Allan Jardine + + + + + **/*.md + publications/** + docs/** + + codestyle/* + eclipse.importorder + + **/javax.annotation.processing.Processor + **/org.apache.druid.initialization.DruidModule + **/org/apache/druid/math/expr/antlr/Expr.g4 + **/dependency-reduced-pom.xml + + **/target/** + **/build/** + **/test/resources/** + **/src/test/avro/** + **/src/test/thrift/** + .travis.yml + + **/*.json + **/jvm.config + **/quickstart/protobuf/** + **/tutorial/conf/** + **/derby.log + **/docker/** + **/client_tls/** + + **/*.iml + + **/hs_err_pid*.log + + + + + + + + + apache-release + + + + maven-assembly-plugin + + + source-release-assembly + none + + + + + + + From a371bbd0952c5aa12d2ace6b1fe8f62fbdeb0431 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 29 Oct 2018 14:47:36 -0700 Subject: [PATCH 33/87] capitalized --- .../SeekableStreamDataSourceMetadata.java | 26 +- .../SeekableStreamIOConfig.java | 16 +- .../SeekableStreamIndexTask.java | 6 +- .../SeekableStreamIndexTaskClient.java | 22 +- .../SeekableStreamIndexTaskRunner.java | 12 +- .../SeekableStreamPartitions.java | 20 +- .../common/OrderedPartitionableRecord.java | 20 +- .../seekablestream/common/RecordSupplier.java | 28 +- .../common/StreamPartition.java | 12 +- .../supervisor/SeekableStreamSupervisor.java | 254 +++++++++--------- ...SeekableStreamSupervisorReportPayload.java | 14 +- .../supervisor/TaskReportData.java | 22 +- 12 files changed, 226 insertions(+), 226 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java index 76d3df5efa3a..db0169653578 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java @@ -28,20 +28,20 @@ import java.util.Map; import java.util.Objects; -public abstract class SeekableStreamDataSourceMetadata implements DataSourceMetadata +public abstract class SeekableStreamDataSourceMetadata implements DataSourceMetadata { - private final SeekableStreamPartitions seekableStreamPartitions; + private final SeekableStreamPartitions seekableStreamPartitions; @JsonCreator public SeekableStreamDataSourceMetadata( - @JsonProperty("partitions") SeekableStreamPartitions seekableStreamPartitions + @JsonProperty("partitions") SeekableStreamPartitions seekableStreamPartitions ) { this.seekableStreamPartitions = seekableStreamPartitions; } @JsonProperty("partitions") - public SeekableStreamPartitions getSeekableStreamPartitions() + public SeekableStreamPartitions getSeekableStreamPartitions() { return seekableStreamPartitions; } @@ -75,17 +75,17 @@ public DataSourceMetadata plus(DataSourceMetadata other) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; + final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; if (that.getSeekableStreamPartitions().getName().equals(seekableStreamPartitions.getName())) { // Same topic, merge offsets. - final Map newMap = Maps.newHashMap(); + final Map newMap = Maps.newHashMap(); - for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { + for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { newMap.put(entry.getKey(), entry.getValue()); } - for (Map.Entry entry : that.getSeekableStreamPartitions().getMap().entrySet()) { + for (Map.Entry entry : that.getSeekableStreamPartitions().getMap().entrySet()) { newMap.put(entry.getKey(), entry.getValue()); } @@ -109,13 +109,13 @@ public DataSourceMetadata minus(DataSourceMetadata other) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; + final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; if (that.getSeekableStreamPartitions().getName().equals(seekableStreamPartitions.getName())) { // Same stream, remove partitions present in "that" from "this" - final Map newMap = Maps.newHashMap(); + final Map newMap = Maps.newHashMap(); - for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { + for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { if (!that.getSeekableStreamPartitions().getMap().containsKey(entry.getKey())) { newMap.put(entry.getKey(), entry.getValue()); } @@ -155,8 +155,8 @@ public String toString() '}'; } - protected abstract SeekableStreamDataSourceMetadata createConcretDataSourceMetaData( + protected abstract SeekableStreamDataSourceMetadata createConcretDataSourceMetaData( String streamId, - Map newMap + Map newMap ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java index afdcf45d0f05..7ce86cc215ff 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java @@ -29,15 +29,15 @@ import javax.annotation.Nullable; import java.util.Set; -public abstract class SeekableStreamIOConfig implements IOConfig +public abstract class SeekableStreamIOConfig implements IOConfig { private static final boolean DEFAULT_USE_TRANSACTION = true; @Nullable private final Integer taskGroupId; private final String baseSequenceName; - private final SeekableStreamPartitions startPartitions; - private final SeekableStreamPartitions endPartitions; + private final SeekableStreamPartitions startPartitions; + private final SeekableStreamPartitions endPartitions; private final boolean useTransaction; private final Optional minimumMessageTime; private final Optional maximumMessageTime; @@ -46,8 +46,8 @@ public abstract class SeekableStreamIOConfig implem public SeekableStreamIOConfig( @JsonProperty("taskGroupId") @Nullable Integer taskGroupId, // can be null for backward compabitility @JsonProperty("baseSequenceName") String baseSequenceName, - @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, - @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, + @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, + @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, @JsonProperty("useTransaction") Boolean useTransaction, @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, @JsonProperty("maximumMessageTime") DateTime maximumMessageTime @@ -86,13 +86,13 @@ public String getBaseSequenceName() } @JsonProperty - public SeekableStreamPartitions getStartPartitions() + public SeekableStreamPartitions getStartPartitions() { return startPartitions; } @JsonProperty - public SeekableStreamPartitions getEndPartitions() + public SeekableStreamPartitions getEndPartitions() { return endPartitions; } @@ -116,7 +116,7 @@ public Optional getMinimumMessageTime() } @JsonProperty - public abstract Set getExclusiveStartSequenceNumberPartitions(); + public abstract Set getExclusiveStartSequenceNumberPartitions(); @Override public abstract String toString(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 3d8cc65be427..e53c0f08dc64 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -48,14 +48,14 @@ import java.util.concurrent.ThreadLocalRandom; -public abstract class SeekableStreamIndexTask extends AbstractTask +public abstract class SeekableStreamIndexTask extends AbstractTask implements ChatHandler { private static final Random RANDOM = ThreadLocalRandom.current(); protected final DataSchema dataSchema; protected final InputRowParser parser; protected final SeekableStreamTuningConfig tuningConfig; - protected final SeekableStreamIOConfig ioConfig; + protected final SeekableStreamIOConfig ioConfig; protected final Optional chatHandlerProvider; protected final String type; protected CircularBuffer savedParseExceptions; @@ -134,7 +134,7 @@ public SeekableStreamTuningConfig getTuningConfig() } @JsonProperty("ioConfig") - public SeekableStreamIOConfig getIOConfig() + public SeekableStreamIOConfig getIOConfig() { return ioConfig; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 5032d161ff35..ded1b6e98aff 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -43,7 +43,7 @@ import java.util.Map; import java.util.TreeMap; -public abstract class SeekableStreamIndexTaskClient extends IndexTaskClient +public abstract class SeekableStreamIndexTaskClient extends IndexTaskClient { private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTaskClient.class); @@ -102,7 +102,7 @@ public boolean resume(final String id) } - public Map pause(final String id) + public Map pause(final String id) { log.debug("Pause task[%s]", id); @@ -212,7 +212,7 @@ public Map getMovingAverages(final String id) } } - public Map getCurrentOffsets(final String id, final boolean retry) + public Map getCurrentOffsets(final String id, final boolean retry) { log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry); @@ -234,7 +234,7 @@ public Map getCurrentOffsets(final String id, final } } - public TreeMap> getCheckpoints(final String id, final boolean retry) + public TreeMap> getCheckpoints(final String id, final boolean retry) { log.debug("GetCheckpoints task[%s] retry[%s]", id, retry); try { @@ -252,7 +252,7 @@ public TreeMap> getCheckpoints(final S } } - public ListenableFuture>> getCheckpointsAsync( + public ListenableFuture>> getCheckpointsAsync( final String id, final boolean retry ) @@ -260,7 +260,7 @@ public ListenableFuture>> getC return doAsync(() -> getCheckpoints(id, retry)); } - public Map getEndOffsets(final String id) + public Map getEndOffsets(final String id) { log.debug("GetEndOffsets task[%s]", id); @@ -278,7 +278,7 @@ public Map getEndOffsets(final String id) public boolean setEndOffsets( final String id, - final Map endOffsets, + final Map endOffsets, final boolean finalize ) throws IOException { @@ -318,26 +318,26 @@ public ListenableFuture getStartTimeAsync(final String id) } - public ListenableFuture> pauseAsync(final String id) + public ListenableFuture> pauseAsync(final String id) { return doAsync(() -> pause(id)); } public ListenableFuture setEndOffsetsAsync( final String id, - final Map endOffsets, + final Map endOffsets, final boolean finalize ) { return doAsync(() -> setEndOffsets(id, endOffsets, finalize)); } - public ListenableFuture> getCurrentOffsetsAsync(final String id, final boolean retry) + public ListenableFuture> getCurrentOffsetsAsync(final String id, final boolean retry) { return doAsync(() -> getCurrentOffsets(id, retry)); } - public ListenableFuture> getEndOffsetsAsync(final String id) + public ListenableFuture> getEndOffsetsAsync(final String id) { return doAsync(() -> getEndOffsets(id)); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index e4358d159586..50bd9599f9da 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -33,10 +33,10 @@ * Interface for abstracting the indexing task run logic. Only used by Kafka indexing tasks, * but will also be used by Kinesis indexing tasks once implemented * - * @param Partition Number Type - * @param Sequence Number Type + * @param Partition Number Type + * @param Sequence Number Type */ -public interface SeekableStreamIndexTaskRunner extends ChatHandler +public interface SeekableStreamIndexTaskRunner extends ChatHandler { Appenderator getAppenderator(); @@ -51,14 +51,14 @@ public interface SeekableStreamIndexTaskRunner exte SeekableStreamIndexTask.Status getStatus(); @VisibleForTesting - Map getCurrentOffsets(); + Map getCurrentOffsets(); @VisibleForTesting - Map getEndOffsets(); + Map getEndOffsets(); @VisibleForTesting Response setEndOffsets( - Map offsets, + Map offsets, boolean finish // this field is only for internal purposes, shouldn't be usually set by users ) throws InterruptedException; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index c916d6c74023..27d406a51db6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -40,24 +40,24 @@ * are used for proper Jackson serialization/deserialization when processing terminologies * used by Kafka and kinesis (i.e. topic vs. name) * - * @param partition id type - * @param sequence number type + * @param partition id type + * @param sequence number type */ -public class SeekableStreamPartitions +public class SeekableStreamPartitions { public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER"; // stream/topic private final String name; // partitionId -> sequence number - private final Map map; + private final Map map; @JsonCreator public SeekableStreamPartitions( @JsonProperty("stream") final String stream, @JsonProperty("topic") final String topic, - @JsonProperty("partitionSequenceNumberMap") final Map partitionSequenceNumberMap, - @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap + @JsonProperty("partitionSequenceNumberMap") final Map partitionSequenceNumberMap, + @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap ) { this.name = stream == null ? topic : stream; @@ -69,7 +69,7 @@ public SeekableStreamPartitions( } // constructor for backward compatibility - public SeekableStreamPartitions(@NotNull final String id, final Map partitionOffsetMap) + public SeekableStreamPartitions(@NotNull final String id, final Map partitionOffsetMap) { this(id, null, partitionOffsetMap, null); } @@ -86,19 +86,19 @@ public String getTopic() return name; } - public Map getMap() + public Map getMap() { return map; } @JsonProperty - public Map getPartitionSequenceNumberMap() + public Map getPartitionSequenceNumberMap() { return map; } @JsonProperty - public Map getPartitionOffsetMap() + public Map getPartitionOffsetMap() { return map; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java index 6d98864274a6..e5114ef05558 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java @@ -23,25 +23,25 @@ import java.util.Objects; /** - * Represents a generic record with a partitionType (partition id) and sequenceType (sequence number) and data + * Represents a generic record with a PartitionType (partition id) and SequenceType (sequence number) and data * from a Kafka/Kinesis stream * - * @param partition id - * @param sequence number + * @param partition id + * @param sequence number */ -public class OrderedPartitionableRecord +public class OrderedPartitionableRecord { public static final String END_OF_SHARD_MARKER = "EOS"; private final String stream; - private final partitionType partitionId; - private final sequenceType sequenceNumber; + private final PartitionType partitionId; + private final SequenceType sequenceNumber; private final List data; public OrderedPartitionableRecord( String stream, - partitionType partitionId, - sequenceType sequenceNumber, + PartitionType partitionId, + SequenceType sequenceNumber, List data ) { @@ -56,12 +56,12 @@ public String getStream() return stream; } - public partitionType getPartitionId() + public PartitionType getPartitionId() { return partitionId; } - public sequenceType getSequenceNumber() + public SequenceType getSequenceNumber() { return sequenceNumber; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java index 8568fb1cb382..ee03ed3d2b84 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java @@ -31,13 +31,13 @@ * The RecordSupplier interface is a wrapper for the incoming seekable data stream * (i.e. Kafka consumer, Kinesis streams) * - * @param Partition Number Type - * @param Sequence Number Type + * @param Partition Number Type + * @param Sequence Number Type */ @Beta -public interface RecordSupplier extends Closeable +public interface RecordSupplier extends Closeable { - void assign(Set> partitions); + void assign(Set> partitions); /** * seek to specified sequence number within a specific partition @@ -45,7 +45,7 @@ public interface RecordSupplier extends Closeable * @param partition partition to seek * @param sequenceNumber sequence number to seek to */ - void seek(StreamPartition partition, sequenceType sequenceNumber); + void seek(StreamPartition partition, SequenceType sequenceNumber); /** * seek to the sequence number immediately following the given sequenceNumber @@ -53,28 +53,28 @@ public interface RecordSupplier extends Closeable * @param partition partition to seek * @param sequenceNumber sequence number to seek */ - void seekAfter(StreamPartition partition, sequenceType sequenceNumber); + void seekAfter(StreamPartition partition, SequenceType sequenceNumber); /** * seek a set of partitions to the earliest record position available in the stream * * @param partitions partitions to seek */ - void seekToEarliest(Set> partitions); + void seekToEarliest(Set> partitions); /** * seek a set of partitions to the latest/newest record position available in the stream * * @param partitions partitions to seek */ - void seekToLatest(Set> partitions); + void seekToLatest(Set> partitions); /** * get the current assignment * * @return set of assignments */ - Collection> getAssignment(); + Collection> getAssignment(); /** * poll the record at the current seeked to sequence in stream @@ -84,7 +84,7 @@ public interface RecordSupplier extends Closeable * @return record */ @Nullable - OrderedPartitionableRecord poll(long timeout); + OrderedPartitionableRecord poll(long timeout); /** * get the latest sequence number in stream @@ -95,7 +95,7 @@ public interface RecordSupplier extends Closeable * * @throws TimeoutException TimeoutException */ - sequenceType getLatestSequenceNumber(StreamPartition partition) throws TimeoutException; + SequenceType getLatestSequenceNumber(StreamPartition partition) throws TimeoutException; /** * get the earliest sequence number in stream @@ -106,7 +106,7 @@ public interface RecordSupplier extends Closeable * * @throws TimeoutException TimeoutException */ - sequenceType getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException; + SequenceType getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException; /** * returns the sequence number that the given partition is currently at @@ -115,7 +115,7 @@ public interface RecordSupplier extends Closeable * * @return sequence number */ - sequenceType position(StreamPartition partition); + SequenceType position(StreamPartition partition); /** * returns the set of partitions under the given stream @@ -124,7 +124,7 @@ public interface RecordSupplier extends Closeable * * @return set of partitions */ - Set getPartitionIds(String stream); + Set getPartitionIds(String stream); /** * close the RecordSupplier diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java index 406797f0718f..93e8e1d931ba 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java @@ -24,20 +24,20 @@ * mostly used by {@link RecordSupplier} and * {@link org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor} * - * @param partition id type + * @param partition id type */ -public class StreamPartition +public class StreamPartition { private final String stream; - private final partitionType partitionId; + private final PartitionType partitionId; - public StreamPartition(String stream, partitionType partitionId) + public StreamPartition(String stream, PartitionType partitionId) { this.stream = stream; this.partitionId = partitionId; } - public static StreamPartition of(String stream, partitionType partitionId) + public static StreamPartition of(String stream, PartitionType partitionId) { return new StreamPartition<>(stream, partitionId); } @@ -47,7 +47,7 @@ public String getStream() return stream; } - public partitionType getPartitionId() + public PartitionType getPartitionId() { return partitionId; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index a39dba760815..563994999523 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -123,10 +123,10 @@ * stream sequences. *

* - * @param partition id type - * @param sequence number type + * @param partition id type + * @param sequence number type */ -public abstract class SeekableStreamSupervisor +public abstract class SeekableStreamSupervisor implements Supervisor { public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED"; @@ -157,22 +157,22 @@ private class TaskGroup // this task group has completed successfully, at which point this will be destroyed and a new task group will be // created with new starting offsets. This allows us to create replacement tasks for failed tasks that process the // same offsets, even if the values in [partitionGroups] has been changed. - final ImmutableMap startingSequences; + final ImmutableMap startingSequences; final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); final Optional minimumMessageTime; final Optional maximumMessageTime; - final Set exclusiveStartSequenceNumberPartitions; - final TreeMap> checkpointSequences = new TreeMap<>(); + final Set exclusiveStartSequenceNumberPartitions; + final TreeMap> checkpointSequences = new TreeMap<>(); final String baseSequenceName; DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action TaskGroup( int groupId, - ImmutableMap startingSequences, + ImmutableMap startingSequences, Optional minimumMessageTime, Optional maximumMessageTime, - Set exclusiveStartSequenceNumberPartitions + Set exclusiveStartSequenceNumberPartitions ) { this.groupId = groupId; @@ -186,7 +186,7 @@ private class TaskGroup this.baseSequenceName = generateSequenceName(startingSequences, minimumMessageTime, maximumMessageTime); } - int addNewCheckpoint(Map checkpoint) + int addNewCheckpoint(Map checkpoint) { checkpointSequences.put(checkpointSequences.lastKey() + 1, checkpoint); return checkpointSequences.lastKey(); @@ -203,7 +203,7 @@ private class TaskData { volatile TaskStatus status; volatile DateTime startTime; - volatile Map currentSequences = new HashMap<>(); + volatile Map currentSequences = new HashMap<>(); @Override public String toString() @@ -319,14 +319,14 @@ protected class CheckpointNotice implements Notice private final Integer nullableTaskGroupId; @Deprecated private final String baseSequenceName; - private final SeekableStreamDataSourceMetadata previousCheckpoint; - private final SeekableStreamDataSourceMetadata currentCheckpoint; + private final SeekableStreamDataSourceMetadata previousCheckpoint; + private final SeekableStreamDataSourceMetadata currentCheckpoint; public CheckpointNotice( @Nullable Integer nullableTaskGroupId, @Deprecated String baseSequenceName, - SeekableStreamDataSourceMetadata previousCheckpoint, - SeekableStreamDataSourceMetadata currentCheckpoint + SeekableStreamDataSourceMetadata previousCheckpoint, + SeekableStreamDataSourceMetadata currentCheckpoint ) { this.baseSequenceName = baseSequenceName; @@ -376,12 +376,12 @@ public void handle() throws ExecutionException, InterruptedException final TaskGroup taskGroup = taskGroups.get(taskGroupId); if (isValidTaskGroup(taskGroupId, taskGroup)) { - final TreeMap> checkpoints = taskGroup.checkpointSequences; + final TreeMap> checkpoints = taskGroup.checkpointSequences; // check validity of previousCheckpoint int index = checkpoints.size(); for (int sequenceId : checkpoints.descendingKeySet()) { - Map checkpoint = checkpoints.get(sequenceId); + Map checkpoint = checkpoints.get(sequenceId); // We have already verified the topic of the current checkpoint is same with that in ioConfig. // See checkpoint(). if (checkpoint.equals(previousCheckpoint.getSeekableStreamPartitions() @@ -399,7 +399,7 @@ public void handle() throws ExecutionException, InterruptedException log.info("Already checkpointed with offsets [%s]", checkpoints.lastEntry().getValue()); return; } - final Map newCheckpoint = checkpointTaskGroup(taskGroup, false).get(); + final Map newCheckpoint = checkpointTaskGroup(taskGroup, false).get(); taskGroup.addNewCheckpoint(newCheckpoint); log.info("Handled checkpoint notice, new checkpoint is [%s] for taskGroup [%s]", newCheckpoint, taskGroupId); } @@ -448,19 +448,19 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task // failures during publishing. // Map<{group ID}, Map<{partition ID}, {startingOffset}>> - private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); + private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); protected final ObjectMapper sortingMapper; - protected final List partitionIds = new CopyOnWriteArrayList<>(); - protected volatile Map latestSequenceFromStream; + protected final List partitionIds = new CopyOnWriteArrayList<>(); + protected volatile Map latestSequenceFromStream; protected volatile DateTime sequenceLastUpdated; - private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); + private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); private final TaskStorage taskStorage; private final TaskMaster taskMaster; private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; - private final SeekableStreamIndexTaskClient taskClient; + private final SeekableStreamIndexTaskClient taskClient; private final SeekableStreamSupervisorSpec spec; private final String dataSource; private final SeekableStreamSupervisorIOConfig ioConfig; @@ -478,15 +478,15 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) private final Object stopLock = new Object(); private final Object stateChangeLock = new Object(); private final Object recordSupplierLock = new Object(); - private final sequenceType END_OF_PARTITION; - private final sequenceType NOT_SET; + private final SequenceType END_OF_PARTITION; + private final SequenceType NOT_SET; private final boolean useExclusiveStartingSequence; private final boolean isCheckpointSupported; private boolean listenerRegistered = false; private long lastRunTime; private int initRetryCounter = 0; private volatile DateTime firstRunTime; - private volatile RecordSupplier recordSupplier; + private volatile RecordSupplier recordSupplier; private volatile boolean started = false; private volatile boolean stopped = false; private volatile boolean lifecycleStarted = false; @@ -497,12 +497,12 @@ public SeekableStreamSupervisor( final TaskStorage taskStorage, final TaskMaster taskMaster, final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, - final SeekableStreamIndexTaskClientFactory> taskClientFactory, + final SeekableStreamIndexTaskClientFactory> taskClientFactory, final ObjectMapper mapper, final SeekableStreamSupervisorSpec spec, final RowIngestionMetersFactory rowIngestionMetersFactory, - final sequenceType NOT_SET, - final sequenceType END_OF_PARTITION, + final SequenceType NOT_SET, + final SequenceType END_OF_PARTITION, final boolean useExclusiveStartingSequence, final boolean isCheckpointSupported ) @@ -804,24 +804,24 @@ public SupervisorReport getStatus() return generateReport(true); } - private SupervisorReport> generateReport( + private SupervisorReport> generateReport( boolean includeOffsets ) { int numPartitions = partitionGroups.values().stream().mapToInt(Map::size).sum(); - final SeekableStreamSupervisorReportPayload payload = createReportPayload( + final SeekableStreamSupervisorReportPayload payload = createReportPayload( numPartitions, includeOffsets ); - SupervisorReport> report = new SupervisorReport<>( + SupervisorReport> report = new SupervisorReport<>( dataSource, DateTimes.nowUtc(), payload ); - List> taskReports = Lists.newArrayList(); + List> taskReports = Lists.newArrayList(); try { for (TaskGroup taskGroup : taskGroups.values()) { @@ -829,7 +829,7 @@ private SupervisorReport currentOffsets = entry.getValue().currentSequences; + Map currentOffsets = entry.getValue().currentSequences; Long remainingSeconds = null; if (startTime != null) { long elapsedMillis = System.currentTimeMillis() - startTime.getMillis(); @@ -857,7 +857,7 @@ private SupervisorReport currentOffsets = entry.getValue().currentSequences; + Map currentOffsets = entry.getValue().currentSequences; Long remainingSeconds = null; if (taskGroup.completionTimeout != null) { remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - System.currentTimeMillis()) @@ -977,11 +977,11 @@ private Map> getCurrentTotalStats() @VisibleForTesting protected void addTaskGroupToActivelyReadingTaskGroup( int taskGroupId, - ImmutableMap partitionOffsets, + ImmutableMap partitionOffsets, Optional minMsgTime, Optional maxMsgTime, Set tasks, - Set exclusiveStartingSequencePartitions + Set exclusiveStartingSequencePartitions ) { TaskGroup group = new TaskGroup( @@ -1003,11 +1003,11 @@ protected void addTaskGroupToActivelyReadingTaskGroup( @VisibleForTesting protected void addTaskGroupToPendingCompletionTaskGroup( int taskGroupId, - ImmutableMap partitionOffsets, + ImmutableMap partitionOffsets, Optional minMsgTime, Optional maxMsgTime, Set tasks, - Set exclusiveStartingSequencePartitions + Set exclusiveStartingSequencePartitions ) { TaskGroup group = new TaskGroup( @@ -1126,7 +1126,7 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) } // Reset only the partitions in dataSourceMetadata if it has not been reset yet @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata resetMetadata = (SeekableStreamDataSourceMetadata) dataSourceMetadata; + final SeekableStreamDataSourceMetadata resetMetadata = (SeekableStreamDataSourceMetadata) dataSourceMetadata; if (resetMetadata.getSeekableStreamPartitions().getName().equals(ioConfig.getId())) { // metadata can be null @@ -1139,15 +1139,15 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata currentMetadata = (SeekableStreamDataSourceMetadata) metadata; + final SeekableStreamDataSourceMetadata currentMetadata = (SeekableStreamDataSourceMetadata) metadata; // defend against consecutive reset requests from replicas // as well as the case where the metadata store do not have an entry for the reset partitions boolean doReset = false; - for (Entry resetPartitionOffset : resetMetadata.getSeekableStreamPartitions() + for (Entry resetPartitionOffset : resetMetadata.getSeekableStreamPartitions() .getMap() .entrySet()) { - final sequenceType partitionOffsetInMetadataStore = currentMetadata == null + final SequenceType partitionOffsetInMetadataStore = currentMetadata == null ? null : currentMetadata.getSeekableStreamPartitions() .getMap() @@ -1224,9 +1224,9 @@ private void killTasksInGroup(TaskGroup taskGroup) } } - private void killTaskGroupForPartitions(Set partitions) + private void killTaskGroupForPartitions(Set partitions) { - for (partitionType partition : partitions) { + for (PartitionType partition : partitions) { int taskGroupId = getTaskGroupIdForPartition(partition); killTasksInGroup(taskGroups.get(taskGroupId)); partitionGroups.remove(taskGroupId); @@ -1261,7 +1261,7 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti taskCount++; @SuppressWarnings("unchecked") - final SeekableStreamIndexTask seekableStreamIndexTask = (SeekableStreamIndexTask) task; + final SeekableStreamIndexTask seekableStreamIndexTask = (SeekableStreamIndexTask) task; final String taskId = task.getId(); // Determine which task group this task belongs to based on one of the partitions handled by this task. If we @@ -1271,7 +1271,7 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti // state, we will permit it to complete even if it doesn't match our current partition allocation to support // seamless schema migration. - Iterator it = seekableStreamIndexTask.getIOConfig() + Iterator it = seekableStreamIndexTask.getIOConfig() .getStartPartitions() .getMap() .keySet() @@ -1310,19 +1310,19 @@ public Boolean apply(SeekableStreamIndexTask.Status status) // update partitionGroups with the publishing task's offsets (if they are greater than what is // existing) so that the next tasks will start reading from where this task left off - Map publishingTaskEndOffsets = taskClient.getEndOffsets(taskId); + Map publishingTaskEndOffsets = taskClient.getEndOffsets(taskId); - for (Entry entry : publishingTaskEndOffsets.entrySet()) { - partitionType partition = entry.getKey(); - sequenceType offset = entry.getValue(); - ConcurrentHashMap partitionOffsets = partitionGroups.get( + for (Entry entry : publishingTaskEndOffsets.entrySet()) { + PartitionType partition = entry.getKey(); + SequenceType offset = entry.getValue(); + ConcurrentHashMap partitionOffsets = partitionGroups.get( getTaskGroupIdForPartition(partition) ); boolean succeeded; do { succeeded = true; - sequenceType previousOffset = partitionOffsets.putIfAbsent(partition, offset); + SequenceType previousOffset = partitionOffsets.putIfAbsent(partition, offset); if (previousOffset != null && (makeSequenceNumber(previousOffset).compareTo(makeSequenceNumber(offset))) < 0) { succeeded = partitionOffsets.replace(partition, previousOffset, offset); @@ -1330,7 +1330,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) } while (!succeeded); } } else { - for (partitionType partition : seekableStreamIndexTask.getIOConfig() + for (PartitionType partition : seekableStreamIndexTask.getIOConfig() .getStartPartitions() .getMap() .keySet()) { @@ -1448,12 +1448,12 @@ private void verifyAndMergeCheckpoints(final Collection taskGroupsToV private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) { final int groupId = taskGroup.groupId; - final List>>> taskSequences = new ArrayList<>(); - final List>>> futures = new ArrayList<>(); + final List>>> taskSequences = new ArrayList<>(); + final List>>> futures = new ArrayList<>(); final List taskIds = new ArrayList<>(); for (String taskId : taskGroup.taskIds()) { - final ListenableFuture>> checkpointsFuture = taskClient.getCheckpointsAsync( + final ListenableFuture>> checkpointsFuture = taskClient.getCheckpointsAsync( taskId, true ); @@ -1462,10 +1462,10 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) } try { - List>> futuresResult = + List>> futuresResult = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); for (int i = 0; i < futuresResult.size(); i++) { - final TreeMap> checkpoints = futuresResult.get(i); + final TreeMap> checkpoints = futuresResult.get(i); final String taskId = taskIds.get(i); if (checkpoints == null) { try { @@ -1498,14 +1498,14 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata latestDataSourceMetadata = (SeekableStreamDataSourceMetadata) rawDataSourceMetadata; + final SeekableStreamDataSourceMetadata latestDataSourceMetadata = (SeekableStreamDataSourceMetadata) rawDataSourceMetadata; final boolean hasValidOffsetsFromDb = latestDataSourceMetadata != null && latestDataSourceMetadata.getSeekableStreamPartitions() != null && ioConfig.getId().equals( latestDataSourceMetadata.getSeekableStreamPartitions().getName() ); - final Map latestOffsetsFromDb; + final Map latestOffsetsFromDb; if (hasValidOffsetsFromDb) { latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamPartitions().getMap(); } else { @@ -1520,7 +1520,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) int taskIndex = 0; while (taskIndex < taskSequences.size()) { - TreeMap> taskCheckpoints = taskSequences.get(taskIndex).rhs; + TreeMap> taskCheckpoints = taskSequences.get(taskIndex).rhs; String taskId = taskSequences.get(taskIndex).lhs; if (earliestConsistentSequenceId.get() == -1) { // find the first replica task with earliest sequenceId consistent with datasource metadata in the metadata @@ -1528,8 +1528,8 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) if (taskCheckpoints.entrySet().stream().anyMatch( sequenceCheckpoint -> sequenceCheckpoint.getValue().entrySet().stream().allMatch( partitionOffset -> { - OrderedSequenceNumber offset = makeSequenceNumber(partitionOffset.getValue()); - OrderedSequenceNumber latestOffset = makeSequenceNumber( + OrderedSequenceNumber offset = makeSequenceNumber(partitionOffset.getValue()); + OrderedSequenceNumber latestOffset = makeSequenceNumber( latestOffsetsFromDb == null ? partitionOffset.getValue() : latestOffsetsFromDb.getOrDefault( partitionOffset @@ -1544,7 +1544,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) ) && earliestConsistentSequenceId.compareAndSet(-1, sequenceCheckpoint.getKey())) || ( pendingCompletionTaskGroups.getOrDefault(groupId, EMPTY_LIST).size() > 0 && earliestConsistentSequenceId.compareAndSet(-1, taskCheckpoints.firstKey()))) { - final SortedMap> latestCheckpoints = new TreeMap<>( + final SortedMap> latestCheckpoints = new TreeMap<>( taskCheckpoints.tailMap(earliestConsistentSequenceId.get()) ); log.info("Setting taskGroup sequences to [%s] for group [%d]", latestCheckpoints, groupId); @@ -1606,7 +1606,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) private void addDiscoveredTaskToPendingCompletionTaskGroups( int groupId, String taskId, - Map startingPartitions + Map startingPartitions ) { final CopyOnWriteArrayList taskGroupList = pendingCompletionTaskGroups.computeIfAbsent( @@ -1667,7 +1667,7 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) } @SuppressWarnings("unchecked") - SeekableStreamIndexTask task = (SeekableStreamIndexTask) taskOptional + SeekableStreamIndexTask task = (SeekableStreamIndexTask) taskOptional .get(); String taskSequenceName = task.getIOConfig().getBaseSequenceName(); @@ -1688,14 +1688,14 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) } protected String generateSequenceName( - Map startPartitions, + Map startPartitions, Optional minimumMessageTime, Optional maximumMessageTime ) { StringBuilder sb = new StringBuilder(); - for (Entry entry : startPartitions.entrySet()) { + for (Entry entry : startPartitions.entrySet()) { sb.append(StringUtils.format("+%s(%s)", entry.getKey().toString(), entry.getValue().toString())); } String partitionOffsetStr = sb.toString().substring(1); @@ -1724,7 +1724,7 @@ protected String generateSequenceName( private void updatePartitionDataFromStream() { - Set partitionIds; + Set partitionIds; try { synchronized (recordSupplierLock) { partitionIds = recordSupplier.getPartitionIds(ioConfig.getId()); @@ -1742,7 +1742,7 @@ private void updatePartitionDataFromStream() log.debug("Found [%d] partitions for stream [%s]", partitionIds.size(), ioConfig.getId()); - Set closedPartitions = getOffsetsFromMetadataStorage() + Set closedPartitions = getOffsetsFromMetadataStorage() .entrySet() .stream() .filter(x -> OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(x.getValue())) @@ -1750,7 +1750,7 @@ private void updatePartitionDataFromStream() .collect(Collectors.toSet()); boolean initialPartitionDiscovery = this.partitionIds.isEmpty(); - for (partitionType partitionId : partitionIds) { + for (PartitionType partitionId : partitionIds) { if (closedPartitions.contains(partitionId)) { log.info("partition [%s] is closed and has no more data, skipping.", partitionId); continue; @@ -1763,7 +1763,7 @@ private void updatePartitionDataFromStream() int taskGroupId = getTaskGroupIdForPartition(partitionId); - ConcurrentHashMap partitionMap = partitionGroups.computeIfAbsent( + ConcurrentHashMap partitionMap = partitionGroups.computeIfAbsent( taskGroupId, k -> new ConcurrentHashMap<>() ); @@ -1848,7 +1848,7 @@ public Boolean apply(@Nullable DateTime startTime) private void checkTaskDuration() throws ExecutionException, InterruptedException, TimeoutException { - final List>> futures = Lists.newArrayList(); + final List>> futures = Lists.newArrayList(); final List futureGroupIds = Lists.newArrayList(); for (Entry entry : taskGroups.entrySet()) { @@ -1880,12 +1880,12 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } } - List> results = Futures.successfulAsList(futures) + List> results = Futures.successfulAsList(futures) .get(futureTimeoutInSeconds, TimeUnit.SECONDS); for (int j = 0; j < results.size(); j++) { Integer groupId = futureGroupIds.get(j); TaskGroup group = taskGroups.get(groupId); - Map endOffsets = results.get(j); + Map endOffsets = results.get(j); if (endOffsets != null) { // set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion @@ -1893,7 +1893,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException pendingCompletionTaskGroups.computeIfAbsent(groupId, k -> new CopyOnWriteArrayList<>()).add(group); // set endOffsets as the next startOffsets - for (Entry entry : endOffsets.entrySet()) { + for (Entry entry : endOffsets.entrySet()) { partitionGroups.get(groupId).put(entry.getKey(), entry.getValue()); } } else { @@ -1915,7 +1915,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } } - private ListenableFuture> checkpointTaskGroup( + private ListenableFuture> checkpointTaskGroup( final TaskGroup taskGroup, final boolean finalize ) @@ -1936,11 +1936,11 @@ private ListenableFuture> checkpointTaskGroup( // publishing failed and we need to re-ingest) return Futures.transform( stopTasksInGroup(taskGroup), - new Function>() + new Function>() { @Nullable @Override - public Map apply(@Nullable Object input) + public Map apply(@Nullable Object input) { return null; } @@ -1960,7 +1960,7 @@ public Map apply(@Nullable Object input) } // 2) Pause running tasks - final List>> pauseFutures = Lists.newArrayList(); + final List>> pauseFutures = Lists.newArrayList(); final List pauseTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); for (final String taskId : pauseTaskIds) { pauseFutures.add(taskClient.pauseAsync(taskId)); @@ -1968,16 +1968,16 @@ public Map apply(@Nullable Object input) return Futures.transform( Futures.successfulAsList(pauseFutures), - new Function>, Map>() + new Function>, Map>() { @Nullable @Override - public Map apply(List> input) + public Map apply(List> input) { // 3) Build a map of the highest offset read by any task in the group for each partition - final Map endOffsets = new HashMap<>(); + final Map endOffsets = new HashMap<>(); for (int i = 0; i < input.size(); i++) { - Map result = input.get(i); + Map result = input.get(i); if (result == null || result.isEmpty()) { // kill tasks that didn't return a value String taskId = pauseTaskIds.get(i); @@ -1986,7 +1986,7 @@ public Map apply(List offset : result.entrySet()) { + for (Entry offset : result.entrySet()) { if (!endOffsets.containsKey(offset.getKey()) || makeSequenceNumber(endOffsets.get(offset.getKey())).compareTo( makeSequenceNumber(offset.getValue())) < 0) { @@ -2241,10 +2241,10 @@ private void createNewTasks() throws JsonProcessingException try { - Map> startingOffsets = generateStartingSequencesForPartitionGroup( + Map> startingOffsets = generateStartingSequencesForPartitionGroup( groupId); - ImmutableMap simpleStartingOffsets = startingOffsets + ImmutableMap simpleStartingOffsets = startingOffsets .entrySet() .stream() .filter(x -> x.getValue().get() != null) @@ -2253,7 +2253,7 @@ private void createNewTasks() throws JsonProcessingException ImmutableMap::copyOf )); - Set exclusiveStartSequenceNumberPartitions = startingOffsets + Set exclusiveStartSequenceNumberPartitions = startingOffsets .entrySet().stream() .filter(x -> x.getValue().get() != null && x.getValue().isExclusive()) .map(Entry::getKey) @@ -2336,15 +2336,15 @@ protected int getNoticesQueueSize() return notices.size(); } - private ImmutableMap> generateStartingSequencesForPartitionGroup( + private ImmutableMap> generateStartingSequencesForPartitionGroup( int groupId ) throws TimeoutException { - ImmutableMap.Builder> builder = ImmutableMap.builder(); - for (Entry entry : partitionGroups.get(groupId).entrySet()) { - partitionType partition = entry.getKey(); - sequenceType offset = entry.getValue(); + ImmutableMap.Builder> builder = ImmutableMap.builder(); + for (Entry entry : partitionGroups.get(groupId).entrySet()) { + PartitionType partition = entry.getKey(); + SequenceType offset = entry.getValue(); if (!NOT_SET.equals(offset)) { // if we are given a startingOffset (set by a previous task group which is pending completion) then use it @@ -2354,7 +2354,7 @@ private ImmutableMap> generat } else { // if we don't have a startingOffset (first run or we had some previous failures and reset the offsets) then // get the offset from metadata storage (if available) or Kafka/Kinesis (otherwise) - OrderedSequenceNumber offsetFromStorage = getOffsetFromStorageForPartition(partition); + OrderedSequenceNumber offsetFromStorage = getOffsetFromStorageForPartition(partition); if (offsetFromStorage != null) { builder.put(partition, offsetFromStorage); @@ -2368,11 +2368,11 @@ private ImmutableMap> generat * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't * find any data, it will retrieve the latest or earliest Kafka/Kinesis offset depending on the useEarliestOffset config. */ - private OrderedSequenceNumber getOffsetFromStorageForPartition(partitionType partition) + private OrderedSequenceNumber getOffsetFromStorageForPartition(PartitionType partition) throws TimeoutException { - final Map metadataOffsets = getOffsetsFromMetadataStorage(); - sequenceType offset = metadataOffsets.get(partition); + final Map metadataOffsets = getOffsetsFromMetadataStorage(); + SequenceType offset = metadataOffsets.get(partition); if (offset != null) { log.debug("Getting offset [%s] from metadata storage for partition [%s]", offset, partition); if (!taskTuningConfig.isSkipSequenceNumberAvailabilityCheck()) { @@ -2418,13 +2418,13 @@ private OrderedSequenceNumber getOffsetFromStorageForPartition(par } } - private Map getOffsetsFromMetadataStorage() + private Map getOffsetsFromMetadataStorage() { final DataSourceMetadata dataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); if (dataSourceMetadata instanceof SeekableStreamDataSourceMetadata && checkSourceMetadataMatch(dataSourceMetadata)) { @SuppressWarnings("unchecked") - SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata) + SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata) .getSeekableStreamPartitions(); if (partitions != null) { if (!ioConfig.getId().equals(partitions.getName())) { @@ -2443,11 +2443,11 @@ && checkSourceMetadataMatch(dataSourceMetadata)) { return Collections.emptyMap(); } - protected sequenceType getOffsetFromStreamForPartition(partitionType partition, boolean useEarliestOffset) + protected SequenceType getOffsetFromStreamForPartition(PartitionType partition, boolean useEarliestOffset) throws TimeoutException { synchronized (recordSupplierLock) { - StreamPartition topicPartition = new StreamPartition<>(ioConfig.getId(), partition); + StreamPartition topicPartition = new StreamPartition<>(ioConfig.getId(), partition); if (!recordSupplier.getAssignment().contains(topicPartition)) { recordSupplier.assign(Collections.singleton(topicPartition)); } @@ -2461,12 +2461,12 @@ protected sequenceType getOffsetFromStreamForPartition(partitionType partition, private void createTasksForGroup(int groupId, int replicas) throws JsonProcessingException { TaskGroup group = taskGroups.get(groupId); - Map startPartitions = group.startingSequences; - Map endPartitions = new HashMap<>(); - for (partitionType partition : startPartitions.keySet()) { + Map startPartitions = group.startingSequences; + Map endPartitions = new HashMap<>(); + for (PartitionType partition : startPartitions.keySet()) { endPartitions.put(partition, END_OF_PARTITION); } - Set exclusiveStartSequenceNumberPartitions = taskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; + Set exclusiveStartSequenceNumberPartitions = taskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; DateTime minimumMessageTime = taskGroups.get(groupId).minimumMessageTime.orNull(); DateTime maximumMessageTime = taskGroups.get(groupId).maximumMessageTime.orNull(); @@ -2483,7 +2483,7 @@ private void createTasksForGroup(int groupId, int replicas) throws JsonProcessin ); - List> taskList = createIndexTasks( + List> taskList = createIndexTasks( replicas, group.baseSequenceName, sortingMapper, @@ -2535,7 +2535,7 @@ private void updateCurrentOffsets() throws InterruptedException, ExecutionExcept ).map( task -> Futures.transform( taskClient.getCurrentOffsetsAsync(task.getKey(), false), - (Function, Void>) (currentSequences) -> { + (Function, Void>) (currentSequences) -> { if (currentSequences != null && !currentSequences.isEmpty()) { task.getValue().currentSequences = currentSequences; @@ -2552,7 +2552,7 @@ private void updateCurrentOffsets() throws InterruptedException, ExecutionExcept private void updateLatestOffsetsFromStream() { synchronized (recordSupplierLock) { - Set partitionIds = null; + Set partitionIds = null; try { partitionIds = recordSupplier.getPartitionIds(ioConfig.getId()); } @@ -2561,7 +2561,7 @@ private void updateLatestOffsetsFromStream() Throwables.propagate(e); } - Set> partitions = partitionIds + Set> partitions = partitionIds .stream() .map(e -> new StreamPartition<>(ioConfig.getId(), e)) .collect(Collectors.toSet()); @@ -2578,7 +2578,7 @@ private void updateLatestOffsetsFromStream() } - protected Map getHighestCurrentOffsets() + protected Map getHighestCurrentOffsets() { return taskGroups .values() @@ -2592,7 +2592,7 @@ protected Map getHighestCurrentOffsets() )); } - private OrderedSequenceNumber makeSequenceNumber(sequenceType seq) + private OrderedSequenceNumber makeSequenceNumber(SequenceType seq) { return makeSequenceNumber(seq, false, false); } @@ -2641,12 +2641,12 @@ public abstract void checkpoint( */ protected abstract SeekableStreamIOConfig createIoConfig( int groupId, - Map startPartitions, - Map endPartitions, + Map startPartitions, + Map endPartitions, String baseSequenceName, DateTime minimumMessageTime, DateTime maximumMessageTime, - Set exclusiveStartSequenceNumberPartitions, + Set exclusiveStartSequenceNumberPartitions, SeekableStreamSupervisorIOConfig ioConfig ); @@ -2658,11 +2658,11 @@ protected abstract SeekableStreamIOConfig createIoConfig( * * @throws JsonProcessingException */ - protected abstract List> createIndexTasks( + protected abstract List> createIndexTasks( int replicas, String baseSequenceName, ObjectMapper sortingMapper, - TreeMap> sequenceOffsets, + TreeMap> sequenceOffsets, SeekableStreamIOConfig taskIoConfig, SeekableStreamTuningConfig taskTuningConfig, RowIngestionMetersFactory rowIngestionMetersFactory @@ -2676,7 +2676,7 @@ protected abstract List> cr * * @return taskgroup id */ - protected abstract int getTaskGroupIdForPartition(partitionType partition); + protected abstract int getTaskGroupIdForPartition(PartitionType partition); /** * checks if the passed in DataSourceMetadata is a specific instance @@ -2706,19 +2706,19 @@ protected abstract List> cr * * @return specific instance of datasource metadata */ - protected abstract SeekableStreamDataSourceMetadata createDataSourceMetaData( + protected abstract SeekableStreamDataSourceMetadata createDataSourceMetaData( String - stream, Map map + stream, Map map ); /** - * wraps the passed in sequenceType sequence number into a {@link OrderedSequenceNumber} object + * wraps the passed in SequenceType sequence number into a {@link OrderedSequenceNumber} object * to facilitate comparison and accomodate exclusive starting sequennce in kinesis * * @return specific instance of [Kafka/Kinesis]OrderedSequenceNumber */ - protected abstract OrderedSequenceNumber makeSequenceNumber( - sequenceType seq, + protected abstract OrderedSequenceNumber makeSequenceNumber( + SequenceType seq, boolean useExclusive, boolean isExclusive ); @@ -2735,21 +2735,21 @@ protected abstract OrderedSequenceNumber makeSequenceNumber( * * @return map of partition id -> lag */ - protected abstract Map getLagPerPartition(Map currentOffsets); + protected abstract Map getLagPerPartition(Map currentOffsets); /** * returns an instance of a specific Kinesis/Kafka recordSupplier * * @return specific instance of Kafka/Kinesis RecordSupplier */ - protected abstract RecordSupplier setupRecordSupplier(); + protected abstract RecordSupplier setupRecordSupplier(); /** * creates a specific instance of Kafka/Kinesis Supervisor Report Payload * * @return specific instance of Kafka/Kinesis Supervisor Report Payload */ - protected abstract SeekableStreamSupervisorReportPayload createReportPayload( + protected abstract SeekableStreamSupervisorReportPayload createReportPayload( int numPartitions, boolean includeOffsets ); @@ -2762,8 +2762,8 @@ protected abstract SeekableStreamSupervisorReportPayload +public abstract class SeekableStreamSupervisorReportPayload { private final String dataSource; private final String id; @@ -38,8 +38,8 @@ public abstract class SeekableStreamSupervisorReportPayload activeTasks; private final List publishingTasks; - private final Map latestOffsets; - private final Map minimumLag; + private final Map latestOffsets; + private final Map minimumLag; private final Long aggregateLag; private final DateTime offsetsLastUpdated; private final boolean suspended; @@ -50,8 +50,8 @@ public SeekableStreamSupervisorReportPayload( int partitions, int replicas, long durationSeconds, - @Nullable Map latestOffsets, - @Nullable Map minimumLag, + @Nullable Map latestOffsets, + @Nullable Map minimumLag, @Nullable Long aggregateLag, @Nullable DateTime offsetsLastUpdated, boolean suspended @@ -131,14 +131,14 @@ public List getPublishingTasks() @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getLatestOffsets() + public Map getLatestOffsets() { return latestOffsets; } @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getMinimumLag() + public Map getMinimumLag() { return minimumLag; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java index 8ca2e1c4a5b5..7bd23442b035 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java @@ -26,24 +26,24 @@ import javax.annotation.Nullable; import java.util.Map; -public class TaskReportData +public class TaskReportData { private final String id; - private final Map startingOffsets; + private final Map startingOffsets; private final DateTime startTime; private final Long remainingSeconds; private final TaskType type; - private Map currentOffsets; - private final Map lag; + private Map currentOffsets; + private final Map lag; public TaskReportData( String id, - @Nullable Map startingOffsets, - @Nullable Map currentOffsets, + @Nullable Map startingOffsets, + @Nullable Map currentOffsets, @Nullable DateTime startTime, Long remainingSeconds, TaskType type, - @Nullable Map lag + @Nullable Map lag ) { this.id = id; @@ -63,14 +63,14 @@ public String getId() @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getStartingOffsets() + public Map getStartingOffsets() { return startingOffsets; } @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getCurrentOffsets() + public Map getCurrentOffsets() { return currentOffsets; } @@ -95,12 +95,12 @@ public TaskType getType() @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getLag() + public Map getLag() { return lag; } - public void setCurrentSequenceNumbers(Map currentOffsets) + public void setCurrentSequenceNumbers(Map currentOffsets) { this.currentOffsets = currentOffsets; } From c16559d33d8be698c08536f5d1bc7def6da368ff Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 29 Oct 2018 15:07:14 -0700 Subject: [PATCH 34/87] removed abstract class loggers --- .../indexing/seekablestream/SeekableStreamIndexTaskClient.java | 2 +- .../seekablestream/supervisor/SeekableStreamSupervisor.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index ded1b6e98aff..f48af510021c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -45,7 +45,7 @@ public abstract class SeekableStreamIndexTaskClient extends IndexTaskClient { - private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTaskClient.class); + private final EmittingLogger log = new EmittingLogger(this.getClass()); public SeekableStreamIndexTaskClient( HttpClient httpClient, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 563994999523..df2e9f1e130a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -130,13 +130,14 @@ public abstract class SeekableStreamSupervisor implements Supervisor { public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED"; - private static final EmittingLogger log = new EmittingLogger(SeekableStreamSupervisor.class); private static final Random RANDOM = ThreadLocalRandom.current(); private static final long MAX_RUN_FREQUENCY_MILLIS = 1000; private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120; private static final int MAX_INITIALIZATION_RETRIES = 20; private static final CopyOnWriteArrayList EMPTY_LIST = Lists.newCopyOnWriteArrayList(); + private final EmittingLogger log = new EmittingLogger(this.getClass()); + // Internal data structures // -------------------------------------------------------- From 1e1231f95c2d187ebebb121f6cf24321501db00c Mon Sep 17 00:00:00 2001 From: Josh Sun Date: Fri, 2 Nov 2018 16:16:35 -0700 Subject: [PATCH 35/87] conform to guava api restrictions --- .../kafka/KafkaDataSourceMetadata.java | 1 - .../druid/indexing/kafka/KafkaIndexTask.java | 4 +- .../kafka/KafkaIndexTaskClientTest.java | 41 +++++++++--------- .../indexing/kafka/KafkaIndexTaskTest.java | 2 +- .../indexing/kinesis/KinesisIndexTask.java | 16 ++++--- .../kinesis/supervisor/KinesisSupervisor.java | 2 +- .../kinesis/KinesisIndexTaskClientTest.java | 41 +++++++++--------- .../kinesis/KinesisIndexTaskTest.java | 42 +++++++++---------- .../SeekableStreamDataSourceMetadata.java | 6 +-- .../supervisor/SeekableStreamSupervisor.java | 31 +++++++------- 10 files changed, 92 insertions(+), 94 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java index 0ef9f0731951..a37941f4f62b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java @@ -24,7 +24,6 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; -import java.util.HashMap; import java.util.Map; public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index b2369c5da650..7c53340b1e5d 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; import org.apache.druid.data.input.InputRow; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; @@ -56,6 +55,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import java.util.ArrayList; import java.util.Map; import java.util.Properties; import java.util.Set; @@ -260,7 +260,7 @@ static void assignPartitions( ) { consumer.assign( - Lists.newArrayList( + new ArrayList<>( partitions.stream().map(n -> new TopicPartition(topic, n)).collect(Collectors.toList()) ) ); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java index c441efd626ac..f74cf9212570 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -23,7 +23,6 @@ import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -61,6 +60,8 @@ import java.io.IOException; import java.net.URL; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -73,7 +74,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport private static final ObjectMapper objectMapper = new DefaultObjectMapper(); private static final String TEST_ID = "test-id"; - private static final List TEST_IDS = Lists.newArrayList("test-id1", "test-id2", "test-id3", "test-id4"); + private static final List TEST_IDS = Arrays.asList("test-id1", "test-id2", "test-id3", "test-id4"); private static final String TEST_HOST = "test-host"; private static final int TEST_PORT = 1234; private static final int TEST_TLS_PORT = -1; @@ -625,8 +626,8 @@ public void testStopAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "stop"))); futures.add(client.stopAsync(testId, false)); @@ -657,8 +658,8 @@ public void testResumeAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "resume"))); futures.add(client.resumeAsync(testId)); @@ -690,8 +691,8 @@ public void testPauseAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List>> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List>> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "pause"))); futures.add(client.pauseAsync(testId)); @@ -723,8 +724,8 @@ public void testGetStatusAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "status"))); futures.add(client.getStatusAsync(testId)); @@ -757,8 +758,8 @@ public void testGetStartTimeAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "time/start"))); futures.add(client.getStartTimeAsync(testId)); @@ -790,8 +791,8 @@ public void testGetCurrentOffsetsAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List>> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List>> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "offsets/current"))); futures.add(client.getCurrentOffsetsAsync(testId, false)); @@ -823,8 +824,8 @@ public void testGetEndOffsetsAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List>> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List>> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "offsets/end"))); futures.add(client.getEndOffsetsAsync(testId)); @@ -856,8 +857,8 @@ public void testSetEndOffsetsAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format( URL_FORMATTER, @@ -895,8 +896,8 @@ public void testSetEndOffsetsAsyncWithResume() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add( new URL( diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 45522184f5fd..5d6af83d6d43 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -2426,7 +2426,7 @@ private List readSegmentColumn(final String column, final SegmentDescrip QueryableIndex index = indexIO.loadIndex(outputLocation); DictionaryEncodedColumn theColumn = (DictionaryEncodedColumn) index.getColumnHolder(column) .getColumn(); - List values = Lists.newArrayList(); + List values = new ArrayList<>(); for (int i = 0; i < theColumn.length(); i++) { int id = theColumn.getSingleValueRow(i); String value = theColumn.lookupName(id); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 622b20630cd7..fa5ce08aa1ec 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -30,8 +30,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import org.apache.druid.data.input.Committer; import org.apache.druid.data.input.InputRow; import org.apache.druid.discovery.DiscoveryDruidNode; @@ -326,7 +324,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception } // Set up sequenceNames. - final Map sequenceNames = Maps.newHashMap(); + final Map sequenceNames = new HashMap<>(); for (String partitionNum : lastOffsets.keySet()) { sequenceNames.put(partitionNum, StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), partitionNum)); } @@ -715,7 +713,7 @@ private StreamAppenderatorDriver newDriver( private Set assignPartitions(RecordSupplier recordSupplier, String topic) { // Initialize consumer assignment. - final Set assignment = Sets.newHashSet(); + final Set assignment = new HashSet<>(); for (Map.Entry entry : lastOffsets.entrySet()) { final String endOffset = endOffsets.get(entry.getKey()); if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(endOffset) @@ -966,7 +964,7 @@ private void sendResetRequestAndWait( private Map getTaskCompletionUnparseableEvents() { - Map unparseableEventsMap = Maps.newHashMap(); + Map unparseableEventsMap = new HashMap<>(); List buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions( savedParseExceptions ); @@ -978,7 +976,7 @@ private Map getTaskCompletionUnparseableEvents() private Map getTaskCompletionRowStats() { - Map metrics = Maps.newHashMap(); + Map metrics = new HashMap<>(); metrics.put( RowIngestionMeters.BUILD_SEGMENTS, rowIngestionMeters.getTotals() @@ -1304,9 +1302,9 @@ public Response getRowStats( ) { authorizationCheck(req, Action.READ); - Map returnMap = Maps.newHashMap(); - Map totalsMap = Maps.newHashMap(); - Map averagesMap = Maps.newHashMap(); + Map returnMap = new HashMap<>(); + Map totalsMap = new HashMap<>(); + Map averagesMap = new HashMap<>(); totalsMap.put( RowIngestionMeters.BUILD_SEGMENTS, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index c3a39d06f602..d5faf01b88cf 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -403,7 +403,7 @@ public TopicPartition apply(PartitionInfo input) }) ); final Map offsetsResponse = new ConcurrentHashMap<>(); - final List> futures = Lists.newArrayList(); + final List> futures = new ArrayList<>(); for (TaskGroup taskGroup : taskGroups.values()) { for (String taskId : taskGroup.taskIds()) { futures.add(Futures.transform( diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java index 2bc15b3ce01c..64b00c6e9a18 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java @@ -23,7 +23,6 @@ import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -61,6 +60,8 @@ import java.io.IOException; import java.net.URL; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -74,7 +75,7 @@ public class KinesisIndexTaskClientTest extends EasyMockSupport private static final ObjectMapper objectMapper = new DefaultObjectMapper(); private static final String TEST_ID = "test-id"; - private static final List TEST_IDS = Lists.newArrayList("test-id1", "test-id2", "test-id3", "test-id4"); + private static final List TEST_IDS = Arrays.asList("test-id1", "test-id2", "test-id3", "test-id4"); private static final String TEST_HOST = "test-host"; private static final int TEST_PORT = 1234; private static final int TEST_TLS_PORT = -1; @@ -694,8 +695,8 @@ public void testStopAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "stop"))); futures.add(client.stopAsync(testId, false)); @@ -730,8 +731,8 @@ public void testResumeAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "resume"))); futures.add(client.resumeAsync(testId)); @@ -767,8 +768,8 @@ public void testPauseAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List>> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List>> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "pause"))); futures.add(client.pauseAsync(testId)); @@ -804,8 +805,8 @@ public void testGetStatusAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "status"))); futures.add(client.getStatusAsync(testId)); @@ -842,8 +843,8 @@ public void testGetStartTimeAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "time/start"))); futures.add(client.getStartTimeAsync(testId)); @@ -879,8 +880,8 @@ public void testGetCurrentOffsetsAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List>> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List>> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "offsets/current"))); futures.add(client.getCurrentOffsetsAsync(testId, false)); @@ -916,8 +917,8 @@ public void testGetEndOffsetsAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List>> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List>> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "offsets/end"))); futures.add(client.getEndOffsetsAsync(testId)); @@ -953,8 +954,8 @@ public void testSetEndOffsetsAsync() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format( URL_FORMATTER, @@ -996,8 +997,8 @@ public void testSetEndOffsetsAsyncWithResume() throws Exception ).times(numRequests); replayAll(); - List expectedUrls = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List expectedUrls = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add( new URL( diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 71f2d13b1047..49dcf71c8dcb 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -40,8 +40,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; import com.google.common.io.Files; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; @@ -171,8 +169,10 @@ import java.lang.reflect.Method; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -260,7 +260,7 @@ public class KinesisIndexTaskTest private static ServiceEmitter emitter; private static ListeningExecutorService taskExec; - private final List runningTasks = Lists.newArrayList(); + private final List runningTasks = new ArrayList<>(); private long handoffConditionTimeout = 0; private boolean reportParseExceptions = false; @@ -280,7 +280,7 @@ public class KinesisIndexTaskTest private File directory; private String stream; private final boolean isIncrementalHandoffSupported = false; - private final Set checkpointRequestsHash = Sets.newHashSet(); + private final Set checkpointRequestsHash = new HashSet<>(); private File reportsFile; private RowIngestionMetersFactory rowIngestionMetersFactory; @@ -408,7 +408,7 @@ public static void tearDownClass() throws Exception } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testRunAfterDataInserted() throws Exception { AmazonKinesis kinesis = getKinesisClientInstance(); @@ -473,7 +473,7 @@ public void testRunAfterDataInserted() throws Exception Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testRunBeforeDataInserted() throws Exception { AmazonKinesis kinesis = getKinesisClientInstance(); @@ -566,7 +566,7 @@ public void testRunBeforeDataInserted() throws Exception } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testRunWithMinimumMessageTime() throws Exception { AmazonKinesis kinesis = getKinesisClientInstance(); @@ -636,7 +636,7 @@ public void testRunWithMinimumMessageTime() throws Exception Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testRunWithMaximumMessageTime() throws Exception { AmazonKinesis kinesis = getKinesisClientInstance(); @@ -708,7 +708,7 @@ public void testRunWithMaximumMessageTime() throws Exception Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testRunWithTransformSpec() throws Exception { AmazonKinesis kinesis = getKinesisClientInstance(); @@ -785,7 +785,7 @@ public void testRunWithTransformSpec() throws Exception Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", desc1)); } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testRunOnNothing() throws Exception { AmazonKinesis kinesis = getKinesisClientInstance(); @@ -833,7 +833,7 @@ public void testRunOnNothing() throws Exception Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testReportParseExceptions() throws Exception { reportParseExceptions = true; @@ -888,7 +888,7 @@ public void testReportParseExceptions() throws Exception Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testMultipleParseExceptionsSuccess() throws Exception { reportParseExceptions = false; @@ -987,7 +987,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testMultipleParseExceptionsFailure() throws Exception { reportParseExceptions = false; @@ -1068,7 +1068,7 @@ public void testMultipleParseExceptionsFailure() throws Exception Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testRunReplicas() throws Exception { // Insert data @@ -1166,7 +1166,7 @@ public void testRunReplicas() throws Exception Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testRunConflicting() throws Exception { // Insert data @@ -1265,7 +1265,7 @@ public void testRunConflicting() throws Exception Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testRunConflictingWithoutTransactions() throws Exception { // Insert data @@ -1362,7 +1362,7 @@ public void testRunConflictingWithoutTransactions() throws Exception Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4)); } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testRunOneTaskTwoPartitions() throws Exception { // Insert data @@ -1446,7 +1446,7 @@ public void testRunOneTaskTwoPartitions() throws Exception ); } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testRunTwoTasksTwoPartitions() throws Exception { // Insert data @@ -1555,7 +1555,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4)); } - @Test(timeout = 60_000L) + @Test(timeout = 120_000L) public void testRestore() throws Exception { // Insert data @@ -1986,7 +1986,7 @@ public void close() @Override public List getLocations() { - return Lists.newArrayList(); + return new ArrayList<>(); } }; toolboxFactory = new TaskToolboxFactory( @@ -2090,7 +2090,7 @@ private List readSegmentColumn(final String column, final SegmentDescrip QueryableIndex index = indexIO.loadIndex(outputLocation); DictionaryEncodedColumn theColumn = (DictionaryEncodedColumn) index.getColumnHolder(column) .getColumn(); - List values = Lists.newArrayList(); + List values = new ArrayList<>(); for (int i = 0; i < theColumn.length(); i++) { int id = theColumn.getSingleValueRow(i); String value = theColumn.lookupName(id); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java index 76d3df5efa3a..636610144a93 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java @@ -21,10 +21,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.Maps; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.java.util.common.IAE; +import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -79,7 +79,7 @@ public DataSourceMetadata plus(DataSourceMetadata other) if (that.getSeekableStreamPartitions().getName().equals(seekableStreamPartitions.getName())) { // Same topic, merge offsets. - final Map newMap = Maps.newHashMap(); + final Map newMap = new HashMap<>(); for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { newMap.put(entry.getKey(), entry.getValue()); @@ -113,7 +113,7 @@ public DataSourceMetadata minus(DataSourceMetadata other) if (that.getSeekableStreamPartitions().getName().equals(seekableStreamPartitions.getName())) { // Same stream, remove partitions present in "that" from "this" - final Map newMap = Maps.newHashMap(); + final Map newMap = new HashMap<>(); for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { if (!that.getSeekableStreamPartitions().getMap().containsKey(entry.getKey())) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index a39dba760815..bc1644231fd3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -34,7 +34,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; @@ -821,7 +820,7 @@ private SupervisorReport> taskReports = Lists.newArrayList(); + List> taskReports = new ArrayList<>(); try { for (TaskGroup taskGroup : taskGroups.values()) { @@ -917,7 +916,7 @@ public Map> getStats() private Map> getCurrentTotalStats() throws InterruptedException, ExecutionException, TimeoutException { - Map> allStats = Maps.newHashMap(); + Map> allStats = new HashMap<>(); final List> futures = new ArrayList<>(); final List> groupAndTaskIds = new ArrayList<>(); @@ -962,7 +961,7 @@ private Map> getCurrentTotalStats() for (int i = 0; i < results.size(); i++) { StatsFromTaskResult result = results.get(i); if (result != null) { - Map groupMap = allStats.computeIfAbsent(result.getGroupId(), k -> Maps.newHashMap()); + Map groupMap = allStats.computeIfAbsent(result.getGroupId(), k -> new HashMap<>()); groupMap.put(result.getTaskId(), result.getStats()); } else { Pair groupAndTaskId = groupAndTaskIds.get(i); @@ -1249,8 +1248,8 @@ private boolean isTaskInPendingCompletionGroups(String taskId) private void discoverTasks() throws ExecutionException, InterruptedException, TimeoutException { int taskCount = 0; - List futureTaskIds = Lists.newArrayList(); - List> futures = Lists.newArrayList(); + List futureTaskIds = new ArrayList<>(); + List> futures = new ArrayList<>(); List tasks = taskStorage.getActiveTasks(); final Map taskGroupsToVerify = new HashMap<>(); @@ -1781,8 +1780,8 @@ private void updatePartitionDataFromStream() private void updateTaskStatus() throws ExecutionException, InterruptedException, TimeoutException { - final List> futures = Lists.newArrayList(); - final List futureTaskIds = Lists.newArrayList(); + final List> futures = new ArrayList<>(); + final List futureTaskIds = new ArrayList<>(); // update status (and startTime if unknown) of current tasks in taskGroups for (TaskGroup group : taskGroups.values()) { @@ -1848,8 +1847,8 @@ public Boolean apply(@Nullable DateTime startTime) private void checkTaskDuration() throws ExecutionException, InterruptedException, TimeoutException { - final List>> futures = Lists.newArrayList(); - final List futureGroupIds = Lists.newArrayList(); + final List>> futures = new ArrayList<>(); + final List futureGroupIds = new ArrayList<>(); for (Entry entry : taskGroups.entrySet()) { Integer groupId = entry.getKey(); @@ -1960,7 +1959,7 @@ public Map apply(@Nullable Object input) } // 2) Pause running tasks - final List>> pauseFutures = Lists.newArrayList(); + final List>> pauseFutures = new ArrayList<>(); final List pauseTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); for (final String taskId : pauseTaskIds) { pauseFutures.add(taskClient.pauseAsync(taskId)); @@ -1998,7 +1997,7 @@ public Map apply(List> setEndOffsetFutures = Lists.newArrayList(); + final List> setEndOffsetFutures = new ArrayList<>(); final List setEndOffsetTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); if (setEndOffsetTaskIds.isEmpty()) { @@ -2063,7 +2062,7 @@ private ListenableFuture stopTasksInGroup(@Nullable TaskGroup taskGroup) return Futures.immediateFuture(null); } - final List> futures = Lists.newArrayList(); + final List> futures = new ArrayList<>(); for (Entry entry : taskGroup.tasks.entrySet()) { final String taskId = entry.getKey(); final TaskData taskData = entry.getValue(); @@ -2080,14 +2079,14 @@ private ListenableFuture stopTasksInGroup(@Nullable TaskGroup taskGroup) private void checkPendingCompletionTasks() throws ExecutionException, InterruptedException, TimeoutException { - List> futures = Lists.newArrayList(); + List> futures = new ArrayList<>(); for (Entry> pendingGroupList : pendingCompletionTaskGroups.entrySet()) { boolean stopTasksInTaskGroup = false; Integer groupId = pendingGroupList.getKey(); CopyOnWriteArrayList taskGroupList = pendingGroupList.getValue(); - List toRemove = Lists.newArrayList(); + List toRemove = new ArrayList<>(); for (TaskGroup group : taskGroupList) { boolean foundSuccess = false, entireTaskGroupFailed = false; @@ -2162,7 +2161,7 @@ private void checkPendingCompletionTasks() private void checkCurrentTaskState() throws ExecutionException, InterruptedException, TimeoutException { - List> futures = Lists.newArrayList(); + List> futures = new ArrayList<>(); Iterator> iTaskGroups = taskGroups.entrySet().iterator(); while (iTaskGroups.hasNext()) { Entry taskGroupEntry = iTaskGroups.next(); From 727bb5dcd12e2766dad264a7a212c47db1256972 Mon Sep 17 00:00:00 2001 From: Josh Sun Date: Fri, 2 Nov 2018 17:47:44 -0700 Subject: [PATCH 36/87] add docker for travis other modules test --- .travis.yml | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 63c2495c577e..f848f03ba2c6 100644 --- a/.travis.yml +++ b/.travis.yml @@ -71,9 +71,13 @@ matrix: # other modules test - - env: + - sudo: required + services: + - docker + env: - NAME="other modules test" - AWS_REGION=us-east-1 # set a aws region for unit tests + - DOCKER_IP=127.0.0.1 install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B before_script: unset _JAVA_OPTIONS script: @@ -83,9 +87,13 @@ matrix: - free -m # other modules test with SQL Compatibility enabled - - env: + - sudo: required + services: + - docker + env: - NAME="other modules test with SQL Compatibility" - AWS_REGION=us-east-1 # set a aws region for unit tests + - DOCKER_IP=127.0.0.1 install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B before_script: unset _JAVA_OPTIONS script: From 8baf786637ac4da79ad02052d77a7bb19ee3666e Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 5 Nov 2018 13:46:41 -0800 Subject: [PATCH 37/87] address comments --- ...ementalPublishingKafkaIndexTaskRunner.java | 30 +- .../kafka/KafkaDataSourceMetadata.java | 8 +- .../druid/indexing/kafka/KafkaIOConfig.java | 13 +- .../kafka/KafkaIndexTaskClientFactory.java | 5 +- .../indexing/kafka/KafkaRecordSupplier.java | 13 +- .../indexing/kafka/KafkaSequenceNumber.java | 6 +- .../kafka/LegacyKafkaIndexTaskRunner.java | 24 +- .../kafka/supervisor/KafkaSupervisor.java | 20 +- .../KafkaSupervisorReportPayload.java | 26 +- .../kafka/supervisor/KafkaSupervisorSpec.java | 10 +- .../KafkaSupervisorTuningConfig.java | 15 +- .../indexing/kafka/KafkaIOConfigTest.java | 16 +- .../indexing/kafka/KafkaIndexTaskTest.java | 12 +- .../kafka/supervisor/KafkaSupervisorTest.java | 171 +- .../kinesis-indexing-service/pom.xml | 44 - .../kinesis/KinesisDataSourceMetadata.java | 8 +- .../indexing/kinesis/KinesisIOConfig.java | 4 +- .../indexing/kinesis/KinesisIndexTask.java | 28 +- .../kinesis/KinesisSequenceNumber.java | 15 +- .../indexing/kinesis/KinesisTuningConfig.java | 32 - .../kinesis/supervisor/KinesisSupervisor.java | 23 +- .../supervisor/KinesisSupervisorIOConfig.java | 5 +- .../KinesisSupervisorReportPayload.java | 3 +- .../supervisor/KinesisSupervisorSpec.java | 20 - .../KinesisSupervisorTuningConfig.java | 9 +- ...rg.apache.druid.initialization.DruidModule | 15 + .../indexing/kinesis/KinesisIOConfigTest.java | 16 +- .../supervisor/KinesisSupervisorTest.java | 98 +- .../supervisor/SupervisorManager.java | 2 +- .../SeekableStreamDataSourceMetadata.java | 36 +- .../SeekableStreamIOConfig.java | 9 +- .../SeekableStreamPartitions.java | 14 +- .../common/OrderedSequenceNumber.java | 16 +- .../seekablestream/common/RecordSupplier.java | 8 +- .../supervisor/SeekableStreamSupervisor.java | 89 +- ...SeekableStreamSupervisorReportPayload.java | 6 +- .../SeekableStreamSupervisorTuningConfig.java | 5 + pom.xml | 2876 ++++++++--------- 38 files changed, 1847 insertions(+), 1903 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index c5a91eaf5e6a..7fc159756063 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -217,10 +217,10 @@ public IncrementalPublishingKafkaIndexTaskRunner( this.authorizerMapper = authorizerMapper; this.chatHandlerProvider = chatHandlerProvider; this.savedParseExceptions = savedParseExceptions; - this.topic = ioConfig.getStartPartitions().getTopic(); + this.topic = ioConfig.getStartPartitions().getStream(); this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters(); - this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndPartitions().getPartitionOffsetMap()); + this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndPartitions().getPartitionSequenceNumberMap()); this.sequences = new CopyOnWriteArrayList<>(); this.ingestionState = IngestionState.NOT_STARTED; @@ -279,7 +279,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception sequences.add(new SequenceMetadata( 0, StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0), - ioConfig.getStartPartitions().getPartitionOffsetMap(), + ioConfig.getStartPartitions().getPartitionSequenceNumberMap(), endOffsets, false )); @@ -327,7 +327,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception appenderator = task.newAppenderator(fireDepartmentMetrics, toolbox); driver = task.newDriver(appenderator, toolbox, fireDepartmentMetrics); - final String topic = ioConfig.getStartPartitions().getTopic(); + final String topic = ioConfig.getStartPartitions().getStream(); // Start up, set up initial offsets. final Object restoredMetadata = driver.startJob(); @@ -338,7 +338,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception partitionOffsetEntry -> Longs.compare( partitionOffsetEntry.getValue(), ioConfig.getStartPartitions() - .getPartitionOffsetMap() + .getPartitionSequenceNumberMap() .get(partitionOffsetEntry.getKey()) ) >= 0 ), "Sequence offsets are not compatible with start offsets of task"); @@ -360,22 +360,22 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception ) ); - nextOffsets.putAll(restoredNextPartitions.getPartitionOffsetMap()); + nextOffsets.putAll(restoredNextPartitions.getPartitionSequenceNumberMap()); // Sanity checks. - if (!restoredNextPartitions.getTopic().equals(ioConfig.getStartPartitions().getTopic())) { + if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) { throw new ISE( "WTF?! Restored topic[%s] but expected topic[%s]", - restoredNextPartitions.getTopic(), - ioConfig.getStartPartitions().getTopic() + restoredNextPartitions.getStream(), + ioConfig.getStartPartitions().getStream() ); } - if (!nextOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionOffsetMap().keySet())) { + if (!nextOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet())) { throw new ISE( "WTF?! Restored partitions[%s] but expected partitions[%s]", nextOffsets.keySet(), - ioConfig.getStartPartitions().getPartitionOffsetMap().keySet() + ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet() ); } // sequences size can be 0 only when all sequences got published and task stopped before it could finish @@ -401,7 +401,7 @@ public Object getMetadata() { return ImmutableMap.of( METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>( - ioConfig.getStartPartitions().getTopic(), + ioConfig.getStartPartitions().getStream(), snapshot ) ); @@ -1146,7 +1146,7 @@ private void sendResetRequestAndWait(Map outOfRangePartiti task.getDataSource(), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( ioConfig.getStartPartitions() - .getTopic(), + .getStream(), partitionOffsetMap )) )); @@ -1784,7 +1784,7 @@ TransactionalSegmentPublisher createPublisher(TaskToolbox toolbox, boolean useTr ); // Sanity check, we should only be publishing things that match our desired end state. - if (!getEndOffsets().equals(finalPartitions.getPartitionOffsetMap())) { + if (!getEndOffsets().equals(finalPartitions.getPartitionSequenceNumberMap())) { throw new ISE( "WTF?! Driver for sequence [%s], attempted to publish invalid metadata[%s].", toString(), @@ -1798,7 +1798,7 @@ TransactionalSegmentPublisher createPublisher(TaskToolbox toolbox, boolean useTr action = new SegmentTransactionalInsertAction( segments, new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( - finalPartitions.getTopic(), + finalPartitions.getStream(), getStartOffsets() )), new KafkaDataSourceMetadata(finalPartitions) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java index a37941f4f62b..55b03c000f30 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java @@ -37,14 +37,8 @@ public KafkaDataSourceMetadata( super(kafkaPartitions); } - @JsonProperty("partitions") - public SeekableStreamPartitions getKafkaPartitions() - { - return super.getSeekableStreamPartitions(); - } - @Override - protected SeekableStreamDataSourceMetadata createConcretDataSourceMetaData( + protected SeekableStreamDataSourceMetadata createConcreteDataSourceMetaData( String streamId, Map newMap ) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java index 9f31e1ef046f..362249ce245a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java @@ -34,7 +34,6 @@ public class KafkaIOConfig extends SeekableStreamIOConfig { private static final boolean DEFAULT_SKIP_OFFSET_GAPS = false; - @Nullable private final Map consumerProperties; private final boolean skipOffsetGaps; @@ -64,32 +63,32 @@ public KafkaIOConfig( this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); this.skipOffsetGaps = skipOffsetGaps != null ? skipOffsetGaps : DEFAULT_SKIP_OFFSET_GAPS; - for (int partition : endPartitions.getMap().keySet()) { + for (int partition : endPartitions.getPartitionSequenceNumberMap().keySet()) { Preconditions.checkArgument( - endPartitions.getMap() + endPartitions.getPartitionSequenceNumberMap() .get(partition) - .compareTo(startPartitions.getMap().get(partition)) >= 0, + .compareTo(startPartitions.getPartitionSequenceNumberMap().get(partition)) >= 0, "end offset must be >= start offset for partition[%s]", partition ); } } + // exclusive starting sequence partitions are used only for kinesis where the starting + // sequence number for certain partitions are discarded because they've already been + // read by a previous task @Override public Set getExclusiveStartSequenceNumberPartitions() { return null; } - - @Nullable @JsonProperty public Map getConsumerProperties() { return consumerProperties; } - @JsonProperty public boolean isSkipOffsetGaps() { diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java index 5b148f2c010d..cfa7c7236553 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientFactory.java @@ -36,10 +36,7 @@ public KafkaIndexTaskClientFactory( @Json ObjectMapper mapper ) { - super( - httpClient, - mapper - ); + super(httpClient, mapper); } @Override diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 82efd2a956c1..71e02623f435 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -65,7 +65,6 @@ public KafkaRecordSupplier( this.consumerProperties = consumerProperties; this.sortingMapper = sortingMapper; this.consumer = getKafkaConsumer(); - this.closed = false; this.records = new LinkedBlockingQueue<>(); } @@ -114,7 +113,7 @@ public Set> getAssignment() Set topicPartitions = consumer.assignment(); return topicPartitions .stream() - .map((TopicPartition e) -> new StreamPartition<>(e.topic(), e.partition())) + .map(e -> new StreamPartition<>(e.topic(), e.partition())) .collect(Collectors.toSet()); } @@ -137,15 +136,21 @@ public OrderedPartitionableRecord poll(long timeout) @Override public Long getLatestSequenceNumber(StreamPartition partition) { + Long currPos = consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); seekToLatest(Collections.singleton(partition)); - return consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); + Long nextPos = consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); + seek(partition, currPos); + return nextPos; } @Override public Long getEarliestSequenceNumber(StreamPartition partition) { + Long currPos = consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); seekToEarliest(Collections.singleton(partition)); - return consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); + Long nextPos = consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); + seek(partition, currPos); + return nextPos; } @Override diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java index de211f0a6fdd..efb953be0888 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java @@ -25,14 +25,14 @@ public class KafkaSequenceNumber extends OrderedSequenceNumber { - private KafkaSequenceNumber(Long sequenceNumber, boolean useExclusive, boolean isExclusive) + private KafkaSequenceNumber(Long sequenceNumber, boolean isExclusive) { - super(sequenceNumber, useExclusive, isExclusive); + super(sequenceNumber, false); } public static KafkaSequenceNumber of(Long sequenceNumber) { - return new KafkaSequenceNumber(sequenceNumber, false, false); + return new KafkaSequenceNumber(sequenceNumber, false); } @Override diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 05bd758b38d1..9f1b1dc1c8a0 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -191,7 +191,7 @@ public class LegacyKafkaIndexTaskRunner implements SeekableStreamIndexTaskRunner this.savedParseExceptions = savedParseExceptions; this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters(); - this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionOffsetMap()); + this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionSequenceNumberMap()); this.ingestionState = IngestionState.NOT_STARTED; } @@ -275,12 +275,12 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception appenderator = appenderator0; - final String topic = ioConfig.getStartPartitions().getTopic(); + final String topic = ioConfig.getStartPartitions().getStream(); // Start up, set up initial offsets. final Object restoredMetadata = driver.startJob(); if (restoredMetadata == null) { - nextOffsets.putAll(ioConfig.getStartPartitions().getPartitionOffsetMap()); + nextOffsets.putAll(ioConfig.getStartPartitions().getPartitionSequenceNumberMap()); } else { final Map restoredMetadataMap = (Map) restoredMetadata; final SeekableStreamPartitions restoredNextPartitions = toolbox.getObjectMapper().convertValue( @@ -292,22 +292,22 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception Long.class ) ); - nextOffsets.putAll(restoredNextPartitions.getPartitionOffsetMap()); + nextOffsets.putAll(restoredNextPartitions.getPartitionSequenceNumberMap()); // Sanity checks. - if (!restoredNextPartitions.getTopic().equals(ioConfig.getStartPartitions().getTopic())) { + if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) { throw new ISE( "WTF?! Restored topic[%s] but expected topic[%s]", - restoredNextPartitions.getTopic(), - ioConfig.getStartPartitions().getTopic() + restoredNextPartitions.getStream(), + ioConfig.getStartPartitions().getStream() ); } - if (!nextOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionOffsetMap().keySet())) { + if (!nextOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet())) { throw new ISE( "WTF?! Restored partitions[%s] but expected partitions[%s]", nextOffsets.keySet(), - ioConfig.getStartPartitions().getPartitionOffsetMap().keySet() + ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet() ); } } @@ -333,7 +333,7 @@ public Object getMetadata() { return ImmutableMap.of( METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>( - ioConfig.getStartPartitions().getTopic(), + ioConfig.getStartPartitions().getStream(), snapshot ) ); @@ -513,7 +513,7 @@ public void run() ); // Sanity check, we should only be publishing things that match our desired end state. - if (!endOffsets.equals(finalPartitions.getPartitionOffsetMap())) { + if (!endOffsets.equals(finalPartitions.getPartitionSequenceNumberMap())) { throw new ISE("WTF?! Driver attempted to publish invalid metadata[%s].", commitMetadata); } @@ -730,7 +730,7 @@ private void sendResetRequestAndWait(Map outOfRangePartiti task.getDataSource(), new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( ioConfig.getStartPartitions() - .getTopic(), + .getStream(), partitionOffsetMap )) )); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index adde2bc85d4a..ad2415ed262a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -87,6 +87,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor private static final long INITIAL_GET_OFFSET_DELAY_MILLIS = 15000; private static final long INITIAL_EMIT_LAG_METRIC_DELAY_MILLIS = 25000; private static final Long NOT_SET = -1L; + private static final Long END_OF_PARTITION = Long.MAX_VALUE; private final ServiceEmitter emitter; private final DruidMonitorSchedulerConfig monitorSchedulerConfig; @@ -113,8 +114,6 @@ public KafkaSupervisor( mapper, spec, rowIngestionMetersFactory, - NOT_SET, - Long.MAX_VALUE, false, true ); @@ -139,10 +138,10 @@ public void checkpoint( Preconditions.checkArgument( spec.getIoConfig() .getTopic() - .equals(((KafkaDataSourceMetadata) currentCheckPoint).getKafkaPartitions().getTopic()), + .equals(((KafkaDataSourceMetadata) currentCheckPoint).getSeekableStreamPartitions().getStream()), "Supervisor topic [%s] and topic in checkpoint [%s] does not match", spec.getIoConfig().getTopic(), - ((KafkaDataSourceMetadata) currentCheckPoint).getKafkaPartitions().getTopic() + ((KafkaDataSourceMetadata) currentCheckPoint).getSeekableStreamPartitions().getStream() ); log.info("Checkpointing [%s] for taskGroup [%s]", currentCheckPoint, taskGroupId); @@ -329,7 +328,6 @@ protected KafkaDataSourceMetadata createDataSourceMetaData( @Override protected OrderedSequenceNumber makeSequenceNumber( Long seq, - boolean useExclusive, boolean isExclusive ) { @@ -383,6 +381,18 @@ protected boolean checkSequenceAvailability( && KafkaSequenceNumber.of(latestOffset).compareTo(KafkaSequenceNumber.of(sequenceFromMetadata)) >= 0; } + @Override + protected Long getNotSetMarker() + { + return NOT_SET; + } + + @Override + protected Long getEndOfPartitionMarker() + { + return END_OF_PARTITION; + } + // the following are for unit testing purposes only @Override @VisibleForTesting diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java index 1eee604c9809..17b8b449e1e3 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java @@ -19,15 +19,11 @@ package org.apache.druid.indexing.kafka.supervisor; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; -import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.joda.time.DateTime; import javax.annotation.Nullable; -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; public class KafkaSupervisorReportPayload extends SeekableStreamSupervisorReportPayload { @@ -60,32 +56,12 @@ public KafkaSupervisorReportPayload( } - @JsonProperty - public String getTopic() - { - return getId(); - } - - @Override - @JsonProperty - public List getActiveTasks() - { - return super.getActiveTasks().stream().map(e -> (TaskReportData) e).collect(Collectors.toList()); - } - - @Override - @JsonProperty - public List getPublishingTasks() - { - return super.getPublishingTasks().stream().map(e -> (TaskReportData) e).collect(Collectors.toList()); - } - @Override public String toString() { return "{" + "dataSource='" + getDataSource() + '\'' + - ", topic='" + getTopic() + '\'' + + ", topic='" + getStream() + '\'' + ", partitions=" + getPartitions() + ", replicas=" + getReplicas() + ", durationSeconds=" + getDurationSeconds() + diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index 2df5c6d6d6b5..33ad3e11bee7 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; @@ -36,7 +35,6 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; -import java.util.List; import java.util.Map; public class KafkaSupervisorSpec extends SeekableStreamSupervisorSpec @@ -116,12 +114,6 @@ public Supervisor createSupervisor() ); } - @Override - public List getDataSources() - { - return ImmutableList.of(getDataSchema().getDataSource()); - } - @Override @JsonProperty public KafkaSupervisorTuningConfig getTuningConfig() @@ -163,6 +155,8 @@ public String toString() "dataSchema=" + getDataSchema() + ", tuningConfig=" + getTuningConfig() + ", ioConfig=" + getIoConfig() + + ", context=" + getContext() + + ", suspend=" + isSuspended() + '}'; } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index fe2362ffad7a..ee1c9d1748fb 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -33,6 +33,7 @@ public class KafkaSupervisorTuningConfig extends KafkaTuningConfig implements SeekableStreamSupervisorTuningConfig { + private static final String DEFAULT_OFFSET_FETCH_PERIOD = "PT30S"; private final Integer workerThreads; private final Integer chatThreads; @@ -89,10 +90,16 @@ public KafkaSupervisorTuningConfig( ); this.workerThreads = workerThreads; this.chatThreads = chatThreads; - this.chatRetries = (chatRetries != null ? chatRetries : 8); - this.httpTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(httpTimeout, "PT10S"); - this.shutdownTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(shutdownTimeout, "PT80S"); - this.offsetFetchPeriod = SeekableStreamSupervisorTuningConfig.defaultDuration(offsetFetchPeriod, "PT30S"); + this.chatRetries = (chatRetries != null ? chatRetries : DEFAULT_CHAT_RETRIES); + this.httpTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(httpTimeout, DEFAULT_HTTP_TIMEOUT); + this.shutdownTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration( + shutdownTimeout, + DEFAULT_SHUTDOWN_TIMEOUT + ); + this.offsetFetchPeriod = SeekableStreamSupervisorTuningConfig.defaultDuration( + offsetFetchPeriod, + DEFAULT_OFFSET_FETCH_PERIOD + ); } @Override diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java index fab3789d71a3..e54b9ac0a014 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java @@ -67,10 +67,10 @@ public void testSerdeWithDefaults() throws Exception ); Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); - Assert.assertEquals("mytopic", config.getStartPartitions().getTopic()); - Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartPartitions().getPartitionOffsetMap()); - Assert.assertEquals("mytopic", config.getEndPartitions().getTopic()); - Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndPartitions().getPartitionOffsetMap()); + Assert.assertEquals("mytopic", config.getStartPartitions().getStream()); + Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartPartitions().getPartitionSequenceNumberMap()); + Assert.assertEquals("mytopic", config.getEndPartitions().getStream()); + Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndPartitions().getPartitionSequenceNumberMap()); Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties()); Assert.assertTrue(config.isUseTransaction()); Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent()); @@ -104,10 +104,10 @@ public void testSerdeWithNonDefaults() throws Exception ); Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); - Assert.assertEquals("mytopic", config.getStartPartitions().getTopic()); - Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartPartitions().getPartitionOffsetMap()); - Assert.assertEquals("mytopic", config.getEndPartitions().getTopic()); - Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndPartitions().getPartitionOffsetMap()); + Assert.assertEquals("mytopic", config.getStartPartitions().getStream()); + Assert.assertEquals(ImmutableMap.of(0, 1L, 1, 10L), config.getStartPartitions().getPartitionSequenceNumberMap()); + Assert.assertEquals("mytopic", config.getEndPartitions().getStream()); + Assert.assertEquals(ImmutableMap.of(0, 15L, 1, 200L), config.getEndPartitions().getPartitionSequenceNumberMap()); Assert.assertEquals(ImmutableMap.of("bootstrap.servers", "localhost:9092"), config.getConsumerProperties()); Assert.assertFalse(config.isUseTransaction()); Assert.assertEquals(DateTimes.of("2016-05-31T12:00Z"), config.getMinimumMessageTime().get()); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 5d6af83d6d43..1448be80817a 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -548,8 +548,8 @@ public void testIncrementalHandOff() throws Exception Thread.sleep(10); } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets) || checkpoint2.getPartitionOffsetMap() - .equals(currentOffsets)); + Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets) || checkpoint2.getPartitionSequenceNumberMap() + .equals(currentOffsets)); task.getRunner().setEndOffsets(currentOffsets, false); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); @@ -675,7 +675,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets)); + Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets)); task.getRunner().setEndOffsets(currentOffsets, false); while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { @@ -688,7 +688,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception } final Map nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertTrue(checkpoint2.getPartitionOffsetMap().equals(nextOffsets)); + Assert.assertTrue(checkpoint2.getPartitionSequenceNumberMap().equals(nextOffsets)); task.getRunner().setEndOffsets(nextOffsets, false); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); @@ -816,7 +816,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception Thread.sleep(10); } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertTrue(checkpoint.getPartitionOffsetMap().equals(currentOffsets)); + Assert.assertTrue(checkpoint.getPartitionSequenceNumberMap().equals(currentOffsets)); task.getRunner().setEndOffsets(currentOffsets, false); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); @@ -827,7 +827,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception DATA_SCHEMA.getDataSource(), 0, new KafkaDataSourceMetadata(startPartitions), - new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, checkpoint.getPartitionOffsetMap())) + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, checkpoint.getPartitionSequenceNumberMap())) ) ) ); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index bc7cc612aa19..0b6ac11d0545 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -288,15 +288,15 @@ public void testNoInitialState() throws Exception Assert.assertFalse("maximumMessageTime", taskConfig.getMaximumMessageTime().isPresent()); Assert.assertFalse("skipOffsetGaps", taskConfig.isSkipOffsetGaps()); - Assert.assertEquals(topic, taskConfig.getStartPartitions().getTopic()); - Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); - - Assert.assertEquals(topic, taskConfig.getEndPartitions().getTopic()); - Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(topic, taskConfig.getStartPartitions().getStream()); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); + + Assert.assertEquals(topic, taskConfig.getEndPartitions().getStream()); + Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)); + Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)); } @Test @@ -351,18 +351,27 @@ public void testMultiTask() throws Exception verifyAll(); KafkaIndexTask task1 = captured.getValues().get(0); - Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().size()); - Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getPartitionOffsetMap().size()); - Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(Long.MAX_VALUE, (long) task1.getIOConfig().getEndPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(2)); - Assert.assertEquals(Long.MAX_VALUE, (long) task1.getIOConfig().getEndPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals( + Long.MAX_VALUE, + (long) task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(0) + ); + Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2)); + Assert.assertEquals( + Long.MAX_VALUE, + (long) task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(2) + ); KafkaIndexTask task2 = captured.getValues().get(1); - Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().size()); - Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getPartitionOffsetMap().size()); - Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(Long.MAX_VALUE, (long) task2.getIOConfig().getEndPartitions().getPartitionOffsetMap().get(1)); + Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1)); + Assert.assertEquals( + Long.MAX_VALUE, + (long) task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(1) + ); } @Test @@ -388,18 +397,18 @@ public void testReplicas() throws Exception verifyAll(); KafkaIndexTask task1 = captured.getValues().get(0); - Assert.assertEquals(3, task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().size()); - Assert.assertEquals(3, task1.getIOConfig().getEndPartitions().getPartitionOffsetMap().size()); - Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(3, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(3, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1)); + Assert.assertEquals(0L, (long) task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2)); KafkaIndexTask task2 = captured.getValues().get(1); - Assert.assertEquals(3, task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().size()); - Assert.assertEquals(3, task2.getIOConfig().getEndPartitions().getPartitionOffsetMap().size()); - Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(3, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(3, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1)); + Assert.assertEquals(0L, (long) task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2)); } @Test @@ -506,9 +515,9 @@ public void testLatestOffset() throws Exception verifyAll(); KafkaIndexTask task = captured.getValue(); - Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(1)); + Assert.assertEquals(1100L, (long) task.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(2)); } @Test @@ -540,9 +549,9 @@ public void testDatasourceMetadata() throws Exception KafkaIndexTask task = captured.getValue(); KafkaIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); - Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(30L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); + Assert.assertEquals(30L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); } @Test(expected = ISE.class) @@ -1154,10 +1163,10 @@ public void testBeginPublishAndQueueNextTasks() throws Exception Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); - Assert.assertEquals(topic, taskConfig.getStartPartitions().getTopic()); - Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(35L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(topic, taskConfig.getStartPartitions().getStream()); + Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); + Assert.assertEquals(35L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); } } @@ -1226,7 +1235,7 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); Assert.assertEquals(1, (int) payload.getReplicas()); - Assert.assertEquals(topic, payload.getTopic()); + Assert.assertEquals(topic, payload.getStream()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); @@ -1247,15 +1256,24 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); // check that the new task was created with starting offsets matching where the publishing task finished - Assert.assertEquals(topic, capturedTaskConfig.getStartPartitions().getTopic()); - Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(20L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); - - Assert.assertEquals(topic, capturedTaskConfig.getEndPartitions().getTopic()); - Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(topic, capturedTaskConfig.getStartPartitions().getStream()); + Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals(20L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); + Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); + + Assert.assertEquals(topic, capturedTaskConfig.getEndPartitions().getStream()); + Assert.assertEquals( + Long.MAX_VALUE, + (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0) + ); + Assert.assertEquals( + Long.MAX_VALUE, + (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1) + ); + Assert.assertEquals( + Long.MAX_VALUE, + (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2) + ); } @Test @@ -1314,7 +1332,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); Assert.assertEquals(1, (int) payload.getReplicas()); - Assert.assertEquals(topic, payload.getTopic()); + Assert.assertEquals(topic, payload.getStream()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); @@ -1335,15 +1353,24 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); // check that the new task was created with starting offsets matching where the publishing task finished - Assert.assertEquals(topic, capturedTaskConfig.getStartPartitions().getTopic()); - Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(0L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); - - Assert.assertEquals(topic, capturedTaskConfig.getEndPartitions().getTopic()); - Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(Long.MAX_VALUE, (long) capturedTaskConfig.getEndPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(topic, capturedTaskConfig.getStartPartitions().getStream()); + Assert.assertEquals(10L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals(0L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); + Assert.assertEquals(30L, (long) capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); + + Assert.assertEquals(topic, capturedTaskConfig.getEndPartitions().getStream()); + Assert.assertEquals( + Long.MAX_VALUE, + (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0) + ); + Assert.assertEquals( + Long.MAX_VALUE, + (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1) + ); + Assert.assertEquals( + Long.MAX_VALUE, + (long) capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2) + ); } @Test @@ -1433,7 +1460,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); Assert.assertEquals(NUM_PARTITIONS, (int) payload.getPartitions()); Assert.assertEquals(1, (int) payload.getReplicas()); - Assert.assertEquals(topic, payload.getTopic()); + Assert.assertEquals(topic, payload.getStream()); Assert.assertEquals(1, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); @@ -1590,8 +1617,8 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception for (Task task : captured.getValues()) { KafkaIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig(); - Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); } } @@ -1679,8 +1706,8 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception for (Task task : captured.getValues()) { KafkaIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig(); - Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); } } @@ -2628,15 +2655,15 @@ public void testFailedInitializationAndRecovery() throws Exception Assert.assertFalse("maximumMessageTime", taskConfig.getMaximumMessageTime().isPresent()); Assert.assertFalse("skipOffsetGaps", taskConfig.isSkipOffsetGaps()); - Assert.assertEquals(topic, taskConfig.getStartPartitions().getTopic()); - Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(topic, taskConfig.getStartPartitions().getStream()); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); + Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); - Assert.assertEquals(topic, taskConfig.getEndPartitions().getTopic()); - Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(0)); - Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(1)); - Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionOffsetMap().get(2)); + Assert.assertEquals(topic, taskConfig.getEndPartitions().getStream()); + Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(0)); + Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(1)); + Assert.assertEquals(Long.MAX_VALUE, (long) taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(2)); } @Test diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index ec949136b23f..9711f9d7d0d3 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -117,48 +117,4 @@ - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.1 - - -Xlint:unchecked - 1.8 - 1.8 - - - - org.apache.maven.plugins - maven-assembly-plugin - 2.5.5 - - - maven-release-plugin - 2.5.3 - - - org.apache.maven.scm - maven-scm-provider-gitexe - 1.9.5 - - - - - org.apache.maven.plugins - maven-jar-plugin - 3.0.2 - - - - true - true - - - - - - - diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java index 1c7227f95651..4592c1b67740 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java @@ -36,14 +36,8 @@ public KinesisDataSourceMetadata( super(kinesisPartitions); } - @JsonProperty("partitions") - public SeekableStreamPartitions getKinesisPartitions() - { - return getSeekableStreamPartitions(); - } - @Override - protected KinesisDataSourceMetadata createConcretDataSourceMetaData(String streamName, Map newMap) + protected KinesisDataSourceMetadata createConcreteDataSourceMetaData(String streamName, Map newMap) { return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(streamName, newMap)); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java index 0cee633bdb77..07baeceaae9a 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java @@ -31,8 +31,8 @@ public class KinesisIOConfig extends SeekableStreamIOConfig { private static final boolean DEFAULT_PAUSE_AFTER_READ = true; - private static final int DEFAULT_RECORDS_PER_FETCH = 4000; - private static final int DEFAULT_FETCH_DELAY_MILLIS = 0; + public static final int DEFAULT_RECORDS_PER_FETCH = 4000; + public static final int DEFAULT_FETCH_DELAY_MILLIS = 0; private final boolean pauseAfterRead; private final String endpoint; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index fa5ce08aa1ec..87ea6c1a1613 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -213,7 +213,7 @@ public KinesisIndexTask( this.ingestionState = IngestionState.NOT_STARTED; this.ioConfig = ioConfig; this.tuningConfig = tuningConfig; - this.endOffsets.putAll(ioConfig.getEndPartitions().getMap()); + this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionSequenceNumberMap()); } @Override @@ -267,7 +267,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception appenderator = appenderator0; - final String topic = ioConfig.getStartPartitions().getName(); + final String topic = ioConfig.getStartPartitions().getStream(); // Start up, set up initial offsets. final Object restoredMetadata = driver.startJob(); @@ -277,7 +277,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception topic, ioConfig.getStartPartitions() ); - lastOffsets.putAll(ioConfig.getStartPartitions().getMap()); + lastOffsets.putAll(ioConfig.getStartPartitions().getPartitionSequenceNumberMap()); } else { log.info("found meatadata [%s] for [%s]", restoredMetadata, topic); @SuppressWarnings("unchecked") @@ -293,22 +293,22 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception String.class ) ); - lastOffsets.putAll(restoredNextPartitions.getMap()); + lastOffsets.putAll(restoredNextPartitions.getPartitionSequenceNumberMap()); // Sanity checks. - if (!restoredNextPartitions.getName().equals(ioConfig.getStartPartitions().getName())) { + if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) { throw new ISE( "WTF?! Restored stream[%s] but expected stream[%s]", - restoredNextPartitions.getName(), - ioConfig.getStartPartitions().getName() + restoredNextPartitions.getStream(), + ioConfig.getStartPartitions().getStream() ); } - if (!lastOffsets.keySet().equals(ioConfig.getStartPartitions().getMap().keySet())) { + if (!lastOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet())) { throw new ISE( "WTF?! Restored partitions[%s] but expected partitions[%s]", lastOffsets.keySet(), - ioConfig.getStartPartitions().getMap().keySet() + ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet() ); } } @@ -340,7 +340,7 @@ public Object getMetadata() { return ImmutableMap.of( METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>( - ioConfig.getStartPartitions().getName(), + ioConfig.getStartPartitions().getStream(), snapshot ) ); @@ -544,7 +544,7 @@ public void run() ); // Sanity check, we should only be publishing things that match our desired end state. - if (!endOffsets.equals(finalPartitions.getMap())) { + if (!endOffsets.equals(finalPartitions.getPartitionSequenceNumberMap())) { throw new ISE( "WTF?! Driver attempted to publish invalid metadata[%s], final sequences are [%s]", commitMetadata, @@ -640,7 +640,7 @@ private RecordSupplier getRecordSupplier() { int fetchThreads = tuningConfig.getFetchThreads() != null ? tuningConfig.getFetchThreads() - : Math.max(1, ioConfig.getStartPartitions().getMap().size()); + : Math.max(1, ioConfig.getStartPartitions().getPartitionSequenceNumberMap().size()); return new KinesisRecordSupplier( ioConfig.getEndpoint(), @@ -662,7 +662,7 @@ private RecordSupplier getRecordSupplier() private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) { final int maxRowsInMemoryPerPartition = (tuningConfig.getMaxRowsInMemory() / - ioConfig.getStartPartitions().getMap().size()); + ioConfig.getStartPartitions().getPartitionSequenceNumberMap().size()); return Appenderators.createRealtime( dataSchema, tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), @@ -939,7 +939,7 @@ private void sendResetRequestAndWait( getDataSource(), new KinesisDataSourceMetadata( new SeekableStreamPartitions<>( - ioConfig.getStartPartitions().getName(), + ioConfig.getStartPartitions().getStream(), partitionOffsetMap ) ) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java index d0cb952efa3a..79f9e4ad9707 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java @@ -32,9 +32,9 @@ public class KinesisSequenceNumber extends OrderedSequenceNumber private final BigInteger intSequence; - private KinesisSequenceNumber(@NotNull String sequenceNumber, boolean useExclusive, boolean isExclusive) + private KinesisSequenceNumber(@NotNull String sequenceNumber, boolean isExclusive) { - super(sequenceNumber, useExclusive, isExclusive); + super(sequenceNumber, isExclusive); this.intSequence = OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(sequenceNumber) ? new BigInteger("-1") : new BigInteger(sequenceNumber); @@ -42,17 +42,12 @@ private KinesisSequenceNumber(@NotNull String sequenceNumber, boolean useExclusi public static KinesisSequenceNumber of(String sequenceNumber) { - return new KinesisSequenceNumber(sequenceNumber, false, false); + return new KinesisSequenceNumber(sequenceNumber, false); } - public static KinesisSequenceNumber of(String sequenceNumber, boolean useExclusive, boolean isExclusive) + public static KinesisSequenceNumber of(String sequenceNumber, boolean isExclusive) { - return new KinesisSequenceNumber(sequenceNumber, useExclusive, isExclusive); - } - - public BigInteger getBigInteger() - { - return intSequence; + return new KinesisSequenceNumber(sequenceNumber, isExclusive); } @Override diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java index 4032518e0b69..c648975150ea 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java @@ -24,7 +24,6 @@ import com.google.common.base.Preconditions; import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; @@ -92,10 +91,6 @@ public KinesisTuningConfig( maxParseExceptions, maxSavedParseExceptions ); - // Cannot be a static because default basePersistDirectory is unique per-instance - final RealtimeTuningConfig defaults = RealtimeTuningConfig.makeDefaultTuningConfig(basePersistDirectory); - - this.recordBufferSize = recordBufferSize == null ? DEFAULT_RECORD_BUFFER_SIZE : recordBufferSize; this.recordBufferOfferTimeout = recordBufferOfferTimeout == null ? DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT @@ -197,33 +192,6 @@ public KinesisTuningConfig withBasePersistDirectory(File dir) ); } - public KinesisTuningConfig withMaxRowsInMemory(int rows) - { - return new KinesisTuningConfig( - rows, - getMaxBytesInMemory(), - getMaxRowsPerSegment(), - getIntermediatePersistPeriod(), - getBasePersistDirectory(), - 0, - getIndexSpec(), - true, - isReportParseExceptions(), - getHandoffConditionTimeout(), - isResetOffsetAutomatically(), - isSkipSequenceNumberAvailabilityCheck(), - getRecordBufferSize(), - getRecordBufferOfferTimeout(), - getRecordBufferFullWait(), - getFetchSequenceNumberTimeout(), - getFetchThreads(), - getSegmentWriteOutMediumFactory(), - isLogParseExceptions(), - getMaxParseExceptions(), - getMaxSavedParseExceptions() - ); - } - @Override public boolean equals(Object o) { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index d5faf01b88cf..8812a36760fd 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -97,8 +97,6 @@ public KinesisSupervisor( mapper, spec, rowIngestionMetersFactory, - NOT_SET, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, true, false ); @@ -278,11 +276,10 @@ protected SeekableStreamDataSourceMetadata createDataSourceMetaD @Override protected OrderedSequenceNumber makeSequenceNumber( String seq, - boolean useExclusive, boolean isExclusive ) { - return KinesisSequenceNumber.of(seq, useExclusive, isExclusive); + return KinesisSequenceNumber.of(seq, isExclusive); } // the following are for unit testing purposes only @@ -338,6 +335,18 @@ protected boolean checkSequenceAvailability(@NotNull String partition, @NotNull && KinesisSequenceNumber.of(earliestSequence).compareTo(KinesisSequenceNumber.of(sequenceFromMetadata)) <= 0; } + @Override + protected String getNotSetMarker() + { + return NOT_SET; + } + + @Override + protected String getEndOfPartitionMarker() + { + return SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER; + } + @Override @VisibleForTesting protected void addTaskGroupToActivelyReadingTaskGroup( @@ -391,14 +400,14 @@ public void run() { try { final Map> topics = lagComputingConsumer.listTopics(); - final List partitionInfoList = topics.get(ioConfig.getName()); + final List partitionInfoList = topics.get(ioConfig.getStream()); lagComputingConsumer.assign( Lists.transform(partitionInfoList, new Function() { @Override public TopicPartition apply(PartitionInfo input) { - return new TopicPartition(ioConfig.getName(), input.partition()); + return new TopicPartition(ioConfig.getStream(), input.partition()); } }) ); @@ -448,7 +457,7 @@ public Long apply(Integer key, Long existingOffsetInMap) long lag = 0; for (PartitionInfo partitionInfo : partitionInfoList) { long diff; - final TopicPartition topicPartition = new TopicPartition(ioConfig.getName(), partitionInfo.partition()); + final TopicPartition topicPartition = new TopicPartition(ioConfig.getStream(), partitionInfo.partition()); lagComputingConsumer.seekToEnd(ImmutableList.of(topicPartition)); if (offsetsResponse.get(topicPartition.partition()) != null) { diff = lagComputingConsumer.position(topicPartition) - offsetsResponse.get(topicPartition.partition()); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java index 0854a88ecb49..1b2729dd5e68 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.indexing.kinesis.KinesisIOConfig; import org.apache.druid.indexing.kinesis.KinesisRegion; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.joda.time.Period; @@ -87,8 +88,8 @@ public KinesisSupervisorIOConfig( this.endpoint = endpoint != null ? endpoint : (region != null ? region.getEndpoint() : KinesisRegion.US_EAST_1.getEndpoint()); - this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : 4000; - this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : 0; + this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : KinesisIOConfig.DEFAULT_RECORDS_PER_FETCH; + this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : KinesisIOConfig.DEFAULT_FETCH_DELAY_MILLIS; this.awsAccessKeyId = awsAccessKeyId; this.awsSecretAccessKey = awsSecretAccessKey; this.awsAssumedRoleArn = awsAssumedRoleArn; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java index d7c8ebb716fc..3c29d88c0b80 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java @@ -52,8 +52,9 @@ public KinesisSupervisorReportPayload( public String toString() { return "{" + + "className=" + this.getClass().getSimpleName() + "dataSource='" + getDataSource() + '\'' + - ", stream='" + getId() + '\'' + + ", stream='" + getStream() + '\'' + ", partitions=" + getPartitions() + ", replicas=" + getReplicas() + ", durationSeconds=" + getDurationSeconds() + diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java index 18b4feb0c067..ae872215f44c 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; @@ -36,7 +35,6 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; -import java.util.List; import java.util.Map; public class KinesisSupervisorSpec extends SeekableStreamSupervisorSpec @@ -119,12 +117,6 @@ public Supervisor createSupervisor() ); } - @Override - public List getDataSources() - { - return ImmutableList.of(getDataSchema().getDataSource()); - } - @Override public String toString() { @@ -150,12 +142,6 @@ public KinesisSupervisorIOConfig getIoConfig() return (KinesisSupervisorIOConfig) super.getIoConfig(); } - @Override - public KinesisSupervisorSpec createSuspendedSpec() - { - return toggleSuspend(true); - } - @Override protected KinesisSupervisorSpec toggleSuspend(boolean suspend) { @@ -175,10 +161,4 @@ protected KinesisSupervisorSpec toggleSuspend(boolean suspend) rowIngestionMetersFactory ); } - - @Override - public KinesisSupervisorSpec createRunningSpec() - { - return toggleSuspend(false); - } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 7693f1edd260..7abe07cf02ed 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -93,9 +93,12 @@ public KinesisSupervisorTuningConfig( this.workerThreads = workerThreads; this.chatThreads = chatThreads; - this.chatRetries = (chatRetries != null ? chatRetries : 8); - this.httpTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(httpTimeout, "PT10S"); - this.shutdownTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(shutdownTimeout, "PT80S"); + this.chatRetries = (chatRetries != null ? chatRetries : DEFAULT_CHAT_RETRIES); + this.httpTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(httpTimeout, DEFAULT_HTTP_TIMEOUT); + this.shutdownTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration( + shutdownTimeout, + DEFAULT_SHUTDOWN_TIMEOUT + ); } @Override diff --git a/extensions-core/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule index 5665f7ce1d9c..4c1d6cb0a28e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule +++ b/extensions-core/kinesis-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -1 +1,16 @@ +# 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. + org.apache.druid.indexing.kinesis.KinesisIndexingServiceModule diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index 3ace799c0ce9..c3aabba34ac5 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -67,15 +67,15 @@ public void testSerdeWithDefaults() throws Exception ); Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); - Assert.assertEquals("mystream", config.getStartPartitions().getName()); + Assert.assertEquals("mystream", config.getStartPartitions().getStream()); Assert.assertEquals( ImmutableMap.of("0", "1", "1", "10"), - config.getStartPartitions().getMap() + config.getStartPartitions().getPartitionSequenceNumberMap() ); - Assert.assertEquals("mystream", config.getEndPartitions().getName()); + Assert.assertEquals("mystream", config.getEndPartitions().getStream()); Assert.assertEquals( ImmutableMap.of("0", "15", "1", "200"), - config.getEndPartitions().getMap() + config.getEndPartitions().getPartitionSequenceNumberMap() ); Assert.assertTrue(config.isUseTransaction()); Assert.assertTrue(config.isPauseAfterRead()); @@ -124,15 +124,15 @@ public void testSerdeWithNonDefaults() throws Exception ); Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); - Assert.assertEquals("mystream", config.getStartPartitions().getName()); + Assert.assertEquals("mystream", config.getStartPartitions().getStream()); Assert.assertEquals( ImmutableMap.of("0", "1", "1", "10"), - config.getStartPartitions().getMap() + config.getStartPartitions().getPartitionSequenceNumberMap() ); - Assert.assertEquals("mystream", config.getEndPartitions().getName()); + Assert.assertEquals("mystream", config.getEndPartitions().getStream()); Assert.assertEquals( ImmutableMap.of("0", "15", "1", "200"), - config.getEndPartitions().getMap() + config.getEndPartitions().getPartitionSequenceNumberMap() ); Assert.assertFalse(config.isUseTransaction()); Assert.assertFalse(config.isPauseAfterRead()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index d761fcc9495e..ce87d4ea199e 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -309,24 +309,24 @@ public void testNoInitialState() throws Exception Assert.assertFalse("minimumMessageTime", taskConfig.getMinimumMessageTime().isPresent()); Assert.assertFalse("maximumMessageTime", taskConfig.getMaximumMessageTime().isPresent()); - Assert.assertEquals(stream, taskConfig.getStartPartitions().getName()); + Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); Assert.assertEquals( getSequenceNumber(res, shardId1, 0), - taskConfig.getStartPartitions().getMap().get(shardId1) + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( getSequenceNumber(res, shardId0, 0), - taskConfig.getStartPartitions().getMap().get(shardId0) + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); - Assert.assertEquals(stream, taskConfig.getEndPartitions().getName()); + Assert.assertEquals(stream, taskConfig.getEndPartitions().getStream()); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - taskConfig.getEndPartitions().getMap().get(shardId1) + taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - taskConfig.getEndPartitions().getMap().get(shardId0) + taskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) ); } @@ -355,27 +355,27 @@ public void testMultiTask() throws Exception verifyAll(); KinesisIndexTask task1 = captured.getValues().get(0); - Assert.assertEquals(1, task1.getIOConfig().getStartPartitions().getMap().size()); - Assert.assertEquals(1, task1.getIOConfig().getEndPartitions().getMap().size()); + Assert.assertEquals(1, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(1, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); Assert.assertEquals( getSequenceNumber(res, shardId1, 0), - task1.getIOConfig().getStartPartitions().getMap().get(shardId1) + task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - task1.getIOConfig().getEndPartitions().getMap().get(shardId1) + task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) ); KinesisIndexTask task2 = captured.getValues().get(1); - Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getMap().size()); - Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getMap().size()); + Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); Assert.assertEquals( getSequenceNumber(res, shardId0, 0), - task2.getIOConfig().getStartPartitions().getMap().get(shardId0) + task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - task2.getIOConfig().getEndPartitions().getMap().get(shardId0) + task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) ); } @@ -403,43 +403,43 @@ public void testReplicas() throws Exception verifyAll(); KinesisIndexTask task1 = captured.getValues().get(0); - Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getMap().size()); - Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getMap().size()); + Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); Assert.assertEquals( getSequenceNumber(res, shardId0, 0), - task1.getIOConfig().getStartPartitions().getMap().get(shardId0) + task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - task1.getIOConfig().getEndPartitions().getMap().get(shardId0) + task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) ); Assert.assertEquals( getSequenceNumber(res, shardId1, 0), - task1.getIOConfig().getStartPartitions().getMap().get(shardId1) + task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - task1.getIOConfig().getEndPartitions().getMap().get(shardId1) + task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) ); KinesisIndexTask task2 = captured.getValues().get(1); - Assert.assertEquals(2, task2.getIOConfig().getStartPartitions().getMap().size()); - Assert.assertEquals(2, task2.getIOConfig().getEndPartitions().getMap().size()); + Assert.assertEquals(2, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); + Assert.assertEquals(2, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); Assert.assertEquals( getSequenceNumber(res, shardId0, 0), - task2.getIOConfig().getStartPartitions().getMap().get(shardId0) + task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - task2.getIOConfig().getEndPartitions().getMap().get(shardId0) + task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) ); Assert.assertEquals( getSequenceNumber(res, shardId1, 0), - task2.getIOConfig().getStartPartitions().getMap().get(shardId1) + task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - task2.getIOConfig().getEndPartitions().getMap().get(shardId1) + task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) ); } @@ -563,11 +563,11 @@ public void testDatasourceMetadata() throws Exception Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); Assert.assertEquals( getSequenceNumber(res, shardId1, 2), - taskConfig.getStartPartitions().getMap().get(shardId1) + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( getSequenceNumber(res, shardId0, 1), - taskConfig.getStartPartitions().getMap().get(shardId0) + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); } @@ -1341,14 +1341,14 @@ public void testBeginPublishAndQueueNextTasks() throws Exception Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); - Assert.assertEquals(stream, taskConfig.getStartPartitions().getName()); + Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); Assert.assertEquals( getSequenceNumber(res, shardId1, 3), - taskConfig.getStartPartitions().getMap().get(shardId1) + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( getSequenceNumber(res, shardId0, 1), - taskConfig.getStartPartitions().getMap().get(shardId0) + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); } } @@ -1442,7 +1442,7 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); Assert.assertEquals(2, (int) payload.getPartitions()); Assert.assertEquals(1, (int) payload.getReplicas()); - Assert.assertEquals(stream, payload.getId()); + Assert.assertEquals(stream, payload.getStream()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); @@ -1474,24 +1474,24 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); // check that the new task was created with starting offsets matching where the publishing task finished - Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getName()); + Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); Assert.assertEquals( getSequenceNumber(res, shardId1, 2), - capturedTaskConfig.getStartPartitions().getMap().get(shardId1) + capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( getSequenceNumber(res, shardId0, 1), - capturedTaskConfig.getStartPartitions().getMap().get(shardId0) + capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); - Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getName()); + Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream()); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - capturedTaskConfig.getEndPartitions().getMap().get(shardId1) + capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - capturedTaskConfig.getEndPartitions().getMap().get(shardId0) + capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) ); } @@ -1573,7 +1573,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); Assert.assertEquals(2, (int) payload.getPartitions()); Assert.assertEquals(1, (int) payload.getReplicas()); - Assert.assertEquals(stream, payload.getId()); + Assert.assertEquals(stream, payload.getStream()); Assert.assertEquals(0, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); @@ -1605,24 +1605,24 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); // check that the new task was created with starting offsets matching where the publishing task finished - Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getName()); + Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); Assert.assertEquals( getSequenceNumber(res, shardId1, 2), - capturedTaskConfig.getStartPartitions().getMap().get(shardId1) + capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( getSequenceNumber(res, shardId0, 1), - capturedTaskConfig.getStartPartitions().getMap().get(shardId0) + capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); - Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getName()); + Assert.assertEquals(stream, capturedTaskConfig.getEndPartitions().getStream()); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - capturedTaskConfig.getEndPartitions().getMap().get(shardId1) + capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, - capturedTaskConfig.getEndPartitions().getMap().get(shardId0) + capturedTaskConfig.getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) ); } @@ -1755,7 +1755,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception Assert.assertEquals(3600L, (long) payload.getDurationSeconds()); Assert.assertEquals(2, (int) payload.getPartitions()); Assert.assertEquals(1, (int) payload.getReplicas()); - Assert.assertEquals(stream, payload.getId()); + Assert.assertEquals(stream, payload.getStream()); Assert.assertEquals(1, payload.getActiveTasks().size()); Assert.assertEquals(1, payload.getPublishingTasks().size()); @@ -1945,10 +1945,10 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); Assert.assertEquals( getSequenceNumber(res, shardId1, 0), - taskConfig.getStartPartitions().getMap().get(shardId1) + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertNull( - taskConfig.getStartPartitions().getMap().get(shardId0) + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); } } @@ -2054,7 +2054,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); Assert.assertEquals( getSequenceNumber(res, shardId1, 0), - taskConfig.getStartPartitions().getMap().get(shardId1) + taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index f2c78a66b8ab..0f1558a2f505 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -70,7 +70,7 @@ public boolean createOrUpdateAndStartSupervisor(SupervisorSpec spec) { Preconditions.checkState(started, "SupervisorManager not started"); Preconditions.checkNotNull(spec, "spec"); - Preconditions.checkNotNull(spec.getId(), "spec.getId()"); + Preconditions.checkNotNull(spec.getId(), "spec.getStream()"); Preconditions.checkNotNull(spec.getDataSources(), "spec.getDatasources()"); synchronized (lock) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java index 7658ae2a60ad..3591924f8732 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java @@ -28,20 +28,20 @@ import java.util.Map; import java.util.Objects; -public abstract class SeekableStreamDataSourceMetadata implements DataSourceMetadata +public abstract class SeekableStreamDataSourceMetadata implements DataSourceMetadata { - private final SeekableStreamPartitions seekableStreamPartitions; + private final SeekableStreamPartitions seekableStreamPartitions; @JsonCreator public SeekableStreamDataSourceMetadata( - @JsonProperty("partitions") SeekableStreamPartitions seekableStreamPartitions + @JsonProperty("partitions") SeekableStreamPartitions seekableStreamPartitions ) { this.seekableStreamPartitions = seekableStreamPartitions; } @JsonProperty("partitions") - public SeekableStreamPartitions getSeekableStreamPartitions() + public SeekableStreamPartitions getSeekableStreamPartitions() { return seekableStreamPartitions; } @@ -75,21 +75,21 @@ public DataSourceMetadata plus(DataSourceMetadata other) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; + final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; - if (that.getSeekableStreamPartitions().getName().equals(seekableStreamPartitions.getName())) { + if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) { // Same topic, merge offsets. - final Map newMap = new HashMap<>(); + final Map newMap = new HashMap<>(); - for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { + for (Map.Entry entry : seekableStreamPartitions.getPartitionSequenceNumberMap().entrySet()) { newMap.put(entry.getKey(), entry.getValue()); } - for (Map.Entry entry : that.getSeekableStreamPartitions().getMap().entrySet()) { + for (Map.Entry entry : that.getSeekableStreamPartitions().getPartitionSequenceNumberMap().entrySet()) { newMap.put(entry.getKey(), entry.getValue()); } - return createConcretDataSourceMetaData(seekableStreamPartitions.getName(), newMap); + return createConcreteDataSourceMetaData(seekableStreamPartitions.getStream(), newMap); } else { // Different topic, prefer "other". return other; @@ -109,19 +109,19 @@ public DataSourceMetadata minus(DataSourceMetadata other) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; + final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; - if (that.getSeekableStreamPartitions().getName().equals(seekableStreamPartitions.getName())) { + if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) { // Same stream, remove partitions present in "that" from "this" - final Map newMap = new HashMap<>(); + final Map newMap = new HashMap<>(); - for (Map.Entry entry : seekableStreamPartitions.getMap().entrySet()) { - if (!that.getSeekableStreamPartitions().getMap().containsKey(entry.getKey())) { + for (Map.Entry entry : seekableStreamPartitions.getPartitionSequenceNumberMap().entrySet()) { + if (!that.getSeekableStreamPartitions().getPartitionSequenceNumberMap().containsKey(entry.getKey())) { newMap.put(entry.getKey(), entry.getValue()); } } - return createConcretDataSourceMetaData(seekableStreamPartitions.getName(), newMap); + return createConcreteDataSourceMetaData(seekableStreamPartitions.getStream(), newMap); } else { // Different stream, prefer "this". return this; @@ -155,8 +155,8 @@ public String toString() '}'; } - protected abstract SeekableStreamDataSourceMetadata createConcretDataSourceMetaData( + protected abstract SeekableStreamDataSourceMetadata createConcreteDataSourceMetaData( String streamId, - Map newMap + Map newMap ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java index 7ce86cc215ff..a369cfbef496 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java @@ -62,12 +62,14 @@ public SeekableStreamIOConfig( this.maximumMessageTime = Optional.fromNullable(maximumMessageTime); Preconditions.checkArgument( - startPartitions.getName().equals(endPartitions.getName()), + startPartitions.getStream().equals(endPartitions.getStream()), "start topic/stream and end topic/stream must match" ); Preconditions.checkArgument( - startPartitions.getMap().keySet().equals(endPartitions.getMap().keySet()), + startPartitions.getPartitionSequenceNumberMap() + .keySet() + .equals(endPartitions.getPartitionSequenceNumberMap().keySet()), "start partition set and end partition set must match" ); } @@ -115,6 +117,9 @@ public Optional getMinimumMessageTime() return minimumMessageTime; } + // exclusive starting sequence partitions are used only for kinesis where the starting + // sequence number for certain partitions are discarded because they've already been + // read by a previous task @JsonProperty public abstract Set getExclusiveStartSequenceNumberPartitions(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index 27d406a51db6..464f100ca80a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; +import com.google.errorprone.annotations.DoNotCall; import javax.validation.constraints.NotNull; import java.util.Map; @@ -31,7 +32,7 @@ /** * class that encapsulates a map of partitionId -> sequenceNumber. * To be backward compatible with both Kafka and Kinesis datasource metadata when - * deserializing json. Redundant constrcturo fields stream, topic and + * deserializing json. Redundant constructor fields stream, topic and * partitionSequenceNumberMap and partitionOffsetMap are created. Only one of topic, stream * should have a non-null value and only one of partitionOffsetMap and partitionSequenceNumberMap * should have a non-null value. @@ -41,7 +42,7 @@ * used by Kafka and kinesis (i.e. topic vs. name) * * @param partition id type - * @param sequence number type + * @param sequence number type */ public class SeekableStreamPartitions { @@ -75,28 +76,25 @@ public SeekableStreamPartitions(@NotNull final String id, final Map getMap() - { - return map; - } - @JsonProperty public Map getPartitionSequenceNumberMap() { return map; } + @DoNotCall @JsonProperty public Map getPartitionOffsetMap() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java index 05955b4947cd..5196b52cf34f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java @@ -24,7 +24,11 @@ /** * Represents a Kafka/Kinesis stream sequence number. Mainly used to do - * comparison and indicate whether the sequence number is exclusive + * comparison and indicate whether the sequence number is exclusive. + *

+ * isExclusive is used to indicate if this sequence number is the starting + * sequence of some Kinesis partition and should be discarded because some + * previous task has already read this sequence number * * @param type of sequence number */ @@ -32,12 +36,10 @@ public abstract class OrderedSequenceNumber implements Comparable that = (OrderedSequenceNumber) o; return isExclusive == that.isExclusive && - useExclusive == that.useExclusive && Objects.equals(sequenceNumber, that.sequenceNumber); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java index ee03ed3d2b84..04055c3e068c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java @@ -32,11 +32,17 @@ * (i.e. Kafka consumer, Kinesis streams) * * @param Partition Number Type - * @param Sequence Number Type + * @param Sequence Number Type */ @Beta public interface RecordSupplier extends Closeable { + /** + * assigns the given parittions to this RecordSupplier. Previously + * assigned partitions will be replaced + * + * @param partitions parititions to assign + */ void assign(Set> partitions); /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 3a3acde54459..cc1ddc94d8fc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -385,7 +385,7 @@ public void handle() throws ExecutionException, InterruptedException // We have already verified the topic of the current checkpoint is same with that in ioConfig. // See checkpoint(). if (checkpoint.equals(previousCheckpoint.getSeekableStreamPartitions() - .getMap() + .getPartitionSequenceNumberMap() )) { break; } @@ -438,12 +438,12 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) // Map<{group ID}, List<{pending completion task groups}>> private final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); - // 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 + // The starting offset for a new partition in [partitionGroups] is initially set to getNotSetMarker(). When a new task group + // is created and is assigned partitions, if the offset in [partitionGroups] is getNotSetMarker() it will take the starting // offset value from the metadata store, and if it can't find it there, from stream. Once a task begins // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet- // completed task, which will cause the next set of tasks to begin reading from where the previous task left - // off. If that previous task now fails, we will set the offset in [partitionGroups] back to NOT_SET which will + // off. If that previous task now fails, we will set the offset in [partitionGroups] back to getNotSetMarker() which will // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task // failures during publishing. @@ -478,8 +478,7 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) private final Object stopLock = new Object(); private final Object stateChangeLock = new Object(); private final Object recordSupplierLock = new Object(); - private final SequenceType END_OF_PARTITION; - private final SequenceType NOT_SET; + private final boolean useExclusiveStartingSequence; private final boolean isCheckpointSupported; private boolean listenerRegistered = false; @@ -501,8 +500,6 @@ public SeekableStreamSupervisor( final ObjectMapper mapper, final SeekableStreamSupervisorSpec spec, final RowIngestionMetersFactory rowIngestionMetersFactory, - final SequenceType NOT_SET, - final SequenceType END_OF_PARTITION, final boolean useExclusiveStartingSequence, final boolean isCheckpointSupported ) @@ -513,8 +510,6 @@ public SeekableStreamSupervisor( this.sortingMapper = mapper.copy().configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); this.spec = spec; this.rowIngestionMetersFactory = rowIngestionMetersFactory; - this.NOT_SET = NOT_SET; - this.END_OF_PARTITION = END_OF_PARTITION; this.useExclusiveStartingSequence = useExclusiveStartingSequence; this.isCheckpointSupported = isCheckpointSupported; @@ -1128,7 +1123,7 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) @SuppressWarnings("unchecked") final SeekableStreamDataSourceMetadata resetMetadata = (SeekableStreamDataSourceMetadata) dataSourceMetadata; - if (resetMetadata.getSeekableStreamPartitions().getName().equals(ioConfig.getId())) { + if (resetMetadata.getSeekableStreamPartitions().getStream().equals(ioConfig.getId())) { // metadata can be null final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); if (metadata != null && !checkSourceMetadataMatch(metadata)) { @@ -1145,12 +1140,12 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) // as well as the case where the metadata store do not have an entry for the reset partitions boolean doReset = false; for (Entry resetPartitionOffset : resetMetadata.getSeekableStreamPartitions() - .getMap() + .getPartitionSequenceNumberMap() .entrySet()) { final SequenceType partitionOffsetInMetadataStore = currentMetadata == null ? null : currentMetadata.getSeekableStreamPartitions() - .getMap() + .getPartitionSequenceNumberMap() .get(resetPartitionOffset.getKey()); final TaskGroup partitionTaskGroup = taskGroups.get( getTaskGroupIdForPartition(resetPartitionOffset.getKey()) @@ -1183,11 +1178,11 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) } } if (metadataUpdateSuccess) { - resetMetadata.getSeekableStreamPartitions().getMap().keySet().forEach(partition -> { + resetMetadata.getSeekableStreamPartitions().getPartitionSequenceNumberMap().keySet().forEach(partition -> { final int groupId = getTaskGroupIdForPartition(partition); killTaskGroupForPartitions(ImmutableSet.of(partition)); taskGroups.remove(groupId); - partitionGroups.get(groupId).replaceAll((partitionId, offset) -> NOT_SET); + partitionGroups.get(groupId).replaceAll((partitionId, offset) -> getNotSetMarker()); }); } else { throw new ISE("Unable to reset metadata"); @@ -1195,7 +1190,7 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) } else { log.warn( "Reset metadata topic [%s] and supervisor's stream name [%s] do not match", - resetMetadata.getSeekableStreamPartitions().getName(), + resetMetadata.getSeekableStreamPartitions().getStream(), ioConfig.getId() ); } @@ -1273,7 +1268,7 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti Iterator it = seekableStreamIndexTask.getIOConfig() .getStartPartitions() - .getMap() + .getPartitionSequenceNumberMap() .keySet() .iterator(); final Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null); @@ -1297,7 +1292,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) if (status == SeekableStreamIndexTask.Status.PUBLISHING) { seekableStreamIndexTask.getIOConfig() .getStartPartitions() - .getMap() + .getPartitionSequenceNumberMap() .keySet() .forEach( partition -> addDiscoveredTaskToPendingCompletionTaskGroups( @@ -1305,7 +1300,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) taskId, seekableStreamIndexTask.getIOConfig() .getStartPartitions() - .getMap() + .getPartitionSequenceNumberMap() )); // update partitionGroups with the publishing task's offsets (if they are greater than what is @@ -1332,7 +1327,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) } else { for (PartitionType partition : seekableStreamIndexTask.getIOConfig() .getStartPartitions() - .getMap() + .getPartitionSequenceNumberMap() .keySet()) { if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { log.warn( @@ -1372,7 +1367,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) ImmutableMap.copyOf( seekableStreamIndexTask.getIOConfig() .getStartPartitions() - .getMap() + .getPartitionSequenceNumberMap() ), seekableStreamIndexTask.getIOConfig().getMinimumMessageTime(), seekableStreamIndexTask.getIOConfig().getMaximumMessageTime(), @@ -1503,11 +1498,11 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) final boolean hasValidOffsetsFromDb = latestDataSourceMetadata != null && latestDataSourceMetadata.getSeekableStreamPartitions() != null && ioConfig.getId().equals( - latestDataSourceMetadata.getSeekableStreamPartitions().getName() + latestDataSourceMetadata.getSeekableStreamPartitions().getStream() ); final Map latestOffsetsFromDb; if (hasValidOffsetsFromDb) { - latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamPartitions().getMap(); + latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamPartitions().getPartitionSequenceNumberMap(); } else { latestOffsetsFromDb = null; } @@ -1584,7 +1579,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) // 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); taskGroups.remove(groupId); - partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET); + partitionGroups.get(groupId).replaceAll((partition, offset) -> getNotSetMarker()); } taskSequences.stream().filter(taskIdSequences -> tasksToKill.contains(taskIdSequences.lhs)).forEach( @@ -1680,7 +1675,7 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) return generateSequenceName( task.getIOConfig() .getStartPartitions() - .getMap(), + .getPartitionSequenceNumberMap(), task.getIOConfig().getMinimumMessageTime(), task.getIOConfig().getMaximumMessageTime() ).equals(taskSequenceName); @@ -1768,7 +1763,7 @@ private void updatePartitionDataFromStream() k -> new ConcurrentHashMap<>() ); - if (partitionMap.putIfAbsent(partitionId, NOT_SET) == null) { + if (partitionMap.putIfAbsent(partitionId, getNotSetMarker()) == null) { log.info( "New partition [%s] discovered for topic [%s], added to task group [%d]", partitionId, @@ -1907,7 +1902,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } // clear partitionGroups, so that latest offsets from db is used as start offsets not the stale ones // if tasks did some successful incremental handoffs - partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET); + partitionGroups.get(groupId).replaceAll((partition, offset) -> getNotSetMarker()); } // remove this task group from the list of current task groups now that it has been handled @@ -2141,7 +2136,7 @@ private void checkPendingCompletionTasks() } // 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) -> getNotSetMarker()); // kill all the tasks in this pending completion group killTasksInGroup(group); // set a flag so the other pending completion groups for this set of partitions will also stop @@ -2253,11 +2248,14 @@ private void createNewTasks() throws JsonProcessingException ImmutableMap::copyOf )); - Set exclusiveStartSequenceNumberPartitions = startingOffsets - .entrySet().stream() - .filter(x -> x.getValue().get() != null && x.getValue().isExclusive()) - .map(Entry::getKey) - .collect(Collectors.toSet()); + Set exclusiveStartSequenceNumberPartitions = !useExclusiveStartingSequence + ? Collections.emptySet() + : startingOffsets + .entrySet().stream() + .filter(x -> x.getValue().get() != null + && x.getValue().isExclusive()) + .map(Entry::getKey) + .collect(Collectors.toSet()); taskGroups.put( groupId, @@ -2346,10 +2344,10 @@ private ImmutableMap> generat PartitionType partition = entry.getKey(); SequenceType offset = entry.getValue(); - if (!NOT_SET.equals(offset)) { + if (!getNotSetMarker().equals(offset)) { // if we are given a startingOffset (set by a previous task group which is pending completion) then use it if (!OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(offset)) { - builder.put(partition, makeSequenceNumber(offset, useExclusiveStartingSequence, true)); + builder.put(partition, makeSequenceNumber(offset, true)); } } else { // if we don't have a startingOffset (first run or we had some previous failures and reset the offsets) then @@ -2401,7 +2399,7 @@ private OrderedSequenceNumber getOffsetFromStorageForPartition(Par throw new ISE(e, "Timeout while fetching earliest sequence number for partition [%s]", partition); } } - return makeSequenceNumber(offset, useExclusiveStartingSequence, true); + return makeSequenceNumber(offset, true); } else { boolean useEarliestSequenceNumber = ioConfig.isUseEarliestSequenceNumber(); if (subsequentlyDiscoveredPartitions.contains(partition)) { @@ -2414,7 +2412,7 @@ private OrderedSequenceNumber getOffsetFromStorageForPartition(Par offset = getOffsetFromStreamForPartition(partition, useEarliestSequenceNumber); log.info("Getting sequence number [%s] for partition [%s]", offset, partition); - return makeSequenceNumber(offset, useExclusiveStartingSequence, false); + return makeSequenceNumber(offset, false); } } @@ -2427,15 +2425,15 @@ && checkSourceMetadataMatch(dataSourceMetadata)) { SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata) .getSeekableStreamPartitions(); if (partitions != null) { - if (!ioConfig.getId().equals(partitions.getName())) { + if (!ioConfig.getId().equals(partitions.getStream())) { log.warn( "Topic/stream in metadata storage [%s] doesn't match spec topic/stream [%s], ignoring stored offsets", - partitions.getName(), + partitions.getStream(), ioConfig.getId() ); return Collections.emptyMap(); - } else if (partitions.getMap() != null) { - return partitions.getMap(); + } else if (partitions.getPartitionSequenceNumberMap() != null) { + return partitions.getPartitionSequenceNumberMap(); } } } @@ -2464,7 +2462,7 @@ private void createTasksForGroup(int groupId, int replicas) throws JsonProcessin Map startPartitions = group.startingSequences; Map endPartitions = new HashMap<>(); for (PartitionType partition : startPartitions.keySet()) { - endPartitions.put(partition, END_OF_PARTITION); + endPartitions.put(partition, getEndOfPartitionMarker()); } Set exclusiveStartSequenceNumberPartitions = taskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; @@ -2594,7 +2592,7 @@ protected Map getHighestCurrentOffsets() private OrderedSequenceNumber makeSequenceNumber(SequenceType seq) { - return makeSequenceNumber(seq, false, false); + return makeSequenceNumber(seq, false); } @@ -2719,7 +2717,6 @@ protected abstract SeekableStreamDataSourceMetadata */ protected abstract OrderedSequenceNumber makeSequenceNumber( SequenceType seq, - boolean useExclusive, boolean isExclusive ); @@ -2766,4 +2763,8 @@ protected abstract boolean checkSequenceAvailability( @NotNull SequenceType sequenceFromMetadata ) throws TimeoutException; + + protected abstract SequenceType getNotSetMarker(); + + protected abstract SequenceType getEndOfPartitionMarker(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java index 843e75418fed..b6f99664086c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.Map; +@JsonInclude(JsonInclude.Include.NON_NULL) public abstract class SeekableStreamSupervisorReportPayload { private final String dataSource; @@ -88,7 +89,7 @@ public String getDataSource() return dataSource; } - public String getId() + public String getStream() { return id; } @@ -130,21 +131,18 @@ public List getPublishingTasks() } @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) public Map getLatestOffsets() { return latestOffsets; } @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) public Map getMinimumLag() { return minimumLag; } @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) public Long getAggregateLag() { return aggregateLag; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java index b926303542f9..753d03f3448d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java @@ -27,6 +27,11 @@ public interface SeekableStreamSupervisorTuningConfig { + int DEFAULT_CHAT_RETRIES = 8; + String DEFAULT_HTTP_TIMEOUT = "PT10S"; + String DEFAULT_SHUTDOWN_TIMEOUT = "PT80S"; + + static Duration defaultDuration(final Period period, final String theDefault) { return (period == null ? new Period(theDefault) : period).toStandardDuration(); diff --git a/pom.xml b/pom.xml index 145d9a545539..dee452e950e7 100644 --- a/pom.xml +++ b/pom.xml @@ -20,1462 +20,1462 @@ - 4.0.0 + 4.0.0 - - org.apache - apache - 21 - + + org.apache + apache + 21 + - org.apache.druid - druid - 0.13.0-incubating-SNAPSHOT - pom + org.apache.druid + druid + 0.13.0-incubating-SNAPSHOT + pom - Druid - Druid - A Distributed Column Store - https://druid.apache.org/ + Druid + Druid - A Distributed Column Store + https://druid.apache.org/ - - - Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0 - - + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0 + + - - - Druid Committers - http://druid.io/community/index.html#committers - - + + + Druid Committers + http://druid.io/community/index.html#committers + + - - - Apache Druid (incubating) developers list - dev-subscribe@druid.apache.org - dev-unsubscribe@druid.apache.org - dev@druid.apache.org - https://mail-archives.apache.org/mod_mbox/druid-dev - - + + + Apache Druid (incubating) developers list + dev-subscribe@druid.apache.org + dev-unsubscribe@druid.apache.org + dev@druid.apache.org + https://mail-archives.apache.org/mod_mbox/druid-dev + + - 2011 + 2011 - - scm:git:ssh://git@github.com/apache/incubator-druid.git - scm:git:ssh://git@github.com/apache/incubator-druid.git - https://github.com/apache/incubator-druid.git - 0.13.0-incubating-SNAPSHOT - + + scm:git:ssh://git@github.com/apache/incubator-druid.git + scm:git:ssh://git@github.com/apache/incubator-druid.git + https://github.com/apache/incubator-druid.git + 0.13.0-incubating-SNAPSHOT + - - 1.8 - 1.8 - UTF-8 - 4.0.0 - 2.12.0 - 1.10.0 - 1.17.0 - 4.0.0 - 8.1.0 - 16.0.1 - 4.1.0 - 9.4.10.v20180503 - 1.19.3 - - 2.6.7 - 2.5 - - 3.10.6.Final - - 4.1.30.Final - 1.7.12 - - 2.8.3 - 2.0.0 - 1.6.6 - 1.11.199 - 2.5.5 - - 3.4.11 - apache.snapshots - Apache Snapshot Repository - https://repository.apache.org/snapshots + + 1.8 + 1.8 + UTF-8 + 4.0.0 + 2.12.0 + 1.10.0 + 1.17.0 + 4.0.0 + 8.1.0 + 16.0.1 + 4.1.0 + 9.4.10.v20180503 + 1.19.3 + + 2.6.7 + 2.5 + + 3.10.6.Final + + 4.1.30.Final + 1.7.12 + + 2.8.3 + 2.0.0 + 1.6.6 + 1.11.199 + 2.5.5 + + 3.4.11 + apache.snapshots + Apache Snapshot Repository + https://repository.apache.org/snapshots - - 3 - + + 3 + - - core - examples - indexing-hadoop - indexing-service - processing - server - sql - services - integration-tests - benchmarks - aws-common - extendedset - hll - - extensions-core/avro-extensions - extensions-core/datasketches - extensions-core/druid-bloom-filter - extensions-core/druid-kerberos - extensions-core/hdfs-storage - extensions-core/histogram - extensions-core/stats - extensions-core/kafka-eight - extensions-core/kafka-extraction-namespace - extensions-core/kafka-indexing-service - extensions-core/kinesis-indexing-service - extensions-core/mysql-metadata-storage - extensions-core/postgresql-metadata-storage - extensions-core/protobuf-extensions - extensions-core/lookups-cached-global - extensions-core/lookups-cached-single - extensions-core/s3-extensions - extensions-core/simple-client-sslcontext - extensions-core/druid-basic-security - - extensions-contrib/influx-extensions - extensions-contrib/azure-extensions - extensions-contrib/cassandra-storage - extensions-contrib/druid-rocketmq - extensions-contrib/cloudfiles-extensions - extensions-contrib/graphite-emitter - extensions-contrib/kafka-eight-simpleConsumer - extensions-contrib/rabbitmq - extensions-contrib/distinctcount - extensions-contrib/parquet-extensions - extensions-contrib/statsd-emitter - extensions-contrib/orc-extensions - extensions-contrib/time-min-max - extensions-contrib/google-extensions - extensions-contrib/virtual-columns - extensions-contrib/thrift-extensions - extensions-contrib/ambari-metrics-emitter - extensions-contrib/sqlserver-metadata-storage - extensions-contrib/kafka-emitter - extensions-contrib/redis-cache - extensions-contrib/opentsdb-emitter - extensions-contrib/materialized-view-maintenance - extensions-contrib/materialized-view-selection - - distribution - + + core + examples + indexing-hadoop + indexing-service + processing + server + sql + services + integration-tests + benchmarks + aws-common + extendedset + hll + + extensions-core/avro-extensions + extensions-core/datasketches + extensions-core/druid-bloom-filter + extensions-core/druid-kerberos + extensions-core/hdfs-storage + extensions-core/histogram + extensions-core/stats + extensions-core/kafka-eight + extensions-core/kafka-extraction-namespace + extensions-core/kafka-indexing-service + extensions-core/kinesis-indexing-service + extensions-core/mysql-metadata-storage + extensions-core/postgresql-metadata-storage + extensions-core/protobuf-extensions + extensions-core/lookups-cached-global + extensions-core/lookups-cached-single + extensions-core/s3-extensions + extensions-core/simple-client-sslcontext + extensions-core/druid-basic-security + + extensions-contrib/influx-extensions + extensions-contrib/azure-extensions + extensions-contrib/cassandra-storage + extensions-contrib/druid-rocketmq + extensions-contrib/cloudfiles-extensions + extensions-contrib/graphite-emitter + extensions-contrib/kafka-eight-simpleConsumer + extensions-contrib/rabbitmq + extensions-contrib/distinctcount + extensions-contrib/parquet-extensions + extensions-contrib/statsd-emitter + extensions-contrib/orc-extensions + extensions-contrib/time-min-max + extensions-contrib/google-extensions + extensions-contrib/virtual-columns + extensions-contrib/thrift-extensions + extensions-contrib/ambari-metrics-emitter + extensions-contrib/sqlserver-metadata-storage + extensions-contrib/kafka-emitter + extensions-contrib/redis-cache + extensions-contrib/opentsdb-emitter + extensions-contrib/materialized-view-maintenance + extensions-contrib/materialized-view-selection + + distribution + - - - ${repoOrgId} - ${repoOrgName} - ${repoOrgUrl} - - + + + ${repoOrgId} + ${repoOrgName} + ${repoOrgUrl} + + - - - - - commons-codec - commons-codec - 1.7 - - - commons-httpclient - commons-httpclient - 3.1 - - - commons-io - commons-io - 2.5 - - - commons-logging - commons-logging - 1.1.1 - - - commons-lang - commons-lang - 2.6 - - - com.amazonaws - aws-java-sdk-ec2 - ${aws.sdk.version} - - - com.amazonaws - aws-java-sdk-s3 - ${aws.sdk.version} - - - com.ning - compress-lzf - 1.0.4 - - - io.airlift - airline - 0.7 - - - - com.google.code.findbugs - annotations - - - - - org.skife.config - config-magic - 0.9 - - - org.slf4j - slf4j-api - - - - - org.apache.zookeeper - zookeeper - ${zookeeper.version} - - - org.slf4j - slf4j-log4j12 - - - log4j - log4j - - - io.netty - netty - - - - - org.apache.curator - curator-client - ${apache.curator.version} - - - org.apache.curator - curator-framework - ${apache.curator.version} - - - org.jboss.netty - netty - - - - - org.apache.curator - curator-recipes - ${apache.curator.version} - - - org.apache.zookeeper - zookeeper - - - - - org.apache.curator - curator-x-discovery - ${apache.curator.version} - - - org.apache.calcite - calcite-core - ${calcite.version} - - - org.apache.calcite - calcite-linq4j - ${calcite.version} - - - org.apache.calcite.avatica - avatica-core - ${avatica.version} - - - org.apache.calcite.avatica - avatica-server - ${avatica.version} - - - com.google.guava - guava - ${guava.version} - - - com.google.inject - guice - ${guice.version} - - - com.google.inject.extensions - guice-servlet - ${guice.version} - - - com.google.inject.extensions - guice-multibindings - ${guice.version} - - - com.google.errorprone - error_prone_annotations - 2.2.0 - - - com.ibm.icu - icu4j - 54.1.1 - - - org.mozilla - rhino - 1.7R5 - - - org.apache.commons - commons-compress - 1.16 - - - org.tukaani - xz - 1.8 - - - com.github.luben - zstd-jni - 1.3.3-1 - - - com.fasterxml.jackson.core - jackson-annotations - ${jackson.version} - - - com.fasterxml.jackson.core - jackson-core - ${jackson.version} - - - com.fasterxml.jackson.core - jackson-databind - ${jackson.version} - - - com.fasterxml.jackson.datatype - jackson-datatype-guava - ${jackson.version} - - - com.fasterxml.jackson.datatype - jackson-datatype-joda - ${jackson.version} - - - com.fasterxml.jackson.dataformat - jackson-dataformat-smile - ${jackson.version} - - - com.fasterxml.jackson.jaxrs - jackson-jaxrs-json-provider - ${jackson.version} - - - com.fasterxml.jackson.jaxrs - jackson-jaxrs-smile-provider - ${jackson.version} - - - org.hibernate - hibernate-validator - 5.1.3.Final - - - javax.validation - validation-api - 1.1.0.Final - - - javax.inject - javax.inject - 1 - - - javax.el - javax.el-api - 3.0.0 - - - org.glassfish - javax.el - 3.0.0 - - - org.jdbi - jdbi - 2.63.1 - - - com.sun.jersey - jersey-core - ${jersey.version} - - - com.sun.jersey.contribs - jersey-guice - ${jersey.version} - - - com.google.inject - guice - - - com.google.inject.extensions - guice-servlet - - - - - com.sun.jersey - jersey-server - ${jersey.version} - - - com.sun.jersey - jersey-json - ${jersey.version} - - - org.eclipse.jetty - jetty-server - ${jetty.version} - - - org.eclipse.jetty - jetty-servlet - ${jetty.version} - - - org.eclipse.jetty - jetty-servlets - ${jetty.version} - - - org.eclipse.jetty - jetty-proxy - ${jetty.version} - - - org.eclipse.jetty - jetty-util - ${jetty.version} - - - org.eclipse.jetty - jetty-security - ${jetty.version} - - - io.netty - netty - ${netty3.version} - - - io.netty - netty-all - ${netty4.version} - - - joda-time - joda-time - 2.9.9 - - - com.google.code.findbugs - jsr305 - 2.0.1 - - - org.apache.logging.log4j - log4j-api - ${log4j.version} - - - org.apache.logging.log4j - log4j-core - ${log4j.version} - - - org.apache.logging.log4j - log4j-slf4j-impl - ${log4j.version} - - - org.apache.logging.log4j - log4j-1.2-api - ${log4j.version} - - - - org.apache.logging.log4j - log4j-jul - ${log4j.version} - - - org.slf4j - jcl-over-slf4j - ${slf4j.version} - - - com.lmax - disruptor - 3.3.6 - - - net.spy - spymemcached - 2.12.3 - - - org.antlr - antlr4-runtime - 4.5.1 - - - org.antlr - antlr4-coordinator - 4.5.1 - - - commons-cli - commons-cli - 1.2 - - - org.apache.commons - commons-dbcp2 - 2.0.1 - - - org.lz4 - lz4-java - 1.5.0 - - - com.google.protobuf - protobuf-java - 3.1.0 - - - io.tesla.aether - tesla-aether - 0.0.5 - - - org.slf4j - slf4j-api - - - - - org.eclipse.aether - aether-api - 0.9.0.M2 - - - org.apache.httpcomponents - httpclient - 4.5.3 - - - org.apache.httpcomponents - httpcore - 4.4.4 - - - org.apache.hadoop - hadoop-client - ${hadoop.compile.version} - provided - - - org.apache.avro - avro - - - - - org.mapdb - mapdb - 1.0.8 - - - org.apache.derby - derbynet - 10.11.1.1 - - - org.apache.derby - derbyclient - 10.11.1.1 - - - org.apache.commons - commons-math3 - 3.6.1 - - - it.unimi.dsi - fastutil - 8.1.0 - - - com.opencsv - opencsv - 4.2 - - - com.jayway.jsonpath - json-path - 2.3.0 - - - net.thisptr - jackson-jq - 0.0.7 - - - org.slf4j - slf4j-api - 1.6.4 - - - org.roaringbitmap - RoaringBitmap - 0.5.18 - - - org.ow2.asm - asm - 5.2 - - - org.ow2.asm - asm-commons - 5.2 - - - org.asynchttpclient - async-http-client - - 2.5.3 - - - org.gridkit.lab - jvm-attach-api - 1.2 - - - net.java.dev.jna - jna - 4.5.1 - - - io.dropwizard.metrics - metrics-core - ${dropwizard.metrics.version} - - - org.codehaus.jackson - jackson-core-asl - 1.9.13 - - - org.codehaus.jackson - jackson-mapper-asl - 1.9.13 - - - javax.servlet - javax.servlet-api - 3.1.0 - - - javax.activation - activation - 1.1.1 - - - commons-pool - commons-pool - 1.6 - - - org.codehaus.plexus - plexus-utils - 3.0.15 - - - com.github.ben-manes.caffeine - caffeine - ${caffeine.version} - - - - org.apache.calcite - calcite-core - ${calcite.version} - test-jar - test - - - org.easymock - easymock - 3.4 - test - - - junit - junit - 4.12 - test - - - org.powermock - powermock-module-junit4 - ${powermock.version} - test - - - org.powermock - powermock-api-easymock - ${powermock.version} - test - - - org.slf4j - slf4j-simple - ${slf4j.version} - - - com.carrotsearch - junit-benchmarks - 0.7.2 - test - - - com.google.caliper - caliper - 0.5-rc1 - test - - - org.apache.curator - curator-test - ${apache.curator.test.version} - - - org.jboss.netty - netty - - - test - - - com.ircclouds.irc - irc-api - 1.0-0014 - - - org.slf4j - slf4j-api - - - - - com.maxmind.geoip2 - geoip2 - 0.4.0 - - - com.google.http-client - google-http-client - - - com.google.http-client - google-http-client-jackson2 - - - com.fasterxml.jackson.core - jackson-databind - - - - - org.testng - testng - 6.8.7 - - - org.hamcrest - hamcrest-all - 1.3 - test - - - pl.pragmatists - JUnitParams - 1.0.4 - test - - - com.google.guava - guava-testlib - ${guava.version} - test - - - - - - - - org.codehaus.mojo - cobertura-maven-plugin - - - org.eluder.coveralls - coveralls-maven-plugin - 4.0.0 - - - org.apache.maven.plugins - maven-checkstyle-plugin - 2.17 - - ${project.build.sourceDirectory} - true - codestyle/checkstyle.xml - codestyle/checkstyle-suppressions.xml - checkstyle.suppressions.file - UTF-8 - codestyle/LICENSE.txt - true - true - + - - com.puppycrawl.tools - checkstyle - 8.0 - - - - - validate - validate - - check - - - - - - org.apache.maven.plugins - maven-pmd-plugin - 3.8 - - true - - /rulesets/java/imports.xml - - - target/generated-sources/ - - - - - validate - validate - - check - - - - - - de.thetaphi - forbiddenapis - 2.3 - - false - - - jdk-unsafe - - - ${project.parent.basedir}/codestyle/joda-time-forbidden-apis.txt - ${project.parent.basedir}/codestyle/druid-forbidden-apis.txt - - - **.SuppressForbidden - - - - - compile - compile - - check - - - - - jdk-unsafe - jdk-system-out - - - - - testCompile - test-compile - - testCheck - - - - - - org.codehaus.mojo - animal-sniffer-maven-plugin - 1.15 - - - check-java-api - test - - check - - - - org.codehaus.mojo.signature - - java18 - 1.0 - - - - sun.nio.ch.DirectBuffer - sun.misc.Cleaner - sun.misc.Unsafe - - - - - - - org.apache.maven.plugins - maven-enforcer-plugin - 1.4.1 - - - enforce-banned-dependencies - - enforce - - - - - 1.8.0 - - - - - com.google.code.findbugs:annotations - - - - true - - - - - - org.apache.maven.plugins - maven-jar-plugin - - - - ${java.version} (${java.vendor} ${java.vm.version}) - ${os.name} ${os.arch} ${os.version} - ${git.build.time} - ${git.build.version} - ${git.commit.id} - ${git.commit.id.describe} - - - - - - pl.project13.maven - git-commit-id-plugin - 2.2.5 - - - - revision - - - - - ${project.basedir}/.git - Etc/UTC - false - json - true - ${project.build.directory}/git.version - false - - git.build.user.email - git.build.host - git.commit.id.describe-short - git.commit.user.* - git.commit.message.* - git.closest.tag.* - git.commit.id.abbrev - git.dirty - - - false - true - 7 - -dirty - true - - - - - - - - org.apache.maven.plugins - maven-surefire-plugin - 2.19.1 - - - - - -Xmx1500m - -XX:MaxDirectMemorySize=512m - -Duser.language=en - -Duser.GroupByQueryRunnerTest.javacountry=US - -Dfile.encoding=UTF-8 - -Duser.timezone=UTC - -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager - - -Ddruid.indexing.doubleStorage=double - - false - - true - - - - org.apache.maven.plugins - maven-release-plugin - - true - - - - org.apache.maven.plugins - maven-clean-plugin - 2.5 - - - org.apache.maven.plugins - maven-dependency-plugin - 2.8 - - - org.apache.maven.plugins - maven-deploy-plugin - 2.7 - - - org.apache.maven.plugins - maven-help-plugin - 2.1.1 - - - org.apache.maven.plugins - maven-install-plugin - 2.3.1 - - - package - - install - - - - - - org.apache.maven.plugins - maven-resources-plugin - 2.5 - - - org.apache.maven.plugins - maven-shade-plugin - 2.2 - - - org.apache.maven.plugins - maven-site-plugin - 3.1 - - - org.scala-tools - maven-scala-plugin - 2.15.2 - - - org.antlr - antlr4-maven-plugin - 4.5.1 - - - org.apache.maven.plugins - maven-assembly-plugin - 3.1.0 - - - org.codehaus.mojo - license-maven-plugin - 1.8 - - - org.codehaus.mojo - exec-maven-plugin - 1.2.1 - - - org.apache.maven.plugins - maven-javadoc-plugin - 2.10.1 - - - -Xdoclint:none + + + commons-codec + commons-codec + 1.7 + + + commons-httpclient + commons-httpclient + 3.1 + + + commons-io + commons-io + 2.5 + + + commons-logging + commons-logging + 1.1.1 + + + commons-lang + commons-lang + 2.6 + + + com.amazonaws + aws-java-sdk-ec2 + ${aws.sdk.version} + + + com.amazonaws + aws-java-sdk-s3 + ${aws.sdk.version} + + + com.ning + compress-lzf + 1.0.4 + + + io.airlift + airline + 0.7 + + + + com.google.code.findbugs + annotations + + + + + org.skife.config + config-magic + 0.9 + + + org.slf4j + slf4j-api + + + + + org.apache.zookeeper + zookeeper + ${zookeeper.version} + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + io.netty + netty + + + + + org.apache.curator + curator-client + ${apache.curator.version} + + + org.apache.curator + curator-framework + ${apache.curator.version} + + + org.jboss.netty + netty + + + + + org.apache.curator + curator-recipes + ${apache.curator.version} + + + org.apache.zookeeper + zookeeper + + + + + org.apache.curator + curator-x-discovery + ${apache.curator.version} + + + org.apache.calcite + calcite-core + ${calcite.version} + + + org.apache.calcite + calcite-linq4j + ${calcite.version} + + + org.apache.calcite.avatica + avatica-core + ${avatica.version} + + + org.apache.calcite.avatica + avatica-server + ${avatica.version} + + + com.google.guava + guava + ${guava.version} + + + com.google.inject + guice + ${guice.version} + + + com.google.inject.extensions + guice-servlet + ${guice.version} + + + com.google.inject.extensions + guice-multibindings + ${guice.version} + + + com.google.errorprone + error_prone_annotations + 2.2.0 + + + com.ibm.icu + icu4j + 54.1.1 + + + org.mozilla + rhino + 1.7R5 + + + org.apache.commons + commons-compress + 1.16 + + + org.tukaani + xz + 1.8 + + + com.github.luben + zstd-jni + 1.3.3-1 + + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + com.fasterxml.jackson.datatype + jackson-datatype-guava + ${jackson.version} + + + com.fasterxml.jackson.datatype + jackson-datatype-joda + ${jackson.version} + + + com.fasterxml.jackson.dataformat + jackson-dataformat-smile + ${jackson.version} + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + ${jackson.version} + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-smile-provider + ${jackson.version} + + + org.hibernate + hibernate-validator + 5.1.3.Final + + + javax.validation + validation-api + 1.1.0.Final + + + javax.inject + javax.inject + 1 + + + javax.el + javax.el-api + 3.0.0 + + + org.glassfish + javax.el + 3.0.0 + + + org.jdbi + jdbi + 2.63.1 + + + com.sun.jersey + jersey-core + ${jersey.version} + + + com.sun.jersey.contribs + jersey-guice + ${jersey.version} + + + com.google.inject + guice + + + com.google.inject.extensions + guice-servlet + + + + + com.sun.jersey + jersey-server + ${jersey.version} + + + com.sun.jersey + jersey-json + ${jersey.version} + + + org.eclipse.jetty + jetty-server + ${jetty.version} + + + org.eclipse.jetty + jetty-servlet + ${jetty.version} + + + org.eclipse.jetty + jetty-servlets + ${jetty.version} + + + org.eclipse.jetty + jetty-proxy + ${jetty.version} + + + org.eclipse.jetty + jetty-util + ${jetty.version} + + + org.eclipse.jetty + jetty-security + ${jetty.version} + + + io.netty + netty + ${netty3.version} + + + io.netty + netty-all + ${netty4.version} + + + joda-time + joda-time + 2.9.9 + + + com.google.code.findbugs + jsr305 + 2.0.1 + + + org.apache.logging.log4j + log4j-api + ${log4j.version} + + + org.apache.logging.log4j + log4j-core + ${log4j.version} + + + org.apache.logging.log4j + log4j-slf4j-impl + ${log4j.version} + + + org.apache.logging.log4j + log4j-1.2-api + ${log4j.version} + + + + org.apache.logging.log4j + log4j-jul + ${log4j.version} + + + org.slf4j + jcl-over-slf4j + ${slf4j.version} + + + com.lmax + disruptor + 3.3.6 + + + net.spy + spymemcached + 2.12.3 + + + org.antlr + antlr4-runtime + 4.5.1 + + + org.antlr + antlr4-coordinator + 4.5.1 + + + commons-cli + commons-cli + 1.2 + + + org.apache.commons + commons-dbcp2 + 2.0.1 + + + org.lz4 + lz4-java + 1.5.0 + + + com.google.protobuf + protobuf-java + 3.1.0 + + + io.tesla.aether + tesla-aether + 0.0.5 + + + org.slf4j + slf4j-api + + + + + org.eclipse.aether + aether-api + 0.9.0.M2 + + + org.apache.httpcomponents + httpclient + 4.5.3 + + + org.apache.httpcomponents + httpcore + 4.4.4 + + + org.apache.hadoop + hadoop-client + ${hadoop.compile.version} + provided + + + org.apache.avro + avro + + + + + org.mapdb + mapdb + 1.0.8 + + + org.apache.derby + derbynet + 10.11.1.1 + + + org.apache.derby + derbyclient + 10.11.1.1 + + + org.apache.commons + commons-math3 + 3.6.1 + + + it.unimi.dsi + fastutil + 8.1.0 + + + com.opencsv + opencsv + 4.2 + + + com.jayway.jsonpath + json-path + 2.3.0 + + + net.thisptr + jackson-jq + 0.0.7 + + + org.slf4j + slf4j-api + 1.6.4 + + + org.roaringbitmap + RoaringBitmap + 0.5.18 + + + org.ow2.asm + asm + 5.2 + + + org.ow2.asm + asm-commons + 5.2 + + + org.asynchttpclient + async-http-client + + 2.5.3 + + + org.gridkit.lab + jvm-attach-api + 1.2 + + + net.java.dev.jna + jna + 4.5.1 + + + io.dropwizard.metrics + metrics-core + ${dropwizard.metrics.version} + + + org.codehaus.jackson + jackson-core-asl + 1.9.13 + + + org.codehaus.jackson + jackson-mapper-asl + 1.9.13 + + + javax.servlet + javax.servlet-api + 3.1.0 + + + javax.activation + activation + 1.1.1 + + + commons-pool + commons-pool + 1.6 + + + org.codehaus.plexus + plexus-utils + 3.0.15 + + + com.github.ben-manes.caffeine + caffeine + ${caffeine.version} + - - org.apache.hadoop.fs - - - - org.codehaus.mojo - cobertura-maven-plugin - 2.7 - - - - org.apache.druid.sql.antlr4.* - - - org/apache/druid/sql/antlr4/**/*.class - - - xml - - true - - - - org.apache.maven.plugins - maven-compiler-plugin - - ${maven.compiler.source} - ${maven.compiler.target} - - - - - maven-remote-resources-plugin - - - process-resource-bundles - none - - - - - - + + org.apache.calcite + calcite-core + ${calcite.version} + test-jar + test + + + org.easymock + easymock + 3.4 + test + + + junit + junit + 4.12 + test + + + org.powermock + powermock-module-junit4 + ${powermock.version} + test + + + org.powermock + powermock-api-easymock + ${powermock.version} + test + + + org.slf4j + slf4j-simple + ${slf4j.version} + + + com.carrotsearch + junit-benchmarks + 0.7.2 + test + + + com.google.caliper + caliper + 0.5-rc1 + test + + + org.apache.curator + curator-test + ${apache.curator.test.version} + + + org.jboss.netty + netty + + + test + + + com.ircclouds.irc + irc-api + 1.0-0014 + + + org.slf4j + slf4j-api + + + + + com.maxmind.geoip2 + geoip2 + 0.4.0 + + + com.google.http-client + google-http-client + + + com.google.http-client + google-http-client-jackson2 + + + com.fasterxml.jackson.core + jackson-databind + + + + + org.testng + testng + 6.8.7 + + + org.hamcrest + hamcrest-all + 1.3 + test + + + pl.pragmatists + JUnitParams + 1.0.4 + test + + + com.google.guava + guava-testlib + ${guava.version} + test + + + - - - strict - + - - org.apache.maven.plugins - maven-compiler-plugin - - javac-with-errorprone - true - true - 1024m - 3000m - ${maven.compiler.source} - ${maven.compiler.target} - false - - -XepDisableWarningsInGeneratedCode + + org.codehaus.mojo + cobertura-maven-plugin + + + org.eluder.coveralls + coveralls-maven-plugin + 4.0.0 + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.17 + + ${project.build.sourceDirectory} + true + codestyle/checkstyle.xml + codestyle/checkstyle-suppressions.xml + checkstyle.suppressions.file + UTF-8 + codestyle/LICENSE.txt + true + true + + + + com.puppycrawl.tools + checkstyle + 8.0 + + + + + validate + validate + + check + + + + + + org.apache.maven.plugins + maven-pmd-plugin + 3.8 + + true + + /rulesets/java/imports.xml + + + target/generated-sources/ + + + + + validate + validate + + check + + + + + + de.thetaphi + forbiddenapis + 2.3 + + false + + + jdk-unsafe + + + ${project.parent.basedir}/codestyle/joda-time-forbidden-apis.txt + ${project.parent.basedir}/codestyle/druid-forbidden-apis.txt + + + **.SuppressForbidden + + + + + compile + compile + + check + + + + + jdk-unsafe + jdk-system-out + + + + + testCompile + test-compile + + testCheck + + + + + + org.codehaus.mojo + animal-sniffer-maven-plugin + 1.15 + + + check-java-api + test + + check + + + + org.codehaus.mojo.signature + + java18 + 1.0 + + + + sun.nio.ch.DirectBuffer + sun.misc.Cleaner + sun.misc.Unsafe + + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + 1.4.1 + + + enforce-banned-dependencies + + enforce + + + + + 1.8.0 + + + + + com.google.code.findbugs:annotations + + + + true + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + ${java.version} (${java.vendor} ${java.vm.version}) + ${os.name} ${os.arch} ${os.version} + ${git.build.time} + ${git.build.version} + ${git.commit.id} + ${git.commit.id.describe} + + + + + + pl.project13.maven + git-commit-id-plugin + 2.2.5 + + + + revision + + + + + ${project.basedir}/.git + Etc/UTC + false + json + true + ${project.build.directory}/git.version + false + + git.build.user.email + git.build.host + git.commit.id.describe-short + git.commit.user.* + git.commit.message.* + git.closest.tag.* + git.commit.id.abbrev + git.dirty + + + false + true + 7 + -dirty + true + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.19.1 + + + + + -Xmx1500m + -XX:MaxDirectMemorySize=512m + -Duser.language=en + -Duser.GroupByQueryRunnerTest.javacountry=US + -Dfile.encoding=UTF-8 + -Duser.timezone=UTC + -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager + + -Ddruid.indexing.doubleStorage=double + + false + + true + + + + org.apache.maven.plugins + maven-release-plugin + + true + + + + org.apache.maven.plugins + maven-clean-plugin + 2.5 + + + org.apache.maven.plugins + maven-dependency-plugin + 2.8 + + + org.apache.maven.plugins + maven-deploy-plugin + 2.7 + + + org.apache.maven.plugins + maven-help-plugin + 2.1.1 + + + org.apache.maven.plugins + maven-install-plugin + 2.3.1 + + + package + + install + + + + + + org.apache.maven.plugins + maven-resources-plugin + 2.5 + + + org.apache.maven.plugins + maven-shade-plugin + 2.2 + + + org.apache.maven.plugins + maven-site-plugin + 3.1 + + + org.scala-tools + maven-scala-plugin + 2.15.2 + + + org.antlr + antlr4-maven-plugin + 4.5.1 + + + org.apache.maven.plugins + maven-assembly-plugin + 3.1.0 + + + org.codehaus.mojo + license-maven-plugin + 1.8 + + + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.10.1 + + + -Xdoclint:none - -Xep:ClassCanBeStatic:ERROR - -Xep:PreconditionsInvalidPlaceholder:ERROR - -Xep:MissingOverride:ERROR - -Xep:DefaultCharset:ERROR + + org.apache.hadoop.fs + + + + org.codehaus.mojo + cobertura-maven-plugin + 2.7 + + + + org.apache.druid.sql.antlr4.* + + + org/apache/druid/sql/antlr4/**/*.class + + + xml + + true + + + + org.apache.maven.plugins + maven-compiler-plugin + + ${maven.compiler.source} + ${maven.compiler.target} + + + + + maven-remote-resources-plugin + + + process-resource-bundles + none + + + + + + - -Xep:ArgumentParameterSwap - -Xep:AssistedInjectAndInjectOnSameConstructor - -Xep:AutoFactoryAtInject - -Xep:ClassName - -Xep:ComparisonContractViolated - -Xep:DepAnn - -Xep:DivZero - -Xep:EmptyIf - -Xep:InjectInvalidTargetingOnScopingAnnotation - -Xep:InjectMoreThanOneQualifier - -Xep:InjectScopeAnnotationOnInterfaceOrAbstractClass - -Xep:InjectScopeOrQualifierAnnotationRetention - -Xep:InjectedConstructorAnnotations - -Xep:InsecureCryptoUsage - -Xep:JMockTestWithoutRunWithOrRuleAnnotation - -Xep:JavaxInjectOnFinalField - -Xep:LockMethodChecker - -Xep:LongLiteralLowerCaseSuffix - -Xep:NoAllocation - -Xep:NonRuntimeAnnotation - -Xep:NumericEquality - -Xep:ParameterPackage - -Xep:ProtoStringFieldReferenceEquality - -Xep:QualifierOnMethodWithoutProvides - -Xep:UnlockMethod - - - - - org.codehaus.plexus - plexus-compiler-javac-errorprone - 2.8.1 - - - - com.google.errorprone - error_prone_core - 2.0.19 - - - - - - - - parallel-test - - false - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - test - - test - - - - - ${maven.fork.count} - true - false - - - -Xmx768m -Duser.language=en -Duser.country=US -Dfile.encoding=UTF-8 - -Duser.timezone=UTC -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager - - -Ddruid.indexing.doubleStorage=double - - - true - - - - - - - - rat - - false - - - - - org.apache.rat - apache-rat-plugin - 0.12 - - - verify - - check - - - - - ${project.basedir}/rat - - - MIT - MIT JQuery - - - Copyright 2012 jQuery Foundation and other contributors; Licensed MIT - jQuery Foundation, Inc. | jquery.org/license - - - - Underscore - Underscore - - - Underscore is freely distributable under the MIT license - - - - Allan Jardine - Allan Jardine - - - Copyright 2009 Allan Jardine. All Rights Reserved - - - - Allan Jardine - Allan Jardine - - - Copyright 2009 Allan Jardine. All Rights Reserved - Copyright 2008-2011 Allan Jardine - GPL v2 or BSD 3 point style - - - + + + strict + + + + org.apache.maven.plugins + maven-compiler-plugin + + javac-with-errorprone + true + true + 1024m + 3000m + ${maven.compiler.source} + ${maven.compiler.target} + false + + -XepDisableWarningsInGeneratedCode - - - MIT JQuery - - - Underscore - - - Allan Jardine - - - - - **/*.md - publications/** - docs/** - - codestyle/* - eclipse.importorder - - **/javax.annotation.processing.Processor - **/org.apache.druid.initialization.DruidModule - **/org/apache/druid/math/expr/antlr/Expr.g4 - **/dependency-reduced-pom.xml - - **/target/** - **/build/** - **/test/resources/** - **/src/test/avro/** - **/src/test/thrift/** - .travis.yml - - **/*.json - **/jvm.config - **/quickstart/protobuf/** - **/tutorial/conf/** - **/derby.log - **/docker/** - **/client_tls/** - - **/*.iml - - **/hs_err_pid*.log - - - - - - - - - apache-release - - - - maven-assembly-plugin - - - source-release-assembly - none - - - - - - - + -Xep:ClassCanBeStatic:ERROR + -Xep:PreconditionsInvalidPlaceholder:ERROR + -Xep:MissingOverride:ERROR + -Xep:DefaultCharset:ERROR + + -Xep:ArgumentParameterSwap + -Xep:AssistedInjectAndInjectOnSameConstructor + -Xep:AutoFactoryAtInject + -Xep:ClassName + -Xep:ComparisonContractViolated + -Xep:DepAnn + -Xep:DivZero + -Xep:EmptyIf + -Xep:InjectInvalidTargetingOnScopingAnnotation + -Xep:InjectMoreThanOneQualifier + -Xep:InjectScopeAnnotationOnInterfaceOrAbstractClass + -Xep:InjectScopeOrQualifierAnnotationRetention + -Xep:InjectedConstructorAnnotations + -Xep:InsecureCryptoUsage + -Xep:JMockTestWithoutRunWithOrRuleAnnotation + -Xep:JavaxInjectOnFinalField + -Xep:LockMethodChecker + -Xep:LongLiteralLowerCaseSuffix + -Xep:NoAllocation + -Xep:NonRuntimeAnnotation + -Xep:NumericEquality + -Xep:ParameterPackage + -Xep:ProtoStringFieldReferenceEquality + -Xep:QualifierOnMethodWithoutProvides + -Xep:UnlockMethod + + + + + org.codehaus.plexus + plexus-compiler-javac-errorprone + 2.8.1 + + + + com.google.errorprone + error_prone_core + 2.0.19 + + + + + + + + parallel-test + + false + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + test + + test + + + + + ${maven.fork.count} + true + false + + + -Xmx768m -Duser.language=en -Duser.country=US -Dfile.encoding=UTF-8 + -Duser.timezone=UTC -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager + + -Ddruid.indexing.doubleStorage=double + + + true + + + + + + + + rat + + false + + + + + org.apache.rat + apache-rat-plugin + 0.12 + + + verify + + check + + + + + ${project.basedir}/rat + + + MIT + MIT JQuery + + + Copyright 2012 jQuery Foundation and other contributors; Licensed MIT + jQuery Foundation, Inc. | jquery.org/license + + + + Underscore + Underscore + + + Underscore is freely distributable under the MIT license + + + + Allan Jardine + Allan Jardine + + + Copyright 2009 Allan Jardine. All Rights Reserved + + + + Allan Jardine + Allan Jardine + + + Copyright 2009 Allan Jardine. All Rights Reserved + Copyright 2008-2011 Allan Jardine + GPL v2 or BSD 3 point style + + + + + + + MIT JQuery + + + Underscore + + + Allan Jardine + + + + + **/*.md + publications/** + docs/** + + codestyle/* + eclipse.importorder + + **/javax.annotation.processing.Processor + **/org.apache.druid.initialization.DruidModule + **/org/apache/druid/math/expr/antlr/Expr.g4 + **/dependency-reduced-pom.xml + + **/target/** + **/build/** + **/test/resources/** + **/src/test/avro/** + **/src/test/thrift/** + .travis.yml + + **/*.json + **/jvm.config + **/quickstart/protobuf/** + **/tutorial/conf/** + **/derby.log + **/docker/** + **/client_tls/** + + **/*.iml + + **/hs_err_pid*.log + + + + + + + + + apache-release + + + + maven-assembly-plugin + + + source-release-assembly + none + + + + + + + From 1568b3f4d3eb865eb8aa9bcdedf03fb1d7007731 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 5 Nov 2018 14:54:23 -0800 Subject: [PATCH 38/87] improve RecordSupplier to supply records in batch --- distribution/pom.xml | 546 +++++++++--------- .../druid/indexing/kafka/KafkaIndexTask.java | 31 +- .../indexing/kafka/KafkaRecordSupplier.java | 45 +- .../kafka/KafkaRecordSupplierTest.java | 70 +-- .../KafkaSupervisorIOConfigTest.java | 4 +- .../indexing/kinesis/KinesisIndexTask.java | 206 +++---- .../kinesis/KinesisRecordSupplier.java | 35 +- .../kinesis/KinesisRecordSupplierTest.java | 51 +- .../seekablestream/common/RecordSupplier.java | 7 +- 9 files changed, 493 insertions(+), 502 deletions(-) diff --git a/distribution/pom.xml b/distribution/pom.xml index fcd191b65d48..995022284578 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -20,286 +20,286 @@ - 4.0.0 + 4.0.0 - pom + pom - distribution - distribution - distribution + distribution + distribution + distribution - - druid - org.apache.druid - 0.13.0-incubating-SNAPSHOT - + + druid + org.apache.druid + 0.13.0-incubating-SNAPSHOT + - - - org.apache.druid - druid-services - ${project.parent.version} - - - - - --clean - + + + org.apache.druid + druid-services + ${project.parent.version} + + + + + --clean + - - - - net.nicoulaj.maven.plugins - checksum-maven-plugin - 1.7 - - - dist-checksum - - files - - - - - - SHA-512 - - false - - - ${project.build.directory} - - *-src.tar.gz - *-bin.tar.gz - - - - false - - - - - - - - dist - - false - - tar - - - - - - org.codehaus.mojo - exec-maven-plugin - - - pull-deps - package - - exec - - - java - - -classpath - - -Ddruid.extensions.loadList=[] - -Ddruid.extensions.directory=${project.build.directory}/extensions - - - -Ddruid.extensions.hadoopDependenciesDir=${project.build.directory}/hadoop-dependencies - - org.apache.druid.cli.Main - tools - pull-deps - --clean - --defaultVersion - ${project.parent.version} - -l - ${settings.localRepository} - -h - org.apache.hadoop:hadoop-client:${hadoop.compile.version} - -c - org.apache.druid.extensions:druid-avro-extensions - -c - org.apache.druid.extensions:druid-bloom-filter - -c - org.apache.druid.extensions:druid-datasketches - -c - org.apache.druid.extensions:druid-hdfs-storage - -c - org.apache.druid.extensions:druid-histogram - -c - org.apache.druid.extensions:druid-kafka-eight - -c - org.apache.druid.extensions:druid-kafka-extraction-namespace - -c - org.apache.druid.extensions:druid-kafka-indexing-service - -c - org.apache.druid.extensions:druid-kinesis-indexing-service - -c - org.apache.druid.extensions:druid-lookups-cached-global - -c - org.apache.druid.extensions:druid-lookups-cached-single - -c - org.apache.druid.extensions:druid-protobuf-extensions - -c - org.apache.druid.extensions:mysql-metadata-storage - -c - org.apache.druid.extensions:postgresql-metadata-storage - -c - org.apache.druid.extensions:druid-kerberos - -c - org.apache.druid.extensions:druid-s3-extensions - -c - org.apache.druid.extensions:druid-stats - -c - org.apache.druid.extensions:druid-examples - -c - org.apache.druid.extensions:simple-client-sslcontext - -c - org.apache.druid.extensions:druid-basic-security - ${druid.distribution.pulldeps.opts} - - - - - - - org.apache.maven.plugins - maven-assembly-plugin - - - distro-assembly - package - - single - - - apache-druid-${project.parent.version} - posix - - src/assembly/assembly.xml - - - - - source-release-assembly-druid - package - - single - - - apache-druid-${project.version}-src - posix - - src/assembly/source-assembly.xml - - false - - - - - - org.codehaus.mojo - license-maven-plugin - - - download-licenses - - download-licenses - - - - - - - - - bundle-contrib-exts - + - - org.codehaus.mojo - exec-maven-plugin - - - pull-deps-contrib-exts - package - - exec - + + net.nicoulaj.maven.plugins + checksum-maven-plugin + 1.7 + + + dist-checksum + + files + + + - java - - -classpath - - -Ddruid.extensions.loadList=[] - -Ddruid.extensions.directory=${project.build.directory}/extensions - - - -Ddruid.extensions.hadoopDependenciesDir=${project.build.directory}/hadoop-dependencies - - org.apache.druid.cli.Main - tools - pull-deps - --defaultVersion - ${project.parent.version} - -l - ${settings.localRepository} - --no-default-hadoop - -c - org.apache.druid.extensions.contrib:ambari-metrics-emitter - -c - org.apache.druid.extensions.contrib:druid-azure-extensions - -c - org.apache.druid.extensions.contrib:druid-cassandra-storage - -c - org.apache.druid.extensions.contrib:druid-cloudfiles-extensions - -c - org.apache.druid.extensions.contrib:druid-distinctcount - -c - org.apache.druid.extensions.contrib:druid-rocketmq - -c - org.apache.druid.extensions.contrib:druid-google-extensions - -c - org.apache.druid.extensions.contrib:druid-kafka-eight-simple-consumer - -c - org.apache.druid.extensions.contrib:graphite-emitter - -c - org.apache.druid.extensions.contrib:druid-opentsdb-emitter - -c - org.apache.druid.extensions.contrib:druid-orc-extensions - -c - org.apache.druid.extensions.contrib:druid-parquet-extensions - -c - org.apache.druid.extensions.contrib:druid-rabbitmq - -c - org.apache.druid.extensions.contrib:druid-redis-cache - -c - org.apache.druid.extensions.contrib:sqlserver-metadata-storage - -c - org.apache.druid.extensions.contrib:statsd-emitter - -c - org.apache.druid.extensions.contrib:druid-thrift-extensions - -c - org.apache.druid.extensions.contrib:druid-time-min-max - -c - org.apache.druid.extensions.contrib:druid-virtual-columns - -c - org.apache.druid.extensions.contrib:materialized-view-maintenance - -c - org.apache.druid.extensions.contrib:materialized-view-selection - + + SHA-512 + + false + + + ${project.build.directory} + + *-src.tar.gz + *-bin.tar.gz + + + + false - - - + - - - + + + + + dist + + false + + tar + + + + + + org.codehaus.mojo + exec-maven-plugin + + + pull-deps + package + + exec + + + java + + -classpath + + -Ddruid.extensions.loadList=[] + -Ddruid.extensions.directory=${project.build.directory}/extensions + + + -Ddruid.extensions.hadoopDependenciesDir=${project.build.directory}/hadoop-dependencies + + org.apache.druid.cli.Main + tools + pull-deps + --clean + --defaultVersion + ${project.parent.version} + -l + ${settings.localRepository} + -h + org.apache.hadoop:hadoop-client:${hadoop.compile.version} + -c + org.apache.druid.extensions:druid-avro-extensions + -c + org.apache.druid.extensions:druid-bloom-filter + -c + org.apache.druid.extensions:druid-datasketches + -c + org.apache.druid.extensions:druid-hdfs-storage + -c + org.apache.druid.extensions:druid-histogram + -c + org.apache.druid.extensions:druid-kafka-eight + -c + org.apache.druid.extensions:druid-kafka-extraction-namespace + -c + org.apache.druid.extensions:druid-kafka-indexing-service + -c + org.apache.druid.extensions:druid-kinesis-indexing-service + -c + org.apache.druid.extensions:druid-lookups-cached-global + -c + org.apache.druid.extensions:druid-lookups-cached-single + -c + org.apache.druid.extensions:druid-protobuf-extensions + -c + org.apache.druid.extensions:mysql-metadata-storage + -c + org.apache.druid.extensions:postgresql-metadata-storage + -c + org.apache.druid.extensions:druid-kerberos + -c + org.apache.druid.extensions:druid-s3-extensions + -c + org.apache.druid.extensions:druid-stats + -c + org.apache.druid.extensions:druid-examples + -c + org.apache.druid.extensions:simple-client-sslcontext + -c + org.apache.druid.extensions:druid-basic-security + ${druid.distribution.pulldeps.opts} + + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + distro-assembly + package + + single + + + apache-druid-${project.parent.version} + posix + + src/assembly/assembly.xml + + + + + source-release-assembly-druid + package + + single + + + apache-druid-${project.version}-src + posix + + src/assembly/source-assembly.xml + + false + + + + + + org.codehaus.mojo + license-maven-plugin + + + download-licenses + + download-licenses + + + + + + + + + bundle-contrib-exts + + + + org.codehaus.mojo + exec-maven-plugin + + + pull-deps-contrib-exts + package + + exec + + + java + + -classpath + + -Ddruid.extensions.loadList=[] + -Ddruid.extensions.directory=${project.build.directory}/extensions + + + -Ddruid.extensions.hadoopDependenciesDir=${project.build.directory}/hadoop-dependencies + + org.apache.druid.cli.Main + tools + pull-deps + --defaultVersion + ${project.parent.version} + -l + ${settings.localRepository} + --no-default-hadoop + -c + org.apache.druid.extensions.contrib:ambari-metrics-emitter + -c + org.apache.druid.extensions.contrib:druid-azure-extensions + -c + org.apache.druid.extensions.contrib:druid-cassandra-storage + -c + org.apache.druid.extensions.contrib:druid-cloudfiles-extensions + -c + org.apache.druid.extensions.contrib:druid-distinctcount + -c + org.apache.druid.extensions.contrib:druid-rocketmq + -c + org.apache.druid.extensions.contrib:druid-google-extensions + -c + org.apache.druid.extensions.contrib:druid-kafka-eight-simple-consumer + -c + org.apache.druid.extensions.contrib:graphite-emitter + -c + org.apache.druid.extensions.contrib:druid-opentsdb-emitter + -c + org.apache.druid.extensions.contrib:druid-orc-extensions + -c + org.apache.druid.extensions.contrib:druid-parquet-extensions + -c + org.apache.druid.extensions.contrib:druid-rabbitmq + -c + org.apache.druid.extensions.contrib:druid-redis-cache + -c + org.apache.druid.extensions.contrib:sqlserver-metadata-storage + -c + org.apache.druid.extensions.contrib:statsd-emitter + -c + org.apache.druid.extensions.contrib:druid-thrift-extensions + -c + org.apache.druid.extensions.contrib:druid-time-min-max + -c + org.apache.druid.extensions.contrib:druid-virtual-columns + -c + org.apache.druid.extensions.contrib:materialized-view-maintenance + -c + org.apache.druid.extensions.contrib:materialized-view-selection + + + + + + + + + \ No newline at end of file diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 7c53340b1e5d..286469be4268 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -32,7 +32,6 @@ import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.TaskResource; -import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; @@ -40,7 +39,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.metadata.PasswordProvider; import org.apache.druid.query.NoopQueryRunner; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; @@ -215,7 +213,11 @@ KafkaConsumer newConsumer() final Properties props = new Properties(); - addConsumerPropertiesFromConfig(props, configMapper, ((KafkaIOConfig) ioConfig).getConsumerProperties()); + KafkaRecordSupplier.addConsumerPropertiesFromConfig( + props, + configMapper, + ((KafkaIOConfig) ioConfig).getConsumerProperties() + ); props.setProperty("enable.auto.commit", "false"); props.setProperty("auto.offset.reset", "none"); @@ -229,29 +231,6 @@ KafkaConsumer newConsumer() } } - public static void addConsumerPropertiesFromConfig( - Properties properties, - ObjectMapper configMapper, - Map consumerProperties - ) - { - // Extract passwords before SSL connection to Kafka - for (Map.Entry entry : consumerProperties.entrySet()) { - String propertyKey = entry.getKey(); - if (propertyKey.equals(KafkaSupervisorIOConfig.TRUST_STORE_PASSWORD_KEY) - || propertyKey.equals(KafkaSupervisorIOConfig.KEY_STORE_PASSWORD_KEY) - || propertyKey.equals(KafkaSupervisorIOConfig.KEY_PASSWORD_KEY)) { - PasswordProvider configPasswordProvider = configMapper.convertValue( - entry.getValue(), - PasswordProvider.class - ); - properties.setProperty(propertyKey, configPasswordProvider.getPassword()); - } else { - properties.setProperty(propertyKey, String.valueOf(entry.getValue())); - } - } - } - static void assignPartitions( final KafkaConsumer consumer, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 71e02623f435..79c253da4319 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -21,19 +21,22 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.metadata.PasswordProvider; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import javax.annotation.Nonnull; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -117,20 +120,20 @@ public Set> getAssignment() .collect(Collectors.toSet()); } + @Nonnull @Override - public OrderedPartitionableRecord poll(long timeout) + public List> poll(long timeout) { - ConsumerRecords polledRecords = consumer.poll(timeout); - if (!polledRecords.isEmpty()) { - ConsumerRecord record = polledRecords.iterator().next(); - return new OrderedPartitionableRecord<>( + List> polledRecords = new ArrayList<>(); + for (ConsumerRecord record : consumer.poll(timeout)) { + polledRecords.add(new OrderedPartitionableRecord<>( record.topic(), record.partition(), record.offset(), record.value() == null ? null : ImmutableList.of(record.value()) - ); + )); } - return null; + return polledRecords; } @Override @@ -179,15 +182,37 @@ public void close() consumer.close(); } + public static void addConsumerPropertiesFromConfig( + Properties properties, + ObjectMapper configMapper, + Map consumerProperties + ) + { + // Extract passwords before SSL connection to Kafka + for (Map.Entry entry : consumerProperties.entrySet()) { + String propertyKey = entry.getKey(); + if (propertyKey.equals(KafkaSupervisorIOConfig.TRUST_STORE_PASSWORD_KEY) + || propertyKey.equals(KafkaSupervisorIOConfig.KEY_STORE_PASSWORD_KEY) + || propertyKey.equals(KafkaSupervisorIOConfig.KEY_PASSWORD_KEY)) { + PasswordProvider configPasswordProvider = configMapper.convertValue( + entry.getValue(), + PasswordProvider.class + ); + properties.setProperty(propertyKey, configPasswordProvider.getPassword()); + } else { + properties.setProperty(propertyKey, String.valueOf(entry.getValue())); + } + } + } + private KafkaConsumer getKafkaConsumer() { final Properties props = new Properties(); props.setProperty("metadata.max.age.ms", "10000"); props.setProperty("group.id", StringUtils.format("kafka-supervisor-%s", getRandomId())); - props.setProperty("max.poll.records", "1"); - KafkaIndexTask.addConsumerPropertiesFromConfig(props, sortingMapper, consumerProperties); + addConsumerPropertiesFromConfig(props, sortingMapper, consumerProperties); props.setProperty("enable.auto.commit", "false"); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java index 0edf45928b5c..6d768f4af2a5 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java @@ -41,7 +41,6 @@ import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -53,6 +52,7 @@ public class KafkaRecordSupplierTest private static final Logger log = new Logger(KafkaRecordSupplierTest.class); private static String topic = "topic"; private static long poll_timeout_millis = 1000; + private static int pollRetry = 5; private static int topicPosFix = 0; private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); @@ -211,19 +211,18 @@ public void testPoll() throws InterruptedException, ExecutionException recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); - Set> supplierRecords = new HashSet<>(); - OrderedPartitionableRecord record = recordSupplier.poll(poll_timeout_millis); + Set> initialRecords = createOrderedPartitionableRecords(); - while (record != null) { - supplierRecords.add(record); - record = recordSupplier.poll(poll_timeout_millis); + List> polledRecords = recordSupplier.poll(poll_timeout_millis); + for (int i = 0; polledRecords.size() != initialRecords.size() && i < pollRetry; i++) { + polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); + Thread.sleep(200); } - Set> initialRecords = createOrderedPartitionableRecords(); Assert.assertEquals(partitions, recordSupplier.getAssignment()); - Assert.assertEquals(records.size(), supplierRecords.size()); - Assert.assertTrue(initialRecords.containsAll(supplierRecords)); + Assert.assertEquals(initialRecords.size(), polledRecords.size()); + Assert.assertTrue(initialRecords.containsAll(polledRecords)); recordSupplier.close(); } @@ -251,12 +250,10 @@ public void testPollAfterMoreDataAdded() throws InterruptedException, ExecutionE recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); - Set> supplierRecords = new HashSet<>(); - OrderedPartitionableRecord record = recordSupplier.poll(poll_timeout_millis); - - while (record != null) { - supplierRecords.add(record); - record = recordSupplier.poll(poll_timeout_millis); + List> polledRecords = recordSupplier.poll(poll_timeout_millis); + for (int i = 0; polledRecords.size() != 13 && i < pollRetry; i++) { + polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); + Thread.sleep(200); } // Insert data @@ -265,16 +262,15 @@ record = recordSupplier.poll(poll_timeout_millis); } - record = recordSupplier.poll(poll_timeout_millis); - while (record != null) { - supplierRecords.add(record); - record = recordSupplier.poll(poll_timeout_millis); + for (int i = 0; polledRecords.size() != records.size() && i < pollRetry; i++) { + polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); + Thread.sleep(200); } Set> initialRecords = createOrderedPartitionableRecords(); - Assert.assertEquals(records.size(), supplierRecords.size()); - Assert.assertTrue(initialRecords.containsAll(supplierRecords)); + Assert.assertEquals(records.size(), polledRecords.size()); + Assert.assertTrue(initialRecords.containsAll(polledRecords)); recordSupplier.close(); @@ -309,19 +305,17 @@ public void testSeek() throws InterruptedException, ExecutionException recordSupplier.seek(partition0, 2L); recordSupplier.seek(partition1, 2L); + Set> initialRecords = createOrderedPartitionableRecords(); - Set> supplierRecords = new HashSet<>(); - OrderedPartitionableRecord record = recordSupplier.poll(poll_timeout_millis); - - while (record != null) { - supplierRecords.add(record); - record = recordSupplier.poll(poll_timeout_millis); + List> polledRecords = recordSupplier.poll(poll_timeout_millis); + for (int i = 0; polledRecords.size() != 11 && i < pollRetry; i++) { + polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); + Thread.sleep(200); } - Set> initialRecords = createOrderedPartitionableRecords(); - Assert.assertEquals(11, supplierRecords.size()); - Assert.assertTrue(initialRecords.containsAll(supplierRecords)); + Assert.assertEquals(11, polledRecords.size()); + Assert.assertTrue(initialRecords.containsAll(polledRecords)); recordSupplier.close(); @@ -355,8 +349,9 @@ public void testSeekToLatest() throws InterruptedException, ExecutionException Assert.assertEquals(0L, (long) recordSupplier.getEarliestSequenceNumber(partition1)); recordSupplier.seekToLatest(partitions); - Assert.assertNull(recordSupplier.poll(poll_timeout_millis)); + List> polledRecords = recordSupplier.poll(poll_timeout_millis); + Assert.assertEquals(Collections.emptyList(), polledRecords); recordSupplier.close(); } @@ -421,15 +416,22 @@ public void testPosition() throws ExecutionException, InterruptedException Assert.assertEquals(4L, (long) recordSupplier.position(partition0)); Assert.assertEquals(5L, (long) recordSupplier.position(partition1)); - recordSupplier.poll(poll_timeout_millis); - Assert.assertTrue(recordSupplier.position(partition0) == 5L || recordSupplier.position(partition1) == 6L); - recordSupplier.seekToEarliest(Collections.singleton(partition0)); Assert.assertEquals(0L, (long) recordSupplier.position(partition0)); recordSupplier.seekToLatest(Collections.singleton(partition0)); Assert.assertEquals(11L, (long) recordSupplier.position(partition0)); + long prevPos = recordSupplier.position(partition0); + recordSupplier.getEarliestSequenceNumber(partition0); + Assert.assertEquals(prevPos, (long) recordSupplier.position(partition0)); + + recordSupplier.getLatestSequenceNumber(partition0); + Assert.assertEquals(prevPos, (long) recordSupplier.position(partition0)); + + recordSupplier.close(); } + + } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java index 243b8eb3335b..cbf25def6b4f 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java @@ -23,8 +23,8 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; -import org.apache.druid.indexing.kafka.KafkaIndexTask; import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.KafkaRecordSupplier; import org.apache.druid.jackson.DefaultObjectMapper; import org.hamcrest.CoreMatchers; import org.joda.time.Duration; @@ -136,7 +136,7 @@ public void testSerdeForConsumerPropertiesWithPasswords() throws Exception KafkaSupervisorIOConfig config = mapper.readValue(jsonStr, KafkaSupervisorIOConfig.class); Properties props = new Properties(); - KafkaIndexTask.addConsumerPropertiesFromConfig(props, mapper, config.getConsumerProperties()); + KafkaRecordSupplier.addConsumerPropertiesFromConfig(props, mapper, config.getConsumerProperties()); Assert.assertEquals("my-topic", config.getTopic()); Assert.assertEquals("localhost:9092", props.getProperty("bootstrap.servers")); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 87ea6c1a1613..21e9097719c5 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -380,133 +380,139 @@ public void run() break; } - OrderedPartitionableRecord record = recordSupplier.poll(POLL_TIMEOUT); + List> records = recordSupplier.poll(POLL_TIMEOUT); - if (record == null) { + if (records.size() == 0) { continue; } - // for the first message we receive, check that we were given a message with a sequenceNumber that matches our - // expected starting sequenceNumber - if (!verifiedAllStartingOffsets && contiguousOffsetCheck.containsKey(record.getPartitionId())) { - if (!contiguousOffsetCheck.get(record.getPartitionId()).equals(record.getSequenceNumber())) { - throw new ISE( - "Starting sequenceNumber [%s] does not match expected [%s] for partition [%s]", - record.getSequenceNumber(), - contiguousOffsetCheck.get(record.getPartitionId()), - record.getPartitionId() - ); - } + for (OrderedPartitionableRecord record : records) { + + // for the first message we receive, check that we were given a message with a sequenceNumber that matches our + // expected starting sequenceNumber + if (!verifiedAllStartingOffsets && contiguousOffsetCheck.containsKey(record.getPartitionId())) { + if (!contiguousOffsetCheck.get(record.getPartitionId()).equals(record.getSequenceNumber())) { + throw new ISE( + "Starting sequenceNumber [%s] does not match expected [%s] for partition [%s]", + record.getSequenceNumber(), + contiguousOffsetCheck.get(record.getPartitionId()), + record.getPartitionId() + ); + } - log.info( - "Verified starting sequenceNumber [%s] for partition [%s]", - record.getSequenceNumber(), record.getPartitionId() - ); + log.info( + "Verified starting sequenceNumber [%s] for partition [%s]", + record.getSequenceNumber(), record.getPartitionId() + ); - contiguousOffsetCheck.remove(record.getPartitionId()); - if (contiguousOffsetCheck.isEmpty()) { - verifiedAllStartingOffsets = true; - log.info("Verified starting offsets for all partitions"); - } + contiguousOffsetCheck.remove(record.getPartitionId()); + if (contiguousOffsetCheck.isEmpty()) { + verifiedAllStartingOffsets = true; + log.info("Verified starting offsets for all partitions"); + } - if (ioConfig.getExclusiveStartSequenceNumberPartitions() != null - && ioConfig.getExclusiveStartSequenceNumberPartitions().contains(record.getPartitionId())) { - log.info("Skipping starting sequenceNumber for partition [%s] marked exclusive", record.getPartitionId()); + if (ioConfig.getExclusiveStartSequenceNumberPartitions() != null + && ioConfig.getExclusiveStartSequenceNumberPartitions().contains(record.getPartitionId())) { + log.info( + "Skipping starting sequenceNumber for partition [%s] marked exclusive", + record.getPartitionId() + ); - continue; + continue; + } } - } - if (log.isTraceEnabled()) { - log.trace( - "Got topic[%s] partition[%s] offset[%s].", - record.getStream(), - record.getPartitionId(), - record.getSequenceNumber() - ); - } + if (log.isTraceEnabled()) { + log.trace( + "Got topic[%s] partition[%s] offset[%s].", + record.getStream(), + record.getPartitionId(), + record.getSequenceNumber() + ); + } - if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(record.getSequenceNumber())) { - lastOffsets.put(record.getPartitionId(), record.getSequenceNumber()); + if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(record.getSequenceNumber())) { + lastOffsets.put(record.getPartitionId(), record.getSequenceNumber()); - } else if (SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffsets.get(record.getPartitionId())) - || record.getSequenceNumber().compareTo(endOffsets.get(record.getPartitionId())) <= 0) { + } else if (SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffsets.get(record.getPartitionId())) + || record.getSequenceNumber().compareTo(endOffsets.get(record.getPartitionId())) <= 0) { - try { - final List valueBytess = record.getData(); + try { + final List valueBytess = record.getData(); - final List rows; - if (valueBytess == null || valueBytess.isEmpty()) { - rows = Utils.nullableListOf((InputRow) null); - } else { - rows = new ArrayList<>(); - for (byte[] valueBytes : valueBytess) { - rows.addAll(parser.parseBatch(ByteBuffer.wrap(valueBytes))); + final List rows; + if (valueBytess == null || valueBytess.isEmpty()) { + rows = Utils.nullableListOf((InputRow) null); + } else { + rows = new ArrayList<>(); + for (byte[] valueBytes : valueBytess) { + rows.addAll(parser.parseBatch(ByteBuffer.wrap(valueBytes))); + } } - } - boolean isPersistRequired = false; - final Map> segmentsToMoveOut = new HashMap<>(); - - for (final InputRow row : rows) { - if (row != null && withinMinMaxRecordTime(row)) { - final String sequenceName = sequenceNames.get(record.getPartitionId()); - final AppenderatorDriverAddResult addResult = driver.add( - row, - sequenceName, - committerSupplier, - false, - false - ); - - if (addResult.isOk()) { - // If the number of rows in the segment exceeds the threshold after adding a row, - // move the segment out from the active segments of BaseAppenderatorDriver to make a new segment. - if (addResult.getNumRowsInSegment() > tuningConfig.getMaxRowsPerSegment()) { - segmentsToMoveOut.computeIfAbsent(sequenceName, k -> new HashSet<>()) - .add(addResult.getSegmentIdentifier()); + boolean isPersistRequired = false; + final Map> segmentsToMoveOut = new HashMap<>(); + + for (final InputRow row : rows) { + if (row != null && withinMinMaxRecordTime(row)) { + final String sequenceName = sequenceNames.get(record.getPartitionId()); + final AppenderatorDriverAddResult addResult = driver.add( + row, + sequenceName, + committerSupplier, + false, + false + ); + + if (addResult.isOk()) { + // If the number of rows in the segment exceeds the threshold after adding a row, + // move the segment out from the active segments of BaseAppenderatorDriver to make a new segment. + if (addResult.getNumRowsInSegment() > tuningConfig.getMaxRowsPerSegment()) { + segmentsToMoveOut.computeIfAbsent(sequenceName, k -> new HashSet<>()) + .add(addResult.getSegmentIdentifier()); + } + isPersistRequired |= addResult.isPersistRequired(); + } else { + // Failure to allocate segment puts determinism at risk, bail out to be safe. + // May want configurable behavior here at some point. + // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks. + throw new ISE("Could not allocate segment for row with timestamp[%s]", row.getTimestamp()); } - isPersistRequired |= addResult.isPersistRequired(); - } else { - // Failure to allocate segment puts determinism at risk, bail out to be safe. - // May want configurable behavior here at some point. - // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks. - throw new ISE("Could not allocate segment for row with timestamp[%s]", row.getTimestamp()); - } - if (addResult.getParseException() != null) { - handleParseException(addResult.getParseException(), record); + if (addResult.getParseException() != null) { + handleParseException(addResult.getParseException(), record); + } else { + rowIngestionMeters.incrementProcessed(); + } } else { - rowIngestionMeters.incrementProcessed(); + rowIngestionMeters.incrementThrownAway(); } - } else { - rowIngestionMeters.incrementThrownAway(); } - } - if (isPersistRequired) { - driver.persist(committerSupplier.get()); + if (isPersistRequired) { + driver.persist(committerSupplier.get()); + } + segmentsToMoveOut.forEach((key, value) -> driver.moveSegmentOut( + key, + new ArrayList(value) + )); + } + catch (ParseException e) { + handleParseException(e, record); } - segmentsToMoveOut.forEach((key, value) -> driver.moveSegmentOut( - key, - new ArrayList(value) - )); - } - catch (ParseException e) { - handleParseException(e, record); - } - lastOffsets.put(record.getPartitionId(), record.getSequenceNumber()); + lastOffsets.put(record.getPartitionId(), record.getSequenceNumber()); - } - if ((lastOffsets.get(record.getPartitionId()).equals(endOffsets.get(record.getPartitionId())) - || OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(lastOffsets.get(record.getPartitionId()))) - && assignment.remove(record.getPartitionId())) { + } + if ((lastOffsets.get(record.getPartitionId()).equals(endOffsets.get(record.getPartitionId())) + || OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(lastOffsets.get(record.getPartitionId()))) + && assignment.remove(record.getPartitionId())) { - log.info("Finished reading stream[%s], partition[%s].", record.getStream(), record.getPartitionId()); - assignPartitions(recordSupplier, topic, assignment); - stillReading = !assignment.isEmpty(); + log.info("Finished reading stream[%s], partition[%s].", record.getStream(), record.getPartitionId()); + assignPartitions(recordSupplier, topic, assignment); + stillReading = !assignment.isEmpty(); + } } } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 35d7136b8b48..e29e20db3f28 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -35,6 +35,7 @@ import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder; import com.amazonaws.util.AwsHostNameUtils; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Queues; import org.apache.druid.common.aws.AWSCredentialsUtils; import org.apache.druid.indexing.kinesis.aws.ConstructibleAWSCredentialsConfig; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; @@ -45,8 +46,9 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; -import javax.annotation.Nullable; +import javax.annotation.Nonnull; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Iterator; @@ -69,6 +71,7 @@ public class KinesisRecordSupplier implements RecordSupplier private static final long PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS = 3000; private static final long EXCEPTION_RETRY_DELAY_MS = 10000; private final String endpoint; + private static final int DEFAULT_POLL_RECORD_LIMIT = 20; private class PartitionResource { @@ -372,9 +375,9 @@ public Collection> getAssignment() return partitionResources.keySet(); } - @Nullable + @Nonnull @Override - public OrderedPartitionableRecord poll(long timeout) + public List> poll(long timeout) { checkIfClosed(); if (checkPartitionsStarted) { @@ -383,23 +386,21 @@ public OrderedPartitionableRecord poll(long timeout) } try { - while (true) { - OrderedPartitionableRecord record = records.poll(timeout, TimeUnit.MILLISECONDS); - if (record == null || partitionResources.containsKey(record.getStreamPartition())) { - return record; - } else if (log.isTraceEnabled()) { - log.trace( - "Skipping stream[%s] / partition[%s] / sequenceNum[%s] because it is not in current assignment", - record.getStream(), - record.getPartitionId(), - record.getSequenceNumber() - ); - } - } + List> polledRecords = new ArrayList<>(); + Queues.drain( + records, + polledRecords, + Math.max(records.size(), DEFAULT_POLL_RECORD_LIMIT), + timeout, + TimeUnit.MILLISECONDS + ); + return polledRecords.stream() + .filter(x -> partitionResources.containsKey(x.getStreamPartition())) + .collect(Collectors.toList()); } catch (InterruptedException e) { log.warn(e, "InterruptedException"); - return null; + return Collections.emptyList(); } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index 3a39396b4099..d4e2115bad32 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -46,7 +46,6 @@ import java.nio.ByteBuffer; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.TimeoutException; @@ -62,7 +61,7 @@ public class KinesisRecordSupplierTest private static final Logger log = new Logger(KinesisRecordSupplierTest.class); private static String stream = "streamm"; - private static long poll_timeout_millis = 1000; + private static long poll_timeout_millis = 2000; private static String shardId1 = "shardId-000000000001"; private static String shardId0 = "shardId-000000000000"; private static int streamPosFix = 0; @@ -214,7 +213,7 @@ public void testSupplierSetup() throws InterruptedException Assert.assertEquals(partitions, recordSupplier.getAssignment()); Assert.assertEquals(ImmutableSet.of(shardId1, shardId0), recordSupplier.getPartitionIds(stream)); - Assert.assertNull(recordSupplier.poll(100)); + Assert.assertEquals(Collections.emptyList(), recordSupplier.poll(100)); recordSupplier.close(); } @@ -256,17 +255,11 @@ public void testPoll() throws InterruptedException recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); - Set> supplierRecords = new HashSet<>(); - OrderedPartitionableRecord record = recordSupplier.poll(poll_timeout_millis); - - while (record != null) { - supplierRecords.add(record); - record = recordSupplier.poll(poll_timeout_millis); - } + List> polledRecords = recordSupplier.poll(poll_timeout_millis); Assert.assertEquals(partitions, recordSupplier.getAssignment()); - Assert.assertEquals(initialRecords.size(), supplierRecords.size()); - Assert.assertTrue(supplierRecords.containsAll(initialRecords)); + Assert.assertEquals(initialRecords.size(), polledRecords.size()); + Assert.assertTrue(polledRecords.containsAll(initialRecords)); recordSupplier.close(); } @@ -309,13 +302,7 @@ public void testPollAfterMoreDataAdded() throws InterruptedException recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); - Set> supplierRecords = new HashSet<>(); - OrderedPartitionableRecord record = recordSupplier.poll(poll_timeout_millis); - - while (record != null) { - supplierRecords.add(record); - record = recordSupplier.poll(poll_timeout_millis); - } + List> polledRecords = recordSupplier.poll(poll_timeout_millis); List insertDataResults2 = insertData(kinesis, generateRecordsRequests(stream, 5, 12)); insertDataResults2.forEach(entry -> initialRecords.add(new OrderedPartitionableRecord<>( @@ -325,14 +312,10 @@ record = recordSupplier.poll(poll_timeout_millis); null ))); - record = recordSupplier.poll(poll_timeout_millis); - while (record != null) { - supplierRecords.add(record); - record = recordSupplier.poll(poll_timeout_millis); - } + polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); - Assert.assertEquals(initialRecords.size(), supplierRecords.size()); - Assert.assertTrue(supplierRecords.containsAll(initialRecords)); + Assert.assertEquals(initialRecords.size(), polledRecords.size()); + Assert.assertTrue(polledRecords.containsAll(initialRecords)); recordSupplier.close(); } @@ -392,17 +375,11 @@ public void testSeek() throws InterruptedException, TimeoutException )) .collect(Collectors.toSet()); - Set> supplierRecords = new HashSet<>(); - OrderedPartitionableRecord record = recordSupplier.poll(poll_timeout_millis); - - while (record != null) { - supplierRecords.add(record); - record = recordSupplier.poll(poll_timeout_millis); - } + List> polledRecords = recordSupplier.poll(poll_timeout_millis); - Assert.assertEquals(8, supplierRecords.size()); - Assert.assertTrue(supplierRecords.containsAll(initialRecords1)); - Assert.assertTrue(supplierRecords.containsAll(initialRecords2)); + Assert.assertEquals(8, polledRecords.size()); + Assert.assertTrue(polledRecords.containsAll(initialRecords1)); + Assert.assertTrue(polledRecords.containsAll(initialRecords2)); recordSupplier.close(); @@ -443,7 +420,7 @@ public void testSeekToLatest() throws InterruptedException, TimeoutException Assert.assertEquals(insertDataResults.get(8).getSequenceNumber(), recordSupplier.getEarliestSequenceNumber(shard0)); recordSupplier.seekToLatest(partitions); - Assert.assertNull(recordSupplier.poll(poll_timeout_millis)); + Assert.assertEquals(Collections.emptyList(), recordSupplier.poll(poll_timeout_millis)); recordSupplier.close(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java index 04055c3e068c..ad25d3c128b1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java @@ -21,9 +21,10 @@ import com.google.common.annotations.Beta; -import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; import java.io.Closeable; import java.util.Collection; +import java.util.List; import java.util.Set; import java.util.concurrent.TimeoutException; @@ -89,8 +90,8 @@ public interface RecordSupplier extends Closeable * * @return record */ - @Nullable - OrderedPartitionableRecord poll(long timeout); + @NotNull + List> poll(long timeout); /** * get the latest sequence number in stream From 5a1ef1c3b06f5ba636b4a76eb4ec8d78e2bd93ad Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 5 Nov 2018 17:34:07 -0800 Subject: [PATCH 39/87] fix strict compile issue --- .../indexing/seekablestream/SeekableStreamPartitions.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index 464f100ca80a..4bbb27340188 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -83,7 +83,7 @@ public String getStream() @DoNotCall @JsonProperty - public String getTopic() + public final String getTopic() { return name; } @@ -96,7 +96,7 @@ public Map getPartitionSequenceNumberMap() @DoNotCall @JsonProperty - public Map getPartitionOffsetMap() + public final Map getPartitionOffsetMap() { return map; } From 04047622b5b9e870bdcc40973a6f45ba60d4b18c Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 8 Nov 2018 16:22:49 -0800 Subject: [PATCH 40/87] add test scope for localstack dependency --- .../kinesis-indexing-service/pom.xml | 179 +++++++++--------- 1 file changed, 90 insertions(+), 89 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index 9711f9d7d0d3..2f7d7d0ddf96 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -20,101 +20,102 @@ - 4.0.0 + 4.0.0 - org.apache.druid.extensions - druid-kinesis-indexing-service - druid-kinesis-indexing-service - druid-kinesis-indexing-service + org.apache.druid.extensions + druid-kinesis-indexing-service + druid-kinesis-indexing-service + druid-kinesis-indexing-service - - org.apache.druid - druid - 0.13.0-incubating-SNAPSHOT - ../../pom.xml - + + org.apache.druid + druid + 0.13.0-incubating-SNAPSHOT + ../../pom.xml + - - - org.apache.druid - druid-core - ${project.parent.version} - provided - - - org.apache.druid - druid-indexing-service - ${project.parent.version} - provided - - - org.apache.druid - druid-server - ${project.parent.version} - provided - - - io.netty - netty - 3.10.4.Final - provided - + + + org.apache.druid + druid-core + ${project.parent.version} + provided + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + provided + + + org.apache.druid + druid-server + ${project.parent.version} + provided + + + io.netty + netty + 3.10.4.Final + provided + - - com.fasterxml.jackson.dataformat - jackson-dataformat-cbor - ${jackson.version} - + + com.fasterxml.jackson.dataformat + jackson-dataformat-cbor + ${jackson.version} + - - com.amazonaws - aws-java-sdk-kinesis - ${aws.sdk.version} - + + com.amazonaws + aws-java-sdk-kinesis + ${aws.sdk.version} + - - com.amazonaws - aws-java-sdk-sts - ${aws.sdk.version} - + + com.amazonaws + aws-java-sdk-sts + ${aws.sdk.version} + - - - org.easymock - easymock - - - junit - junit - 4.12 - test - - - org.apache.druid - druid-server - ${project.parent.version} - test-jar - test - - - org.apache.druid - druid-processing - ${project.parent.version} - test-jar - test - - - org.apache.druid - druid-indexing-service - ${project.parent.version} - test-jar - test - - - cloud.localstack - localstack-utils - 0.1.13 - - + + + org.easymock + easymock + + + junit + junit + 4.12 + test + + + org.apache.druid + druid-server + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-processing + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + test-jar + test + + + cloud.localstack + localstack-utils + 0.1.13 + test + + From c37ba2424c9910ac2d87497e4c1ca4d6ae624a4e Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 13 Nov 2018 18:14:22 -0800 Subject: [PATCH 41/87] kinesis indexing task refactoring --- ...ementalPublishingKafkaIndexTaskRunner.java | 1771 +-------------- .../druid/indexing/kafka/KafkaIOConfig.java | 15 +- .../druid/indexing/kafka/KafkaIndexTask.java | 189 +- .../indexing/kafka/KafkaRecordSupplier.java | 16 +- .../kafka/LegacyKafkaIndexTaskRunner.java | 91 +- .../kafka/supervisor/KafkaSupervisor.java | 48 +- .../supervisor/KafkaSupervisorIOConfig.java | 2 +- .../kafka/KafkaIndexTaskClientTest.java | 194 +- .../indexing/kafka/KafkaIndexTaskTest.java | 30 +- .../kafka/KafkaRecordSupplierTest.java | 18 +- .../kafka/supervisor/KafkaSupervisorTest.java | 13 +- .../indexing/kinesis/KinesisIOConfig.java | 8 +- .../indexing/kinesis/KinesisIndexTask.java | 1274 +---------- .../kinesis/KinesisIndexTaskRunner.java | 154 ++ .../kinesis/KinesisRecordSupplier.java | 194 +- .../kinesis/KinesisSequenceNumber.java | 27 +- .../indexing/kinesis/KinesisTuningConfig.java | 31 +- .../kinesis/supervisor/KinesisSupervisor.java | 34 +- .../supervisor/KinesisSupervisorIOConfig.java | 7 - .../supervisor/KinesisSupervisorSpec.java | 3 + .../KinesisSupervisorTuningConfig.java | 10 +- .../KinesisDataSourceMetadataTest.java | 4 +- .../indexing/kinesis/KinesisIOConfigTest.java | 2 + .../kinesis/KinesisIndexTaskClientTest.java | 14 +- .../kinesis/KinesisIndexTaskTest.java | 972 +++++++-- .../kinesis/KinesisRecordSupplierTest.java | 254 ++- .../kinesis/KinesisTuningConfigTest.java | 8 +- .../supervisor/KinesisSupervisorTest.java | 503 ++++- .../SeekableStreamDataSourceMetadata.java | 4 +- .../SeekableStreamIOConfig.java | 13 +- .../SeekableStreamIndexTask.java | 154 +- .../SeekableStreamIndexTaskClient.java | 10 +- .../SeekableStreamIndexTaskRunner.java | 1912 ++++++++++++++++- .../SeekableStreamPartitions.java | 25 +- .../common/OrderedPartitionableRecord.java | 2 +- .../seekablestream/common/RecordSupplier.java | 30 +- .../supervisor/SeekableStreamSupervisor.java | 436 ++-- .../SeekableStreamSupervisorIOConfig.java | 10 +- 38 files changed, 4593 insertions(+), 3889 deletions(-) create mode 100644 extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 7fc159756063..fd55d653eefb 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -18,188 +18,50 @@ */ package org.apache.druid.indexing.kafka; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.base.Function; -import com.google.common.base.Joiner; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; -import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Sets; -import com.google.common.primitives.Longs; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.SettableFuture; -import org.apache.druid.data.input.Committer; -import org.apache.druid.data.input.InputRow; +import com.google.common.collect.ImmutableSet; import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.discovery.DiscoveryDruidNode; -import org.apache.druid.discovery.LookupNodeService; -import org.apache.druid.discovery.NodeType; -import org.apache.druid.indexer.IngestionState; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; -import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; -import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; -import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.CheckPointDataSourceMetadataAction; -import org.apache.druid.indexing.common.actions.ResetDataSourceMetadataAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.IndexTaskUtils; -import org.apache.druid.indexing.common.task.RealtimeIndexTask; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; -import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.collect.Utils; -import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.appenderator.Appenderator; -import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; -import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata; -import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; -import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.server.security.Access; -import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizerMapper; -import org.apache.druid.timeline.DataSegment; import org.apache.druid.utils.CircularBuffer; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetOutOfRangeException; import org.apache.kafka.common.TopicPartition; -import org.joda.time.DateTime; +import javax.annotation.Nonnull; import javax.annotation.Nullable; -import javax.annotation.ParametersAreNonnullByDefault; -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.Consumes; -import javax.ws.rs.DefaultValue; -import javax.ws.rs.GET; -import javax.ws.rs.POST; -import javax.ws.rs.Path; -import javax.ws.rs.Produces; -import javax.ws.rs.QueryParam; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import java.io.File; +import javax.validation.constraints.NotNull; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; import java.util.TreeMap; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; /** * Kafka indexing task runner supporting incremental segments publishing */ -public class IncrementalPublishingKafkaIndexTaskRunner implements SeekableStreamIndexTaskRunner +public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner { private static final EmittingLogger log = new EmittingLogger(IncrementalPublishingKafkaIndexTaskRunner.class); - private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; - private static final String METADATA_PUBLISH_PARTITIONS = "publishPartitions"; - - private final Map endOffsets; - private final Map nextOffsets = new ConcurrentHashMap<>(); - private final Map lastPersistedOffsets = new ConcurrentHashMap<>(); - - // The pause lock and associated conditions are to support coordination between the Jetty threads and the main - // ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully - // the ingestion loop has been stopped at the returned offsets and will not ingest any more data until resumed. The - // fields are used as follows (every step requires acquiring [pauseLock]): - // Pausing: - // - In pause(), [pauseRequested] is set to true and then execution waits for [status] to change to PAUSED, with the - // condition checked when [hasPaused] is signalled. - // - In possiblyPause() called from the main loop, if [pauseRequested] is true, [status] is set to PAUSED, - // [hasPaused] is signalled, and execution pauses until [pauseRequested] becomes false, either by being set or by - // the [pauseMillis] timeout elapsing. [pauseRequested] is checked when [shouldResume] is signalled. - // Resuming: - // - In resume(), [pauseRequested] is set to false, [shouldResume] is signalled, and execution waits for [status] to - // change to something other than PAUSED, with the condition checked when [shouldResume] is signalled. - // - In possiblyPause(), when [shouldResume] is signalled, if [pauseRequested] has become false the pause loop ends, - // [status] is changed to STARTING and [shouldResume] is signalled. - - private final Lock pauseLock = new ReentrantLock(); - private final Condition hasPaused = pauseLock.newCondition(); - private final Condition shouldResume = pauseLock.newCondition(); - - private final AtomicBoolean stopRequested = new AtomicBoolean(false); - private final AtomicBoolean publishOnStop = new AtomicBoolean(false); - - // [statusLock] is used to synchronize the Jetty thread calling stopGracefully() with the main run thread. It prevents - // the main run thread from switching into a publishing state while the stopGracefully() thread thinks it's still in - // a pre-publishing state. This is important because stopGracefully() will try to use the [stopRequested] flag to stop - // the main thread where possible, but this flag is not honored once publishing has begun so in this case we must - // interrupt the thread. The lock ensures that if the run thread is about to transition into publishing state, it - // blocks until after stopGracefully() has set [stopRequested] and then does a final check on [stopRequested] before - // transitioning to publishing state. - private final Object statusLock = new Object(); - - private final Lock pollRetryLock = new ReentrantLock(); - private final Condition isAwaitingRetry = pollRetryLock.newCondition(); - - private final KafkaIndexTask task; - private final KafkaIOConfig ioConfig; private final KafkaTuningConfig tuningConfig; - private final InputRowParser parser; - private final AuthorizerMapper authorizerMapper; - private final Optional chatHandlerProvider; - private final CircularBuffer savedParseExceptions; - private final String topic; - private final RowIngestionMeters rowIngestionMeters; - - private final Set publishingSequences = Sets.newConcurrentHashSet(); - private final List> publishWaitList = new ArrayList<>(); - private final List> handOffWaitList = new ArrayList<>(); - - private volatile DateTime startTime; - private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread) - private volatile TaskToolbox toolbox; - private volatile Thread runThread; - private volatile Appenderator appenderator; - private volatile StreamAppenderatorDriver driver; - private volatile FireDepartmentMetrics fireDepartmentMetrics; - private volatile IngestionState ingestionState; - - private volatile boolean pauseRequested = false; - private volatile long nextCheckpointTime; - - private volatile CopyOnWriteArrayList sequences; - private volatile Throwable backgroundThreadException; + private final KafkaIndexTask task; public IncrementalPublishingKafkaIndexTaskRunner( KafkaIndexTask task, @@ -210,851 +72,66 @@ public IncrementalPublishingKafkaIndexTaskRunner( RowIngestionMetersFactory rowIngestionMetersFactory ) { + super( + task, + parser, + authorizerMapper, + chatHandlerProvider, + savedParseExceptions, + rowIngestionMetersFactory, + true + ); this.task = task; - this.ioConfig = task.getIOConfig(); this.tuningConfig = task.getTuningConfig(); - this.parser = parser; - this.authorizerMapper = authorizerMapper; - this.chatHandlerProvider = chatHandlerProvider; - this.savedParseExceptions = savedParseExceptions; - this.topic = ioConfig.getStartPartitions().getStream(); - this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters(); - - this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndPartitions().getPartitionSequenceNumberMap()); - this.sequences = new CopyOnWriteArrayList<>(); - this.ingestionState = IngestionState.NOT_STARTED; - - resetNextCheckpointTime(); } @Override - public TaskStatus run(TaskToolbox toolbox) - { - try { - return runInternal(toolbox); - } - catch (Exception e) { - log.error(e, "Encountered exception while running task."); - final String errorMsg = Throwables.getStackTraceAsString(e); - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(errorMsg)); - return TaskStatus.failure( - task.getId(), - errorMsg - ); - } - } - - private TaskStatus runInternal(TaskToolbox toolbox) throws Exception - { - log.info("Starting up!"); - - startTime = DateTimes.nowUtc(); - status = Status.STARTING; - this.toolbox = toolbox; - - if (!restoreSequences()) { - final TreeMap> checkpoints = getCheckPointsFromContext(toolbox, task); - if (checkpoints != null) { - Iterator>> sequenceOffsets = checkpoints.entrySet().iterator(); - Map.Entry> previous = sequenceOffsets.next(); - while (sequenceOffsets.hasNext()) { - Map.Entry> current = sequenceOffsets.next(); - sequences.add(new SequenceMetadata( - previous.getKey(), - StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), - previous.getValue(), - current.getValue(), - true - )); - previous = current; - } - sequences.add(new SequenceMetadata( - previous.getKey(), - StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), - previous.getValue(), - endOffsets, - false - )); - } else { - sequences.add(new SequenceMetadata( - 0, - StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0), - ioConfig.getStartPartitions().getPartitionSequenceNumberMap(), - endOffsets, - false - )); - } - } - log.info("Starting with sequences: %s", sequences); - - if (chatHandlerProvider.isPresent()) { - log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); - chatHandlerProvider.get().register(task.getId(), this, false); - } else { - log.warn("No chat handler detected"); - } - - runThread = Thread.currentThread(); - - // Set up FireDepartmentMetrics - final FireDepartment fireDepartmentForMetrics = new FireDepartment( - task.getDataSchema(), - new RealtimeIOConfig(null, null, null), - null - ); - fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); - toolbox.getMonitorScheduler() - .addMonitor(TaskRealtimeMetricsMonitorBuilder.build(task, fireDepartmentForMetrics, rowIngestionMeters)); - - final String lookupTier = task.getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER); - LookupNodeService lookupNodeService = lookupTier == null ? - toolbox.getLookupNodeService() : - new LookupNodeService(lookupTier); - DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( - toolbox.getDruidNode(), - NodeType.PEON, - ImmutableMap.of( - toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), - lookupNodeService.getName(), lookupNodeService - ) - ); - - Throwable caughtExceptionOuter = null; - try (final KafkaConsumer consumer = task.newConsumer()) { - toolbox.getDataSegmentServerAnnouncer().announce(); - toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); - - appenderator = task.newAppenderator(fireDepartmentMetrics, toolbox); - driver = task.newDriver(appenderator, toolbox, fireDepartmentMetrics); - - final String topic = ioConfig.getStartPartitions().getStream(); - - // Start up, set up initial offsets. - final Object restoredMetadata = driver.startJob(); - if (restoredMetadata == null) { - // no persist has happened so far - // so either this is a brand new task or replacement of a failed task - Preconditions.checkState(sequences.get(0).startOffsets.entrySet().stream().allMatch( - partitionOffsetEntry -> Longs.compare( - partitionOffsetEntry.getValue(), - ioConfig.getStartPartitions() - .getPartitionSequenceNumberMap() - .get(partitionOffsetEntry.getKey()) - ) >= 0 - ), "Sequence offsets are not compatible with start offsets of task"); - nextOffsets.putAll(sequences.get(0).startOffsets); - } else { - @SuppressWarnings("unchecked") - final Map restoredMetadataMap = (Map) restoredMetadata; - final SeekableStreamPartitions restoredNextPartitions = toolbox - .getObjectMapper() - .convertValue( - restoredMetadataMap.get(METADATA_NEXT_PARTITIONS), - toolbox.getObjectMapper() - .getTypeFactory() - .constructParametrizedType( - SeekableStreamPartitions.class, - SeekableStreamPartitions.class, - Integer.class, - Long.class - ) - ); - - nextOffsets.putAll(restoredNextPartitions.getPartitionSequenceNumberMap()); - - // Sanity checks. - if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) { - throw new ISE( - "WTF?! Restored topic[%s] but expected topic[%s]", - restoredNextPartitions.getStream(), - ioConfig.getStartPartitions().getStream() - ); - } - - if (!nextOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet())) { - throw new ISE( - "WTF?! Restored partitions[%s] but expected partitions[%s]", - nextOffsets.keySet(), - ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet() - ); - } - // sequences size can be 0 only when all sequences got published and task stopped before it could finish - // which is super rare - if (sequences.size() == 0 || sequences.get(sequences.size() - 1).isCheckpointed()) { - this.endOffsets.putAll(sequences.size() == 0 - ? nextOffsets - : sequences.get(sequences.size() - 1).getEndOffsets()); - log.info("End offsets changed to [%s]", endOffsets); - } - } - - // Set up committer. - final Supplier committerSupplier = () -> { - final Map snapshot = ImmutableMap.copyOf(nextOffsets); - lastPersistedOffsets.clear(); - lastPersistedOffsets.putAll(snapshot); - - return new Committer() - { - @Override - public Object getMetadata() - { - return ImmutableMap.of( - METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>( - ioConfig.getStartPartitions().getStream(), - snapshot - ) - ); - } - - @Override - public void run() - { - // Do nothing. - } - }; - }; - - // restart publishing of sequences (if any) - maybePersistAndPublishSequences(committerSupplier); - - Set assignment = assignPartitionsAndSeekToNext(consumer, topic); - - ingestionState = IngestionState.BUILD_SEGMENTS; - - // Main loop. - // Could eventually support leader/follower mode (for keeping replicas more in sync) - boolean stillReading = !assignment.isEmpty(); - status = Status.READING; - Throwable caughtExceptionInner = null; - try { - while (stillReading) { - if (possiblyPause()) { - // The partition assignments may have changed while paused by a call to setEndOffsets() so reassign - // partitions upon resuming. This is safe even if the end offsets have not been modified. - assignment = assignPartitionsAndSeekToNext(consumer, topic); - - if (assignment.isEmpty()) { - log.info("All partitions have been fully read"); - publishOnStop.set(true); - stopRequested.set(true); - } - } - - // if stop is requested or task's end offset is set by call to setEndOffsets method with finish set to true - if (stopRequested.get() || sequences.get(sequences.size() - 1).isCheckpointed()) { - status = Status.PUBLISHING; - break; - } - - if (backgroundThreadException != null) { - throw new RuntimeException(backgroundThreadException); - } - - checkPublishAndHandoffFailure(); - - maybePersistAndPublishSequences(committerSupplier); - - // The retrying business is because the KafkaConsumer throws OffsetOutOfRangeException if the seeked-to - // offset is not present in the topic-partition. This can happen if we're asking a task to read from data - // that has not been written yet (which is totally legitimate). So let's wait for it to show up. - ConsumerRecords records = ConsumerRecords.empty(); - try { - records = consumer.poll(KafkaIndexTask.POLL_TIMEOUT_MILLIS); - } - catch (OffsetOutOfRangeException e) { - log.warn("OffsetOutOfRangeException with message [%s]", e.getMessage()); - possiblyResetOffsetsOrWait(e.offsetOutOfRangePartitions(), consumer, toolbox); - stillReading = !assignment.isEmpty(); - } - - SequenceMetadata sequenceToCheckpoint = null; - for (ConsumerRecord record : records) { - log.trace( - "Got topic[%s] partition[%d] offset[%,d].", - record.topic(), - record.partition(), - record.offset() - ); - - if (record.offset() < endOffsets.get(record.partition())) { - if (record.offset() != nextOffsets.get(record.partition())) { - if (ioConfig.isSkipOffsetGaps()) { - log.warn( - "Skipped to offset[%,d] after offset[%,d] in partition[%d].", - record.offset(), - nextOffsets.get(record.partition()), - record.partition() - ); - } else { - throw new ISE( - "WTF?! Got offset[%,d] after offset[%,d] in partition[%d].", - record.offset(), - nextOffsets.get(record.partition()), - record.partition() - ); - } - } - - try { - final byte[] valueBytes = record.value(); - final List rows = valueBytes == null - ? Utils.nullableListOf((InputRow) null) - : parser.parseBatch(ByteBuffer.wrap(valueBytes)); - boolean isPersistRequired = false; - - final SequenceMetadata sequenceToUse = sequences - .stream() - .filter(sequenceMetadata -> sequenceMetadata.canHandle(record)) - .findFirst() - .orElse(null); - - if (sequenceToUse == null) { - throw new ISE( - "WTH?! cannot find any valid sequence for record with partition [%d] and offset [%d]. Current sequences: %s", - record.partition(), - record.offset(), - sequences - ); - } - - for (InputRow row : rows) { - if (row != null && task.withinMinMaxRecordTime(row)) { - final AppenderatorDriverAddResult addResult = driver.add( - row, - sequenceToUse.getSequenceName(), - committerSupplier, - // skip segment lineage check as there will always be one segment - // for combination of sequence and segment granularity. - // It is necessary to skip it as the task puts messages polled from all the - // assigned Kafka partitions into a single Druid segment, thus ordering of - // messages among replica tasks across assigned partitions is not guaranteed - // which may cause replica tasks to ask for segments with different interval - // in different order which might cause SegmentAllocateAction to fail. - true, - // do not allow incremental persists to happen until all the rows from this batch - // of rows are indexed - false - ); - - if (addResult.isOk()) { - // If the number of rows in the segment exceeds the threshold after adding a row, - // move the segment out from the active segments of BaseAppenderatorDriver to make a new segment. - if (addResult.isPushRequired(tuningConfig) && !sequenceToUse.isCheckpointed()) { - sequenceToCheckpoint = sequenceToUse; - } - isPersistRequired |= addResult.isPersistRequired(); - } else { - // Failure to allocate segment puts determinism at risk, bail out to be safe. - // May want configurable behavior here at some point. - // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks. - throw new ISE("Could not allocate segment for row with timestamp[%s]", row.getTimestamp()); - } - - if (addResult.getParseException() != null) { - handleParseException(addResult.getParseException(), record); - } else { - rowIngestionMeters.incrementProcessed(); - } - } else { - rowIngestionMeters.incrementThrownAway(); - } - } - if (isPersistRequired) { - Futures.addCallback( - driver.persistAsync(committerSupplier.get()), - new FutureCallback() - { - @Override - public void onSuccess(@Nullable Object result) - { - log.info("Persist completed with metadata [%s]", result); - } - - @Override - public void onFailure(@ParametersAreNonnullByDefault Throwable t) - { - log.error("Persist failed, dying"); - backgroundThreadException = t; - } - } - ); - } - } - catch (ParseException e) { - handleParseException(e, record); - } - - nextOffsets.put(record.partition(), record.offset() + 1); - } - - if (nextOffsets.get(record.partition()).equals(endOffsets.get(record.partition())) - && assignment.remove(record.partition())) { - log.info("Finished reading topic[%s], partition[%,d].", record.topic(), record.partition()); - KafkaIndexTask.assignPartitions(consumer, topic, assignment); - stillReading = !assignment.isEmpty(); - } - } - - if (System.currentTimeMillis() > nextCheckpointTime) { - sequenceToCheckpoint = sequences.get(sequences.size() - 1); - } - - if (sequenceToCheckpoint != null && stillReading) { - Preconditions.checkArgument( - sequences.get(sequences.size() - 1) - .getSequenceName() - .equals(sequenceToCheckpoint.getSequenceName()), - "Cannot checkpoint a sequence [%s] which is not the latest one, sequences %s", - sequenceToCheckpoint, - sequences - ); - requestPause(); - final CheckPointDataSourceMetadataAction checkpointAction = new CheckPointDataSourceMetadataAction( - task.getDataSource(), - ioConfig.getTaskGroupId(), - task.getIOConfig().getBaseSequenceName(), - new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( - topic, - sequenceToCheckpoint.getStartOffsets() - )), - new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, nextOffsets)) - ); - if (!toolbox.getTaskActionClient().submit(checkpointAction)) { - throw new ISE("Checkpoint request with offsets [%s] failed, dying", nextOffsets); - } - } - } - ingestionState = IngestionState.COMPLETED; - } - catch (Exception e) { - // (1) catch all exceptions while reading from kafka - caughtExceptionInner = e; - log.error(e, "Encountered exception in run() before persisting."); - throw e; - } - finally { - log.info("Persisting all pending data"); - try { - driver.persist(committerSupplier.get()); // persist pending data - } - catch (Exception e) { - if (caughtExceptionInner != null) { - caughtExceptionInner.addSuppressed(e); - } else { - throw e; - } - } - } - - synchronized (statusLock) { - if (stopRequested.get() && !publishOnStop.get()) { - throw new InterruptedException("Stopping without publishing"); - } - - status = Status.PUBLISHING; - } - - for (SequenceMetadata sequenceMetadata : sequences) { - if (!publishingSequences.contains(sequenceMetadata.getSequenceName())) { - // this is done to prevent checks in sequence specific commit supplier from failing - sequenceMetadata.setEndOffsets(nextOffsets); - sequenceMetadata.updateAssignments(nextOffsets); - publishingSequences.add(sequenceMetadata.getSequenceName()); - // persist already done in finally, so directly add to publishQueue - publishAndRegisterHandoff(sequenceMetadata); - } - } - - if (backgroundThreadException != null) { - throw new RuntimeException(backgroundThreadException); - } - - // Wait for publish futures to complete. - Futures.allAsList(publishWaitList).get(); - - // Wait for handoff futures to complete. - // Note that every publishing task (created by calling AppenderatorDriver.publish()) has a corresponding - // handoffFuture. handoffFuture can throw an exception if 1) the corresponding publishFuture failed or 2) it - // failed to persist sequences. It might also return null if handoff failed, but was recoverable. - // See publishAndRegisterHandoff() for details. - List handedOffList = Collections.emptyList(); - if (tuningConfig.getHandoffConditionTimeout() == 0) { - handedOffList = Futures.allAsList(handOffWaitList).get(); - } else { - try { - handedOffList = Futures.allAsList(handOffWaitList) - .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); - } - catch (TimeoutException e) { - // Handoff timeout is not an indexing failure, but coordination failure. We simply ignore timeout exception - // here. - log.makeAlert("Timed out after [%d] millis waiting for handoffs", tuningConfig.getHandoffConditionTimeout()) - .addData("TaskId", task.getId()) - .emit(); - } - } - - for (SegmentsAndMetadata handedOff : handedOffList) { - log.info( - "Handoff completed for segments[%s] with metadata[%s].", - Joiner.on(", ").join( - handedOff.getSegments().stream().map(DataSegment::getIdentifier).collect(Collectors.toList()) - ), - Preconditions.checkNotNull(handedOff.getCommitMetadata(), "commitMetadata") - ); - } - - appenderator.close(); - } - catch (InterruptedException | RejectedExecutionException e) { - // (2) catch InterruptedException and RejectedExecutionException thrown for the whole ingestion steps including - // the final publishing. - caughtExceptionOuter = e; - try { - Futures.allAsList(publishWaitList).cancel(true); - Futures.allAsList(handOffWaitList).cancel(true); - if (appenderator != null) { - appenderator.closeNow(); - } - } - catch (Exception e2) { - e.addSuppressed(e2); - } - - // handle the InterruptedException that gets wrapped in a RejectedExecutionException - if (e instanceof RejectedExecutionException - && (e.getCause() == null || !(e.getCause() instanceof InterruptedException))) { - throw e; - } - - // if we were interrupted because we were asked to stop, handle the exception and return success, else rethrow - if (!stopRequested.get()) { - Thread.currentThread().interrupt(); - throw e; - } - - log.info("The task was asked to stop before completing"); - } - catch (Exception e) { - // (3) catch all other exceptions thrown for the whole ingestion steps including the final publishing. - caughtExceptionOuter = e; - try { - Futures.allAsList(publishWaitList).cancel(true); - Futures.allAsList(handOffWaitList).cancel(true); - if (appenderator != null) { - appenderator.closeNow(); - } - } - catch (Exception e2) { - e.addSuppressed(e2); - } - throw e; - } - finally { - try { - if (driver != null) { - driver.close(); - } - if (chatHandlerProvider.isPresent()) { - chatHandlerProvider.get().unregister(task.getId()); - } - - toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); - toolbox.getDataSegmentServerAnnouncer().unannounce(); - } - catch (Exception e) { - if (caughtExceptionOuter != null) { - caughtExceptionOuter.addSuppressed(e); - } else { - throw e; - } - } - } - - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(null)); - return TaskStatus.success(task.getId()); - } - - private void checkPublishAndHandoffFailure() throws ExecutionException, InterruptedException - { - // Check if any publishFuture failed. - final List> publishFinished = publishWaitList - .stream() - .filter(Future::isDone) - .collect(Collectors.toList()); - - for (ListenableFuture publishFuture : publishFinished) { - // If publishFuture failed, the below line will throw an exception and catched by (1), and then (2) or (3). - publishFuture.get(); - } - - publishWaitList.removeAll(publishFinished); - - // Check if any handoffFuture failed. - final List> handoffFinished = handOffWaitList - .stream() - .filter(Future::isDone) - .collect(Collectors.toList()); - - for (ListenableFuture handoffFuture : handoffFinished) { - // If handoffFuture failed, the below line will throw an exception and catched by (1), and then (2) or (3). - handoffFuture.get(); - } - - handOffWaitList.removeAll(handoffFinished); - } - - private void publishAndRegisterHandoff(SequenceMetadata sequenceMetadata) - { - log.info("Publishing segments for sequence [%s]", sequenceMetadata); - - final ListenableFuture publishFuture = Futures.transform( - driver.publish( - sequenceMetadata.createPublisher(toolbox, ioConfig.isUseTransaction()), - sequenceMetadata.getCommitterSupplier(topic, lastPersistedOffsets).get(), - Collections.singletonList(sequenceMetadata.getSequenceName()) - ), - (Function) publishedSegmentsAndMetadata -> { - if (publishedSegmentsAndMetadata == null) { - throw new ISE( - "Transaction failure publishing segments for sequence [%s]", - sequenceMetadata - ); - } else { - return publishedSegmentsAndMetadata; - } - } - ); - publishWaitList.add(publishFuture); - - // Create a handoffFuture for every publishFuture. The created handoffFuture must fail if publishFuture fails. - final SettableFuture handoffFuture = SettableFuture.create(); - handOffWaitList.add(handoffFuture); - - Futures.addCallback( - publishFuture, - new FutureCallback() - { - @Override - public void onSuccess(SegmentsAndMetadata publishedSegmentsAndMetadata) - { - log.info( - "Published segments[%s] with metadata[%s].", - publishedSegmentsAndMetadata.getSegments() - .stream() - .map(DataSegment::getIdentifier) - .collect(Collectors.toList()), - Preconditions.checkNotNull(publishedSegmentsAndMetadata.getCommitMetadata(), "commitMetadata") - ); - - sequences.remove(sequenceMetadata); - publishingSequences.remove(sequenceMetadata.getSequenceName()); - try { - persistSequences(); - } - catch (IOException e) { - log.error(e, "Unable to persist state, dying"); - handoffFuture.setException(e); - throw new RuntimeException(e); - } - - Futures.transform( - driver.registerHandoff(publishedSegmentsAndMetadata), - new Function() - { - @Nullable - @Override - public Void apply(@Nullable SegmentsAndMetadata handoffSegmentsAndMetadata) - { - if (handoffSegmentsAndMetadata == null) { - log.warn( - "Failed to handoff segments[%s]", - publishedSegmentsAndMetadata.getSegments() - .stream() - .map(DataSegment::getIdentifier) - .collect(Collectors.toList()) - ); - } - handoffFuture.set(handoffSegmentsAndMetadata); - return null; - } - } - ); - } - - @Override - public void onFailure(@ParametersAreNonnullByDefault Throwable t) - { - log.error(t, "Error while publishing segments for sequence[%s]", sequenceMetadata); - handoffFuture.setException(t); - } - } - ); - } - - private static File getSequencesPersistFile(TaskToolbox toolbox) - { - return new File(toolbox.getPersistDir(), "sequences.json"); - } - - private boolean restoreSequences() throws IOException - { - final File sequencesPersistFile = getSequencesPersistFile(toolbox); - if (sequencesPersistFile.exists()) { - sequences = new CopyOnWriteArrayList<>( - toolbox.getObjectMapper().>readValue( - sequencesPersistFile, - new TypeReference>() - { - } - ) - ); - return true; - } else { - return false; - } - } - - private synchronized void persistSequences() throws IOException - { - log.info("Persisting Sequences Metadata [%s]", sequences); - toolbox.getObjectMapper().writerWithType( - new TypeReference>() - { - } - ).writeValue(getSequencesPersistFile(toolbox), sequences); - } - - private Map getTaskCompletionReports(@Nullable String errorMsg) - { - return TaskReport.buildTaskReports( - new IngestionStatsAndErrorsTaskReport( - task.getId(), - new IngestionStatsAndErrorsTaskReportData( - ingestionState, - getTaskCompletionUnparseableEvents(), - getTaskCompletionRowStats(), - errorMsg - ) - ) - ); - } - - private Map getTaskCompletionUnparseableEvents() - { - Map unparseableEventsMap = new HashMap<>(); - List buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions( - savedParseExceptions - ); - if (buildSegmentsParseExceptionMessages != null) { - unparseableEventsMap.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegmentsParseExceptionMessages); - } - return unparseableEventsMap; - } - - private Map getTaskCompletionRowStats() + protected Long getNextSequenceNumber( + RecordSupplier recordSupplier, StreamPartition partition, @NotNull Long sequenceNumber + ) { - Map metrics = new HashMap<>(); - metrics.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getTotals() - ); - return metrics; + return sequenceNumber + 1; } - private void maybePersistAndPublishSequences(Supplier committerSupplier) - throws InterruptedException - { - for (SequenceMetadata sequenceMetadata : sequences) { - sequenceMetadata.updateAssignments(nextOffsets); - if (!sequenceMetadata.isOpen() && !publishingSequences.contains(sequenceMetadata.getSequenceName())) { - publishingSequences.add(sequenceMetadata.getSequenceName()); - try { - Object result = driver.persist(committerSupplier.get()); - log.info( - "Persist completed with results: [%s], adding sequence [%s] to publish queue", - result, - sequenceMetadata - ); - publishAndRegisterHandoff(sequenceMetadata); - } - catch (InterruptedException e) { - log.warn("Interrupted while persisting sequence [%s]", sequenceMetadata); - throw e; - } - } - } - } - - private Set assignPartitionsAndSeekToNext(KafkaConsumer consumer, String topic) - { - // Initialize consumer assignment. - final Set assignment = new HashSet<>(); - for (Map.Entry entry : nextOffsets.entrySet()) { - final long endOffset = endOffsets.get(entry.getKey()); - if (entry.getValue() < endOffset) { - assignment.add(entry.getKey()); - } else if (entry.getValue() == endOffset) { - log.info("Finished reading partition[%d].", entry.getKey()); - } else { - throw new ISE( - "WTF?! Cannot start from offset[%,d] > endOffset[%,d]", - entry.getValue(), - endOffset - ); - } + @Nonnull + @Override + protected List> getRecords( + RecordSupplier recordSupplier, + TaskToolbox toolbox + ) throws Exception + { + // The retrying business is because the KafkaConsumer throws OffsetOutOfRangeException if the seeked-to + // offset is not present in the topic-partition. This can happen if we're asking a task to read from data + // that has not been written yet (which is totally legitimate). So let's wait for it to show up. + List> records = new ArrayList<>(); + try { + records = recordSupplier.poll(KafkaIndexTask.POLL_TIMEOUT_MILLIS); } - - KafkaIndexTask.assignPartitions(consumer, topic, assignment); - - // Seek to starting offsets. - for (final int partition : assignment) { - final long offset = nextOffsets.get(partition); - log.info("Seeking partition[%d] to offset[%,d].", partition, offset); - consumer.seek(new TopicPartition(topic, partition), offset); + catch (OffsetOutOfRangeException e) { + log.warn("OffsetOutOfRangeException with message [%s]", e.getMessage()); + possiblyResetOffsetsOrWait(e.offsetOutOfRangePartitions(), recordSupplier, toolbox); } - return assignment; + return records; } - /** - * Checks if the pauseRequested flag was set and if so blocks until pauseRequested is cleared. - *

- * Sets paused = true and signals paused so callers can be notified when the pause command has been accepted. - *

- * - * @return true if a pause request was handled, false otherwise - */ - private boolean possiblyPause() throws InterruptedException + @Override + protected SeekableStreamPartitions createSeekableStreamPartitions( + ObjectMapper mapper, + Object object + ) { - pauseLock.lockInterruptibly(); - try { - if (pauseRequested) { - status = Status.PAUSED; - hasPaused.signalAll(); - - while (pauseRequested) { - log.info("Pausing ingestion until resumed"); - shouldResume.await(); - } - - status = Status.READING; - shouldResume.signalAll(); - log.info("Ingestion loop resumed"); - return true; - } - } - finally { - pauseLock.unlock(); - } - - return false; + return mapper.convertValue(object, mapper.getTypeFactory().constructParametrizedType( + SeekableStreamPartitions.class, + SeekableStreamPartitions.class, + Integer.class, + Long.class + )); } private void possiblyResetOffsetsOrWait( Map outOfRangePartitions, - KafkaConsumer consumer, + RecordSupplier recordSupplier, TaskToolbox taskToolbox ) throws InterruptedException, IOException { @@ -1065,10 +142,20 @@ private void possiblyResetOffsetsOrWait( final TopicPartition topicPartition = outOfRangePartition.getKey(); final long nextOffset = outOfRangePartition.getValue(); // seek to the beginning to get the least available offset - consumer.seekToBeginning(Collections.singletonList(topicPartition)); - final long leastAvailableOffset = consumer.position(topicPartition); + StreamPartition streamPartition = StreamPartition.of( + topicPartition.topic(), + topicPartition.partition() + ); + recordSupplier.seekToEarliest(ImmutableSet.of(streamPartition)); + final Long leastAvailableOffset = recordSupplier.getEarliestSequenceNumber(streamPartition); + if (leastAvailableOffset == null) { + throw new ISE( + "got null sequence number for partition[%s] when fetching from kafka!", + topicPartition.partition() + ); + } // reset the seek - consumer.seek(topicPartition, nextOffset); + recordSupplier.seek(streamPartition, nextOffset); // Reset consumer offset if resetOffsetAutomatically is set to true // and the current message offset in the kafka partition is more than the // next message offset that we are trying to fetch @@ -1080,7 +167,12 @@ private void possiblyResetOffsetsOrWait( } if (doReset) { - sendResetRequestAndWait(resetPartitions, taskToolbox); + sendResetRequestAndWait(resetPartitions.entrySet() + .stream() + .collect(Collectors.toMap(x -> StreamPartition.of( + x.getKey().topic(), + x.getKey().partition() + ), Map.Entry::getValue)), taskToolbox); } else { log.warn("Retrying in %dms", task.getPollRetryMs()); pollRetryLock.lockInterruptibly(); @@ -1096,728 +188,31 @@ private void possiblyResetOffsetsOrWait( } } - private void handleParseException(ParseException pe, ConsumerRecord record) - { - if (pe.isFromPartiallyValidRow()) { - rowIngestionMeters.incrementProcessedWithError(); - } else { - rowIngestionMeters.incrementUnparseable(); - } - - if (tuningConfig.isLogParseExceptions()) { - log.error( - pe, - "Encountered parse exception on row from partition[%d] offset[%d]", - record.partition(), - record.offset() - ); - } - - if (savedParseExceptions != null) { - savedParseExceptions.add(pe); - } - - if (rowIngestionMeters.getUnparseable() + rowIngestionMeters.getProcessedWithError() - > tuningConfig.getMaxParseExceptions()) { - log.error("Max parse exceptions exceeded, terminating task..."); - throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); - } - } - - private boolean isPaused() - { - return status == Status.PAUSED; - } - - private void requestPause() - { - pauseRequested = true; - } - - private void sendResetRequestAndWait(Map outOfRangePartitions, TaskToolbox taskToolbox) - throws IOException - { - Map partitionOffsetMap = new HashMap<>(); - for (Map.Entry outOfRangePartition : outOfRangePartitions.entrySet()) { - partitionOffsetMap.put(outOfRangePartition.getKey().partition(), outOfRangePartition.getValue()); - } - boolean result = taskToolbox.getTaskActionClient() - .submit(new ResetDataSourceMetadataAction( - task.getDataSource(), - new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( - ioConfig.getStartPartitions() - .getStream(), - partitionOffsetMap - )) - )); - - if (result) { - log.makeAlert("Resetting Kafka offsets for datasource [%s]", task.getDataSource()) - .addData("partitions", partitionOffsetMap.keySet()) - .emit(); - // wait for being killed by supervisor - requestPause(); - } else { - log.makeAlert("Failed to send reset request for partitions [%s]", partitionOffsetMap.keySet()).emit(); - } - } - - /** - * Authorizes action to be performed on this task's datasource - * - * @return authorization result - */ - private Access authorizationCheck(final HttpServletRequest req, Action action) - { - return IndexTaskUtils.datasourceAuthorizationCheck(req, action, task.getDataSource(), authorizerMapper); - } - - @Override - public Appenderator getAppenderator() - { - return appenderator; - } - - @Override - public RowIngestionMeters getRowIngestionMeters() - { - return rowIngestionMeters; - } - - @Override - public void stopGracefully() - { - log.info("Stopping gracefully (status: [%s])", status); - stopRequested.set(true); - - synchronized (statusLock) { - if (status == Status.PUBLISHING) { - runThread.interrupt(); - return; - } - } - - try { - if (pauseLock.tryLock(KafkaIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { - try { - if (pauseRequested) { - pauseRequested = false; - shouldResume.signalAll(); - } - } - finally { - pauseLock.unlock(); - } - } else { - log.warn("While stopping: failed to acquire pauseLock before timeout, interrupting run thread"); - runThread.interrupt(); - return; - } - - if (pollRetryLock.tryLock(KafkaIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { - try { - isAwaitingRetry.signalAll(); - } - finally { - pollRetryLock.unlock(); - } - } else { - log.warn("While stopping: failed to acquire pollRetryLock before timeout, interrupting run thread"); - runThread.interrupt(); - } - } - catch (Exception e) { - Throwables.propagate(e); - } - } - - @POST - @Path("/stop") - public Response stop(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.WRITE); - stopGracefully(); - return Response.status(Response.Status.OK).build(); - } - - @GET - @Path("/status") - @Produces(MediaType.APPLICATION_JSON) - public Status getStatusHTTP(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.READ); - return status; - } - @Override - public Status getStatus() - { - return status; - } - - @GET - @Path("/offsets/current") - @Produces(MediaType.APPLICATION_JSON) - public Map getCurrentOffsets(@Context final HttpServletRequest req) + protected OrderedSequenceNumber createSequencenNumber(Long sequenceNumber) { - authorizationCheck(req, Action.READ); - return getCurrentOffsets(); + return KafkaSequenceNumber.of(sequenceNumber); } @Override - public Map getCurrentOffsets() - { - return nextOffsets; - } - - @GET - @Path("/offsets/end") - @Produces(MediaType.APPLICATION_JSON) - public Map getEndOffsetsHTTP(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.READ); - return getEndOffsets(); - } - - @Override - public Map getEndOffsets() - { - return endOffsets; - } - - @POST - @Path("/offsets/end") - @Consumes(MediaType.APPLICATION_JSON) - @Produces(MediaType.APPLICATION_JSON) - public Response setEndOffsetsHTTP( - Map offsets, - @QueryParam("finish") @DefaultValue("true") final boolean finish, - // this field is only for internal purposes, shouldn't be usually set by users - @Context final HttpServletRequest req - ) throws InterruptedException - { - authorizationCheck(req, Action.WRITE); - return setEndOffsets(offsets, finish); - } - - @GET - @Path("/rowStats") - @Produces(MediaType.APPLICATION_JSON) - public Response getRowStats( - @Context final HttpServletRequest req - ) - { - authorizationCheck(req, Action.READ); - Map returnMap = new HashMap<>(); - Map totalsMap = new HashMap<>(); - Map averagesMap = new HashMap<>(); - - totalsMap.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getTotals() - ); - averagesMap.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getMovingAverages() - ); - - returnMap.put("movingAverages", averagesMap); - returnMap.put("totals", totalsMap); - return Response.ok(returnMap).build(); - } - - @GET - @Path("/unparseableEvents") - @Produces(MediaType.APPLICATION_JSON) - public Response getUnparseableEvents( - @Context final HttpServletRequest req + protected SeekableStreamDataSourceMetadata createDataSourceMetadata( + SeekableStreamPartitions partitions ) { - authorizationCheck(req, Action.READ); - List events = IndexTaskUtils.getMessagesFromSavedParseExceptions(savedParseExceptions); - return Response.ok(events).build(); - } - - @Override - public Response setEndOffsets( - Map offsets, - final boolean finish // this field is only for internal purposes, shouldn't be usually set by users - ) throws InterruptedException - { - if (offsets == null) { - return Response.status(Response.Status.BAD_REQUEST) - .entity("Request body must contain a map of { partition:endOffset }") - .build(); - } else if (!endOffsets.keySet().containsAll(offsets.keySet())) { - return Response.status(Response.Status.BAD_REQUEST) - .entity( - StringUtils.format( - "Request contains partitions not being handled by this task, my partitions: %s", - endOffsets.keySet() - ) - ) - .build(); - } else { - try { - pauseLock.lockInterruptibly(); - // Perform all sequence related checks before checking for isPaused() - // and after acquiring pauseLock to correctly guard against duplicate requests - Preconditions.checkState(sequences.size() > 0, "WTH?! No Sequences found to set end offsets"); - - final SequenceMetadata latestSequence = sequences.get(sequences.size() - 1); - if ((latestSequence.getStartOffsets().equals(offsets) && !finish) || - (latestSequence.getEndOffsets().equals(offsets) && finish)) { - log.warn("Ignoring duplicate request, end offsets already set for sequences [%s]", sequences); - return Response.ok(offsets).build(); - } else if (latestSequence.isCheckpointed()) { - return Response.status(Response.Status.BAD_REQUEST) - .entity(StringUtils.format( - "WTH?! Sequence [%s] has already endOffsets set, cannot set to [%s]", - latestSequence, - offsets - )).build(); - } else if (!isPaused()) { - return Response.status(Response.Status.BAD_REQUEST) - .entity("Task must be paused before changing the end offsets") - .build(); - } - - for (Map.Entry entry : offsets.entrySet()) { - if (entry.getValue().compareTo(nextOffsets.get(entry.getKey())) < 0) { - return Response.status(Response.Status.BAD_REQUEST) - .entity( - StringUtils.format( - "End offset must be >= current offset for partition [%s] (current: %s)", - entry.getKey(), - nextOffsets.get(entry.getKey()) - ) - ) - .build(); - } - } - - resetNextCheckpointTime(); - latestSequence.setEndOffsets(offsets); - - if (finish) { - log.info("Updating endOffsets from [%s] to [%s]", endOffsets, offsets); - endOffsets.putAll(offsets); - } else { - // create new sequence - final SequenceMetadata newSequence = new SequenceMetadata( - latestSequence.getSequenceId() + 1, - StringUtils.format("%s_%d", ioConfig.getBaseSequenceName(), latestSequence.getSequenceId() + 1), - offsets, - endOffsets, - false - ); - sequences.add(newSequence); - } - - persistSequences(); - } - catch (Exception e) { - log.error(e, "Unable to set end offsets, dying"); - backgroundThreadException = e; - // should resume to immediately finish kafka index task as failed - resume(); - return Response.status(Response.Status.INTERNAL_SERVER_ERROR) - .entity(Throwables.getStackTraceAsString(e)) - .build(); - } - finally { - pauseLock.unlock(); - } - } - - resume(); - - return Response.ok(offsets).build(); - } - - private void resetNextCheckpointTime() - { - nextCheckpointTime = DateTimes.nowUtc().plus(tuningConfig.getIntermediateHandoffPeriod()).getMillis(); - } - - @GET - @Path("/checkpoints") - @Produces(MediaType.APPLICATION_JSON) - public Map> getCheckpointsHTTP(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.READ); - return getCheckpoints(); - } - - private Map> getCheckpoints() - { - return new TreeMap<>(sequences.stream() - .collect(Collectors.toMap( - SequenceMetadata::getSequenceId, - SequenceMetadata::getStartOffsets - ))); - } - - /** - * Signals the ingestion loop to pause. - * - * @return one of the following Responses: 400 Bad Request if the task has started publishing; 202 Accepted if the - * method has timed out and returned before the task has paused; 200 OK with a map of the current partition offsets - * in the response body if the task successfully paused - */ - @POST - @Path("/pause") - @Produces(MediaType.APPLICATION_JSON) - public Response pauseHTTP( - @Context final HttpServletRequest req - ) throws InterruptedException - { - authorizationCheck(req, Action.WRITE); - return pause(); + return new KafkaDataSourceMetadata(partitions); } @Override - public Response pause() throws InterruptedException + protected OrderedSequenceNumber createSequenceNumber(Long sequenceNumber) { - if (!(status == Status.PAUSED || status == Status.READING)) { - return Response.status(Response.Status.BAD_REQUEST) - .entity(StringUtils.format("Can't pause, task is not in a pausable state (state: [%s])", status)) - .build(); - } - - pauseLock.lockInterruptibly(); - try { - pauseRequested = true; - - pollRetryLock.lockInterruptibly(); - try { - isAwaitingRetry.signalAll(); - } - finally { - pollRetryLock.unlock(); - } - - if (isPaused()) { - shouldResume.signalAll(); // kick the monitor so it re-awaits with the new pauseMillis - } - - long nanos = TimeUnit.SECONDS.toNanos(2); - while (!isPaused()) { - if (nanos <= 0L) { - return Response.status(Response.Status.ACCEPTED) - .entity("Request accepted but task has not yet paused") - .build(); - } - nanos = hasPaused.awaitNanos(nanos); - } - } - finally { - pauseLock.unlock(); - } - - try { - return Response.ok().entity(toolbox.getObjectMapper().writeValueAsString(getCurrentOffsets())).build(); - } - catch (JsonProcessingException e) { - throw Throwables.propagate(e); - } - } - - @POST - @Path("/resume") - public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException - { - authorizationCheck(req, Action.WRITE); - resume(); - return Response.status(Response.Status.OK).build(); - } - - @Override - public void resume() throws InterruptedException - { - pauseLock.lockInterruptibly(); - try { - pauseRequested = false; - shouldResume.signalAll(); - - long nanos = TimeUnit.SECONDS.toNanos(5); - while (isPaused()) { - if (nanos <= 0L) { - throw new RuntimeException("Resume command was not accepted within 5 seconds"); - } - nanos = shouldResume.awaitNanos(nanos); - } - } - finally { - pauseLock.unlock(); - } - } - - @GET - @Path("/time/start") - @Produces(MediaType.APPLICATION_JSON) - public DateTime getStartTime(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.WRITE); - return startTime; - } - - private static class SequenceMetadata - { - /** - * Lock for accessing {@link #endOffsets} and {@link #checkpointed}. This lock is required because - * {@link #setEndOffsets)} can be called by both the main thread and the HTTP thread. - */ - private final ReentrantLock lock = new ReentrantLock(); - - private final int sequenceId; - private final String sequenceName; - private final Map startOffsets; - private final Map endOffsets; - private final Set assignments; - private final boolean sentinel; - private boolean checkpointed; - - @JsonCreator - public SequenceMetadata( - @JsonProperty("sequenceId") int sequenceId, - @JsonProperty("sequenceName") String sequenceName, - @JsonProperty("startOffsets") Map startOffsets, - @JsonProperty("endOffsets") Map endOffsets, - @JsonProperty("checkpointed") boolean checkpointed - ) - { - Preconditions.checkNotNull(sequenceName); - Preconditions.checkNotNull(startOffsets); - Preconditions.checkNotNull(endOffsets); - this.sequenceId = sequenceId; - this.sequenceName = sequenceName; - this.startOffsets = ImmutableMap.copyOf(startOffsets); - this.endOffsets = new HashMap<>(endOffsets); - this.assignments = new HashSet<>(startOffsets.keySet()); - this.checkpointed = checkpointed; - this.sentinel = false; - } - - @JsonProperty - public int getSequenceId() - { - return sequenceId; - } - - @JsonProperty - public boolean isCheckpointed() - { - lock.lock(); - try { - return checkpointed; - } - finally { - lock.unlock(); - } - } - - @JsonProperty - public String getSequenceName() - { - return sequenceName; - } - - @JsonProperty - public Map getStartOffsets() - { - return startOffsets; - } - - @JsonProperty - public Map getEndOffsets() - { - lock.lock(); - try { - return endOffsets; - } - finally { - lock.unlock(); - } - } - - @JsonProperty - public boolean isSentinel() - { - return sentinel; - } - - void setEndOffsets(Map newEndOffsets) - { - lock.lock(); - try { - endOffsets.putAll(newEndOffsets); - checkpointed = true; - } - finally { - lock.unlock(); - } - } - - void updateAssignments(Map nextPartitionOffset) - { - lock.lock(); - try { - assignments.clear(); - nextPartitionOffset.forEach((key, value) -> { - if (Longs.compare(endOffsets.get(key), nextPartitionOffset.get(key)) > 0) { - assignments.add(key); - } - }); - } - finally { - lock.unlock(); - } - } - - boolean isOpen() - { - return !assignments.isEmpty(); - } - - boolean canHandle(ConsumerRecord record) - { - lock.lock(); - try { - final Long partitionEndOffset = endOffsets.get(record.partition()); - return isOpen() - && partitionEndOffset != null - && record.offset() >= startOffsets.get(record.partition()) - && record.offset() < partitionEndOffset; - } - finally { - lock.unlock(); - } - } - - @Override - public String toString() - { - lock.lock(); - try { - return "SequenceMetadata{" + - "sequenceName='" + sequenceName + '\'' + - ", sequenceId=" + sequenceId + - ", startOffsets=" + startOffsets + - ", endOffsets=" + endOffsets + - ", assignments=" + assignments + - ", sentinel=" + sentinel + - ", checkpointed=" + checkpointed + - '}'; - } - finally { - lock.unlock(); - } - } - - Supplier getCommitterSupplier(String topic, Map lastPersistedOffsets) - { - // Set up committer. - return () -> - new Committer() - { - @Override - public Object getMetadata() - { - lock.lock(); - - try { - Preconditions.checkState( - assignments.isEmpty(), - "This committer can be used only once all the records till offsets [%s] have been consumed, also make" - + " sure to call updateAssignments before using this committer", - endOffsets - ); - - // merge endOffsets for this sequence with globally lastPersistedOffsets - // This is done because this committer would be persisting only sub set of segments - // corresponding to the current sequence. Generally, lastPersistedOffsets should already - // cover endOffsets but just to be sure take max of offsets and persist that - for (Map.Entry partitionOffset : endOffsets.entrySet()) { - lastPersistedOffsets.put( - partitionOffset.getKey(), - Math.max( - partitionOffset.getValue(), - lastPersistedOffsets.getOrDefault(partitionOffset.getKey(), 0L) - ) - ); - } - - // Publish metadata can be different from persist metadata as we are going to publish only - // subset of segments - return ImmutableMap.of( - METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>(topic, lastPersistedOffsets), - METADATA_PUBLISH_PARTITIONS, new SeekableStreamPartitions<>(topic, endOffsets) - ); - } - finally { - lock.unlock(); - } - } - - @Override - public void run() - { - // Do nothing. - } - }; - } - - TransactionalSegmentPublisher createPublisher(TaskToolbox toolbox, boolean useTransaction) - { - return (segments, commitMetadata) -> { - final SeekableStreamPartitions finalPartitions = toolbox.getObjectMapper().convertValue( - ((Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_PUBLISH_PARTITIONS), - toolbox.getObjectMapper() - .getTypeFactory() - .constructParametrizedType( - SeekableStreamPartitions.class, - SeekableStreamPartitions.class, - Integer.class, - Long.class - ) - ); - - // Sanity check, we should only be publishing things that match our desired end state. - if (!getEndOffsets().equals(finalPartitions.getPartitionSequenceNumberMap())) { - throw new ISE( - "WTF?! Driver for sequence [%s], attempted to publish invalid metadata[%s].", - toString(), - commitMetadata - ); - } - - final SegmentTransactionalInsertAction action; - - if (useTransaction) { - action = new SegmentTransactionalInsertAction( - segments, - new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( - finalPartitions.getStream(), - getStartOffsets() - )), - new KafkaDataSourceMetadata(finalPartitions) - ); - } else { - action = new SegmentTransactionalInsertAction(segments, null, null); - } - - log.info("Publishing with isTransaction[%s].", useTransaction); - - return toolbox.getTaskActionClient().submit(action); - }; - } + return KafkaSequenceNumber.of(sequenceNumber); } @Nullable - private static TreeMap> getCheckPointsFromContext( + @Override + protected TreeMap> getCheckPointsFromContext( TaskToolbox toolbox, - KafkaIndexTask task + SeekableStreamIndexTask task ) throws IOException { final String checkpointsString = task.getContextValue("checkpoints"); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java index 362249ce245a..443da7c0ba56 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java @@ -32,10 +32,7 @@ public class KafkaIOConfig extends SeekableStreamIOConfig { - private static final boolean DEFAULT_SKIP_OFFSET_GAPS = false; - private final Map consumerProperties; - private final boolean skipOffsetGaps; @JsonCreator public KafkaIOConfig( @@ -57,11 +54,11 @@ public KafkaIOConfig( endPartitions, useTransaction, minimumMessageTime, - maximumMessageTime + maximumMessageTime, + skipOffsetGaps ); this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); - this.skipOffsetGaps = skipOffsetGaps != null ? skipOffsetGaps : DEFAULT_SKIP_OFFSET_GAPS; for (int partition : endPartitions.getPartitionSequenceNumberMap().keySet()) { Preconditions.checkArgument( @@ -89,12 +86,6 @@ public Map getConsumerProperties() return consumerProperties; } - @JsonProperty - public boolean isSkipOffsetGaps() - { - return skipOffsetGaps; - } - @Override public String toString() { @@ -107,7 +98,7 @@ public String toString() ", useTransaction=" + isUseTransaction() + ", minimumMessageTime=" + getMinimumMessageTime() + ", maximumMessageTime=" + getMaximumMessageTime() + - ", skipOffsetGaps=" + skipOffsetGaps + + ", skipOffsetGaps=" + isSkipOffsetGaps() + '}'; } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 286469be4268..80e72ede0869 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -24,29 +24,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; -import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.query.NoopQueryRunner; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryRunner; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.appenderator.Appenderator; -import org.apache.druid.segment.realtime.appenderator.Appenderators; -import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -54,6 +39,7 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import java.util.ArrayList; +import java.util.HashMap; import java.util.Map; import java.util.Properties; import java.util.Set; @@ -65,9 +51,8 @@ public class KafkaIndexTask extends SeekableStreamIndexTask private static final EmittingLogger log = new EmittingLogger(KafkaIndexTask.class); static final long POLL_TIMEOUT_MILLIS = TimeUnit.MILLISECONDS.toMillis(100); - static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15; - private final SeekableStreamIndexTaskRunner runner; + private final KafkaIOConfig ioConfig; private final ObjectMapper configMapper; // This value can be tuned in some tests @@ -100,26 +85,8 @@ public KafkaIndexTask( "index_kafka" ); this.configMapper = configMapper; - if (context != null && context.get(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED) != null - && ((boolean) context.get(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED))) { - runner = new IncrementalPublishingKafkaIndexTaskRunner( - this, - parser, - authorizerMapper, - this.chatHandlerProvider, - savedParseExceptions, - rowIngestionMetersFactory - ); - } else { - runner = new LegacyKafkaIndexTaskRunner( - this, - parser, - authorizerMapper, - this.chatHandlerProvider, - savedParseExceptions, - rowIngestionMetersFactory - ); - } + this.ioConfig = ioConfig; + } long getPollRetryMs() @@ -128,83 +95,26 @@ long getPollRetryMs() } @Override - public TaskStatus run(final TaskToolbox toolbox) + protected RecordSupplier getRecordSupplier() { - return runner.run(toolbox); - } + ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); - @Override - public boolean canRestore() - { - return true; - } + final Map props = new HashMap<>(ioConfig.getConsumerProperties()); - @Override - public void stopGracefully() - { - runner.stopGracefully(); - } + props.put("auto.offset.reset", "none"); + props.put("key.deserializer", ByteArrayDeserializer.class.getName()); + props.put("value.deserializer", ByteArrayDeserializer.class.getName()); - @Override - public QueryRunner getQueryRunner(Query query) - { - if (runner.getAppenderator() == null) { - // Not yet initialized, no data yet, just return a noop runner. - return new NoopQueryRunner<>(); + return new KafkaRecordSupplier(props, configMapper); + } + finally { + Thread.currentThread().setContextClassLoader(currCtxCl); } - - return (queryPlus, responseContext) -> queryPlus.run(runner.getAppenderator(), responseContext); - } - - Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) - { - return Appenderators.createRealtime( - dataSchema, - tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), - metrics, - toolbox.getSegmentPusher(), - toolbox.getObjectMapper(), - toolbox.getIndexIO(), - toolbox.getIndexMergerV9(), - toolbox.getQueryRunnerFactoryConglomerate(), - toolbox.getSegmentAnnouncer(), - toolbox.getEmitter(), - toolbox.getQueryExecutorService(), - toolbox.getCache(), - toolbox.getCacheConfig(), - toolbox.getCachePopulatorStats() - ); - } - - StreamAppenderatorDriver newDriver( - final Appenderator appenderator, - final TaskToolbox toolbox, - final FireDepartmentMetrics metrics - ) - { - return new StreamAppenderatorDriver( - appenderator, - new ActionBasedSegmentAllocator( - toolbox.getTaskActionClient(), - dataSchema, - (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( - schema.getDataSource(), - row.getTimestamp(), - schema.getGranularitySpec().getQueryGranularity(), - schema.getGranularitySpec().getSegmentGranularity(), - sequenceName, - previousSegmentId, - skipSegmentLineageCheck - ) - ), - toolbox.getSegmentHandoffNotifierFactory(), - new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), - toolbox.getDataSegmentKiller(), - toolbox.getObjectMapper(), - metrics - ); } + @Deprecated KafkaConsumer newConsumer() { ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); @@ -231,7 +141,6 @@ KafkaConsumer newConsumer() } } - static void assignPartitions( final KafkaConsumer consumer, final String topic, @@ -245,39 +154,29 @@ static void assignPartitions( ); } - boolean withinMinMaxRecordTime(final InputRow row) + @Override + protected SeekableStreamIndexTaskRunner createTaskRunner() { - final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent() - && ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp()); - - final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent() - && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp()); - - if (!Intervals.ETERNITY.contains(row.getTimestamp())) { - final String errorMsg = StringUtils.format( - "Encountered row with timestamp that cannot be represented as a long: [%s]", - row + if (context != null && context.get(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED) != null + && ((boolean) context.get(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED))) { + return new IncrementalPublishingKafkaIndexTaskRunner( + this, + parser, + authorizerMapper, + chatHandlerProvider, + savedParseExceptions, + rowIngestionMetersFactory + ); + } else { + return new LegacyKafkaIndexTaskRunner( + this, + parser, + authorizerMapper, + chatHandlerProvider, + savedParseExceptions, + rowIngestionMetersFactory ); - throw new ParseException(errorMsg); - } - - if (log.isDebugEnabled()) { - if (beforeMinimumMessageTime) { - log.debug( - "CurrentTimeStamp[%s] is before MinimumMessageTime[%s]", - row.getTimestamp(), - ioConfig.getMinimumMessageTime().get() - ); - } else if (afterMaximumMessageTime) { - log.debug( - "CurrentTimeStamp[%s] is after MaximumMessageTime[%s]", - row.getTimestamp(), - ioConfig.getMaximumMessageTime().get() - ); - } } - - return !beforeMinimumMessageTime && !afterMaximumMessageTime; } @Override @@ -293,18 +192,6 @@ void setPollRetryMs(long retryMs) this.pollRetryMs = retryMs; } - @VisibleForTesting - Appenderator getAppenderator() - { - return runner.getAppenderator(); - } - - @VisibleForTesting - SeekableStreamIndexTaskRunner getRunner() - { - return runner; - } - @Override @JsonProperty("ioConfig") public KafkaIOConfig getIOConfig() diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 79c253da4319..9220cb1b02d5 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -27,7 +27,6 @@ import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.PasswordProvider; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -43,22 +42,17 @@ import java.util.Properties; import java.util.Random; import java.util.Set; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; public class KafkaRecordSupplier implements RecordSupplier { - private static final EmittingLogger log = new EmittingLogger(KafkaRecordSupplier.class); private static final Random RANDOM = ThreadLocalRandom.current(); private final KafkaConsumer consumer; private final Map consumerProperties; private final ObjectMapper sortingMapper; private boolean closed; - private final BlockingQueue> records; - public KafkaRecordSupplier( Map consumerProperties, @@ -68,7 +62,6 @@ public KafkaRecordSupplier( this.consumerProperties = consumerProperties; this.sortingMapper = sortingMapper; this.consumer = getKafkaConsumer(); - this.records = new LinkedBlockingQueue<>(); } @Override @@ -78,6 +71,7 @@ public void assign(Set> streamPartitions) .stream() .map(x -> new TopicPartition(x.getStream(), x.getPartitionId())) .collect(Collectors.toSet())); + seekToEarliest(streamPartitions); } @Override @@ -86,12 +80,6 @@ public void seek(StreamPartition partition, Long sequenceNumber) consumer.seek(new TopicPartition(partition.getStream(), partition.getPartitionId()), sequenceNumber); } - @Override - public void seekAfter(StreamPartition partition, Long sequenceNumber) - { - seek(partition, sequenceNumber + 1); - } - @Override public void seekToEarliest(Set> partitions) { @@ -157,7 +145,7 @@ public Long getEarliestSequenceNumber(StreamPartition partition) } @Override - public Long position(StreamPartition partition) + public Long getPosition(StreamPartition partition) { return consumer.position(new TopicPartition(partition.getStream(), partition.getPartitionId())); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 9f1b1dc1c8a0..7486b48cda6a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -44,9 +44,14 @@ import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.RealtimeIndexTask; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -94,6 +99,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; @@ -110,7 +116,7 @@ * This class will be removed in a future release. */ @Deprecated -public class LegacyKafkaIndexTaskRunner implements SeekableStreamIndexTaskRunner +public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner { private static final EmittingLogger log = new EmittingLogger(LegacyKafkaIndexTaskRunner.class); private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; @@ -182,6 +188,15 @@ public class LegacyKafkaIndexTaskRunner implements SeekableStreamIndexTaskRunner RowIngestionMetersFactory rowIngestionMetersFactory ) { + super( + task, + parser, + authorizerMapper, + chatHandlerProvider, + savedParseExceptions, + rowIngestionMetersFactory, + true + ); this.task = task; this.ioConfig = task.getIOConfig(); this.tuningConfig = task.getTuningConfig(); @@ -609,6 +624,22 @@ public void run() ); } + @Override + protected Long getNextSequenceNumber( + RecordSupplier recordSupplier, StreamPartition partition, Long sequenceNumber + ) + { + return sequenceNumber + 1; + } + + @Override + protected List> getRecords( + RecordSupplier recordSupplier, TaskToolbox toolbox + ) + { + return null; + } + private Set assignPartitionsAndSeekToNext(KafkaConsumer consumer, String topic) { // Initialize consumer assignment. @@ -674,6 +705,21 @@ private boolean possiblyPause() throws InterruptedException return false; } + @Override + protected SeekableStreamPartitions createSeekableStreamPartitions( + ObjectMapper mapper, + Object obeject + ) + { + return null; + } + + @Override + protected OrderedSequenceNumber createSequencenNumber(Long sequenceNumber) + { + return null; + } + private void possiblyResetOffsetsOrWait( Map outOfRangePartitions, KafkaConsumer consumer, @@ -702,7 +748,7 @@ private void possiblyResetOffsetsOrWait( } if (doReset) { - sendResetRequestAndWait(resetPartitions, taskToolbox); + sendResetRequestAndAndWait(resetPartitions, taskToolbox); } else { log.warn("Retrying in %dms", task.getPollRetryMs()); pollRetryLock.lockInterruptibly(); @@ -718,7 +764,7 @@ private void possiblyResetOffsetsOrWait( } } - private void sendResetRequestAndWait(Map outOfRangePartitions, TaskToolbox taskToolbox) + private void sendResetRequestAndAndWait(Map outOfRangePartitions, TaskToolbox taskToolbox) throws IOException { Map partitionOffsetMap = new HashMap<>(); @@ -751,6 +797,16 @@ private void requestPause() pauseRequested = true; } + @Nullable + @Override + protected TreeMap> getCheckPointsFromContext( + TaskToolbox toolbox, + SeekableStreamIndexTask task + ) + { + return null; + } + private void handleParseException(ParseException pe, ConsumerRecord record) { if (pe.isFromPartiallyValidRow()) { @@ -873,6 +929,7 @@ private Access authorizationCheck(final HttpServletRequest req, Action action) return IndexTaskUtils.datasourceAuthorizationCheck(req, action, task.getDataSource(), authorizerMapper); } + @Override @POST @Path("/stop") public Response stop(@Context final HttpServletRequest req) @@ -882,6 +939,7 @@ public Response stop(@Context final HttpServletRequest req) return Response.status(Response.Status.OK).build(); } + @Override @GET @Path("/status") @Produces(MediaType.APPLICATION_JSON) @@ -897,6 +955,7 @@ public Status getStatus() return status; } + @Override @GET @Path("/offsets/current") @Produces(MediaType.APPLICATION_JSON) @@ -912,6 +971,7 @@ public Map getCurrentOffsets() return nextOffsets; } + @Override @GET @Path("/offsets/end") @Produces(MediaType.APPLICATION_JSON) @@ -928,10 +988,10 @@ public Map getEndOffsets() } @Override - public Response setEndOffsets(Map offsets, boolean finish) throws InterruptedException + public Response setEndOffsets(Map sequenceNumbers, boolean finish) throws InterruptedException { // finish is not used in this mode - return setEndOffsets(offsets); + return setEndOffsets(sequenceNumbers); } @POST @@ -947,6 +1007,7 @@ public Response setEndOffsetsHTTP( return setEndOffsets(offsets); } + @Override @GET @Path("/rowStats") @Produces(MediaType.APPLICATION_JSON) @@ -973,6 +1034,7 @@ public Response getRowStats( return Response.ok(returnMap).build(); } + @Override @GET @Path("/unparseableEvents") @Produces(MediaType.APPLICATION_JSON) @@ -1050,6 +1112,7 @@ private boolean isPaused() * method has timed out and returned before the task has paused; 200 OK with a map of the current partition offsets * in the response body if the task successfully paused */ + @Override @POST @Path("/pause") @Produces(MediaType.APPLICATION_JSON) @@ -1108,6 +1171,7 @@ public Response pause() throws InterruptedException } } + @Override @POST @Path("/resume") public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException @@ -1138,6 +1202,21 @@ public void resume() throws InterruptedException } } + @Override + protected SeekableStreamDataSourceMetadata createDataSourceMetadata( + SeekableStreamPartitions partitions + ) + { + return null; + } + + @Override + protected OrderedSequenceNumber createSequenceNumber(Long sequenceNumber) + { + return null; + } + + @Override @GET @Path("/time/start") @Produces(MediaType.APPLICATION_JSON) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index ad2415ed262a..9779249a0cff 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -25,7 +25,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import com.google.common.base.Optional; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.Task; @@ -58,8 +57,6 @@ import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; import org.joda.time.DateTime; -import javax.annotation.Nullable; -import javax.validation.constraints.NotNull; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -114,8 +111,7 @@ public KafkaSupervisor( mapper, spec, rowIngestionMetersFactory, - false, - true + false ); this.spec = spec; @@ -125,37 +121,6 @@ public KafkaSupervisor( } - @Override - public void checkpoint( - @Nullable Integer taskGroupId, - @Deprecated String baseSequenceName, - DataSourceMetadata previousCheckPoint, - DataSourceMetadata currentCheckPoint - ) - { - Preconditions.checkNotNull(previousCheckPoint, "previousCheckpoint"); - Preconditions.checkNotNull(currentCheckPoint, "current checkpoint cannot be null"); - Preconditions.checkArgument( - spec.getIoConfig() - .getTopic() - .equals(((KafkaDataSourceMetadata) currentCheckPoint).getSeekableStreamPartitions().getStream()), - "Supervisor topic [%s] and topic in checkpoint [%s] does not match", - spec.getIoConfig().getTopic(), - ((KafkaDataSourceMetadata) currentCheckPoint).getSeekableStreamPartitions().getStream() - ); - - log.info("Checkpointing [%s] for taskGroup [%s]", currentCheckPoint, taskGroupId); - addNotice( - new CheckpointNotice( - taskGroupId, - baseSequenceName, - (KafkaDataSourceMetadata) previousCheckPoint, - (KafkaDataSourceMetadata) currentCheckPoint - ) - ); - } - - @Override protected RecordSupplier setupRecordSupplier() { @@ -370,17 +335,6 @@ private Runnable emitLag() }; } - @Override - protected boolean checkSequenceAvailability( - @NotNull Integer partition, - @NotNull Long sequenceFromMetadata - ) throws TimeoutException - { - Long latestOffset = getOffsetFromStreamForPartition(partition, false); - return latestOffset != null - && KafkaSequenceNumber.of(latestOffset).compareTo(KafkaSequenceNumber.of(sequenceFromMetadata)) >= 0; - } - @Override protected Long getNotSetMarker() { diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java index da6411e5deee..693f4fe6c359 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java @@ -78,7 +78,7 @@ public KafkaSupervisorIOConfig( @JsonProperty public String getTopic() { - return getId(); + return getStream(); } @JsonProperty diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java index f74cf9212570..b59ddc58a94d 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -30,7 +30,7 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IndexTaskClient; import org.apache.druid.indexing.common.TaskInfoProvider; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; @@ -112,17 +112,20 @@ public void setUp() headers = createMock(HttpHeaders.class); client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider); - EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)) + .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)) + .andReturn(Optional.of(TaskStatus.running(TEST_ID))) .anyTimes(); - EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); for (String testId : TEST_IDS) { EasyMock.expect(taskInfoProvider.getTaskLocation(testId)) - .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) - .anyTimes(); + .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); EasyMock.expect(taskInfoProvider.getTaskStatus(testId)) - .andReturn(Optional.of(TaskStatus.running(testId))) - .anyTimes(); + .andReturn(Optional.of(TaskStatus.running(testId))) + .anyTimes(); } } @@ -137,14 +140,16 @@ public void testNoTaskLocation() throws IOException { EasyMock.reset(taskInfoProvider); EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); - EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.running(TEST_ID))).anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)) + .andReturn(Optional.of(TaskStatus.running(TEST_ID))) + .anyTimes(); replayAll(); Assert.assertEquals(false, client.stop(TEST_ID, true)); Assert.assertEquals(false, client.resume(TEST_ID)); Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID)); Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID)); - Assert.assertEquals(SeekableStreamIndexTask.Status.NOT_STARTED, client.getStatus(TEST_ID)); + Assert.assertEquals(Status.NOT_STARTED, client.getStatus(TEST_ID)); Assert.assertEquals(null, client.getStartTime(TEST_ID)); Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsets(TEST_ID, true)); Assert.assertEquals(ImmutableMap.of(), client.getEndOffsets(TEST_ID)); @@ -161,9 +166,12 @@ public void testTaskNotRunnableException() expectedException.expectMessage("Aborting request because task [test-id] is not runnable"); EasyMock.reset(taskInfoProvider); - EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) - .anyTimes(); - EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)).andReturn(Optional.of(TaskStatus.failure(TEST_ID))).anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)) + .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)) + .andReturn(Optional.of(TaskStatus.failure(TEST_ID))) + .anyTimes(); replayAll(); client.getCurrentOffsets(TEST_ID, true); @@ -219,10 +227,10 @@ public void testBadRequest() public void testTaskLocationMismatch() { EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) - .andReturn(HttpResponseStatus.OK); + .andReturn(HttpResponseStatus.OK); EasyMock.expect(responseHolder.getResponse()).andReturn(response); EasyMock.expect(responseHolder.getContent()).andReturn("") - .andReturn("{}"); + .andReturn("{}"); EasyMock.expect(response.headers()).andReturn(headers); EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id"); EasyMock.expect( @@ -248,7 +256,11 @@ public void testGetCurrentOffsets() throws Exception Capture captured = Capture.newInstance(); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -276,14 +288,18 @@ public void testGetCurrentOffsetsWithRetry() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) - .andReturn(HttpResponseStatus.OK).times(1); + .andReturn(HttpResponseStatus.OK).times(1); EasyMock.expect(responseHolder.getContent()).andReturn("").times(2) - .andReturn("{\"0\":1, \"1\":10}"); + .andReturn("{\"0\":1, \"1\":10}"); EasyMock.expect(responseHolder.getResponse()).andReturn(response).times(2); EasyMock.expect(response.headers()).andReturn(headers).times(2); EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(3); @@ -340,7 +356,11 @@ public void testGetEndOffsets() throws Exception Capture captured = Capture.newInstance(); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -369,12 +389,16 @@ public void testGetStartTime() throws Exception Capture captured = Capture.newInstance(); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) - .andReturn(HttpResponseStatus.OK); + .andReturn(HttpResponseStatus.OK); EasyMock.expect(responseHolder.getResponse()).andReturn(response); EasyMock.expect(response.headers()).andReturn(headers); EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(null); EasyMock.expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(2); replayAll(); @@ -396,17 +420,21 @@ public void testGetStartTime() throws Exception @Test public void testGetStatus() throws Exception { - SeekableStreamIndexTask.Status status = SeekableStreamIndexTask.Status.READING; + Status status = Status.READING; Capture captured = Capture.newInstance(); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); EasyMock.expect(responseHolder.getContent()).andReturn(StringUtils.format("\"%s\"", status.toString())).anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); - SeekableStreamIndexTask.Status results = client.getStatus(TEST_ID); + Status results = client.getStatus(TEST_ID); verifyAll(); Request request = captured.getValue(); @@ -426,7 +454,11 @@ public void testPause() throws Exception Capture captured = Capture.newInstance(); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -454,16 +486,28 @@ public void testPauseWithSubsequentGetOffsets() throws Exception Capture captured2 = Capture.newInstance(); Capture captured3 = Capture.newInstance(); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.ACCEPTED).times(2) - .andReturn(HttpResponseStatus.OK).times(2); + .andReturn(HttpResponseStatus.OK).times(2); EasyMock.expect(responseHolder.getContent()).andReturn("\"PAUSED\"") - .andReturn("{\"0\":1, \"1\":10}").anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + .andReturn("{\"0\":1, \"1\":10}").anyTimes(); + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); - EasyMock.expect(httpClient.go(EasyMock.capture(captured2), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured2), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); - EasyMock.expect(httpClient.go(EasyMock.capture(captured3), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured3), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); @@ -504,7 +548,11 @@ public void testResume() throws Exception { Capture captured = Capture.newInstance(); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -528,7 +576,11 @@ public void testSetEndOffsets() throws Exception Capture captured = Capture.newInstance(); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -553,7 +605,11 @@ public void testSetEndOffsetsAndResume() throws Exception Capture captured = Capture.newInstance(); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -576,7 +632,11 @@ public void testStop() throws Exception { Capture captured = Capture.newInstance(); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -598,7 +658,11 @@ public void testStopAndPublish() throws Exception { Capture captured = Capture.newInstance(); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ); replayAll(); @@ -621,7 +685,11 @@ public void testStopAsync() throws Exception final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -653,7 +721,11 @@ public void testResumeAsync() throws Exception final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -686,7 +758,11 @@ public void testPauseAsync() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -719,19 +795,23 @@ public void testGetStatusAsync() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); EasyMock.expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); List expectedUrls = new ArrayList<>(); - List> futures = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "status"))); futures.add(client.getStatusAsync(testId)); } - List responses = Futures.allAsList(futures).get(); + List responses = Futures.allAsList(futures).get(); verifyAll(); List requests = captured.getValues(); @@ -741,7 +821,7 @@ public void testGetStatusAsync() throws Exception for (int i = 0; i < numRequests; i++) { Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); - Assert.assertEquals(SeekableStreamIndexTask.Status.READING, responses.get(i)); + Assert.assertEquals(Status.READING, responses.get(i)); } } @@ -753,7 +833,11 @@ public void testGetStartTimeAsync() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); EasyMock.expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -786,7 +870,11 @@ public void testGetCurrentOffsetsAsync() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -819,7 +907,11 @@ public void testGetEndOffsetsAsync() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -852,7 +944,11 @@ public void testSetEndOffsetsAsync() throws Exception final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); @@ -891,7 +987,11 @@ public void testSetEndOffsetsAsyncWithResume() throws Exception final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go(EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT))).andReturn( + EasyMock.expect(httpClient.go( + EasyMock.capture(captured), + EasyMock.anyObject(FullResponseHandler.class), + EasyMock.eq(TEST_HTTP_TIMEOUT) + )).andReturn( Futures.immediateFuture(responseHolder) ).times(numRequests); replayAll(); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 1448be80817a..06c61a0691eb 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -77,7 +77,7 @@ import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; import org.apache.druid.indexing.test.TestDataSegmentKiller; @@ -439,7 +439,7 @@ public void testRunBeforeDataInserted() throws Exception final ListenableFuture future = runTask(task); // Wait for the task to start reading - while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { + while (task.getRunner().getStatus() != Status.READING) { Thread.sleep(10); } @@ -544,7 +544,7 @@ public void testIncrementalHandOff() throws Exception ) ); final ListenableFuture future = runTask(task); - while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { + while (task.getRunner().getStatus() != Status.PAUSED) { Thread.sleep(10); } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); @@ -670,7 +670,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception ) ); final ListenableFuture future = runTask(task); - while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { + while (task.getRunner().getStatus() != Status.PAUSED) { Thread.sleep(10); } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); @@ -678,7 +678,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets)); task.getRunner().setEndOffsets(currentOffsets, false); - while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { + while (task.getRunner().getStatus() != Status.PAUSED) { Thread.sleep(10); } @@ -812,7 +812,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception final ListenableFuture future = runTask(task); // task will pause for checkpointing - while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.PAUSED) { + while (task.getRunner().getStatus() != Status.PAUSED) { Thread.sleep(10); } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); @@ -872,7 +872,7 @@ public void testRunWithMinimumMessageTime() throws Exception final ListenableFuture future = runTask(task); // Wait for the task to start reading - while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { + while (task.getRunner().getStatus() != Status.READING) { Thread.sleep(10); } @@ -926,7 +926,7 @@ public void testRunWithMaximumMessageTime() throws Exception final ListenableFuture future = runTask(task); // Wait for the task to start reading - while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { + while (task.getRunner().getStatus() != Status.READING) { Thread.sleep(10); } @@ -990,7 +990,7 @@ public void testRunWithTransformSpec() throws Exception final ListenableFuture future = runTask(task); // Wait for the task to start reading - while (task.getRunner().getStatus() != SeekableStreamIndexTask.Status.READING) { + while (task.getRunner().getStatus() != Status.READING) { Thread.sleep(10); } @@ -1831,7 +1831,7 @@ public void testRunWithPauseAndResume() throws Exception } Assert.assertEquals(2, countEvents(task)); - Assert.assertEquals(SeekableStreamIndexTask.Status.READING, task.getRunner().getStatus()); + Assert.assertEquals(Status.READING, task.getRunner().getStatus()); Map currentOffsets = objectMapper.readValue( task.getRunner().pause().getEntity().toString(), @@ -1839,7 +1839,7 @@ public void testRunWithPauseAndResume() throws Exception { } ); - Assert.assertEquals(SeekableStreamIndexTask.Status.PAUSED, task.getRunner().getStatus()); + Assert.assertEquals(Status.PAUSED, task.getRunner().getStatus()); // Insert remaining data try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { @@ -1902,13 +1902,13 @@ public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception runTask(task); - while (!task.getRunner().getStatus().equals(SeekableStreamIndexTask.Status.READING)) { + while (!task.getRunner().getStatus().equals(Status.READING)) { Thread.sleep(2000); } task.getRunner().pause(); - while (!task.getRunner().getStatus().equals(SeekableStreamIndexTask.Status.PAUSED)) { + while (!task.getRunner().getStatus().equals(Status.PAUSED)) { Thread.sleep(25); } } @@ -1941,12 +1941,12 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva runTask(task); - while (!task.getRunner().getStatus().equals(SeekableStreamIndexTask.Status.READING)) { + while (!task.getRunner().getStatus().equals(Status.READING)) { Thread.sleep(20); } for (int i = 0; i < 5; i++) { - Assert.assertEquals(task.getRunner().getStatus(), SeekableStreamIndexTask.Status.READING); + Assert.assertEquals(task.getRunner().getStatus(), Status.READING); // Offset should not be reset Assert.assertTrue(task.getRunner().getCurrentOffsets().get(0) == 200L); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java index 6d768f4af2a5..0f892540b170 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java @@ -407,27 +407,27 @@ public void testPosition() throws ExecutionException, InterruptedException recordSupplier.assign(partitions); - Assert.assertEquals(0L, (long) recordSupplier.position(partition0)); - Assert.assertEquals(0L, (long) recordSupplier.position(partition1)); + Assert.assertEquals(0L, (long) recordSupplier.getPosition(partition0)); + Assert.assertEquals(0L, (long) recordSupplier.getPosition(partition1)); recordSupplier.seek(partition0, 4L); recordSupplier.seek(partition1, 5L); - Assert.assertEquals(4L, (long) recordSupplier.position(partition0)); - Assert.assertEquals(5L, (long) recordSupplier.position(partition1)); + Assert.assertEquals(4L, (long) recordSupplier.getPosition(partition0)); + Assert.assertEquals(5L, (long) recordSupplier.getPosition(partition1)); recordSupplier.seekToEarliest(Collections.singleton(partition0)); - Assert.assertEquals(0L, (long) recordSupplier.position(partition0)); + Assert.assertEquals(0L, (long) recordSupplier.getPosition(partition0)); recordSupplier.seekToLatest(Collections.singleton(partition0)); - Assert.assertEquals(11L, (long) recordSupplier.position(partition0)); + Assert.assertEquals(11L, (long) recordSupplier.getPosition(partition0)); - long prevPos = recordSupplier.position(partition0); + long prevPos = recordSupplier.getPosition(partition0); recordSupplier.getEarliestSequenceNumber(partition0); - Assert.assertEquals(prevPos, (long) recordSupplier.position(partition0)); + Assert.assertEquals(prevPos, (long) recordSupplier.getPosition(partition0)); recordSupplier.getLatestSequenceNumber(partition0); - Assert.assertEquals(prevPos, (long) recordSupplier.position(partition0)); + Assert.assertEquals(prevPos, (long) recordSupplier.getPosition(partition0)); recordSupplier.close(); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 0b6ac11d0545..0fcf2cbdf034 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -59,8 +59,7 @@ import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.Status; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.java.util.common.DateTimes; @@ -2321,7 +2320,7 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException Assert.assertTrue(serviceEmitter.getStackTrace() .startsWith("org.apache.druid.java.util.common.ISE: WTH?! cannot find")); Assert.assertEquals( - "WTH?! cannot find taskGroup [0] among all taskGroups [{}]", + "WTH?! cannot find taskGroup [0] among all activelyReadingTaskGroups [{}]", serviceEmitter.getExceptionMessage() ); Assert.assertEquals(ISE.class, serviceEmitter.getExceptionClass()); @@ -2518,11 +2517,11 @@ public void testSuspendedRunningTasks() throws Exception ) ).anyTimes(); expect(taskClient.getStatusAsync("id1")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + .andReturn(Futures.immediateFuture(Status.PUBLISHING)); expect(taskClient.getStatusAsync("id2")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); expect(taskClient.getStatusAsync("id3")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)); @@ -2667,7 +2666,7 @@ public void testFailedInitializationAndRecovery() throws Exception } @Test - public void testGetCurrentTotalStats() throws Exception + public void testGetCurrentTotalStats() { supervisor = getSupervisor(1, 2, true, "PT1H", null, null, false); supervisor.addTaskGroupToActivelyReadingTaskGroup( diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java index 07baeceaae9a..10cde9ea52f0 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.joda.time.DateTime; import java.util.Set; @@ -72,8 +73,13 @@ public KinesisIOConfig( endPartitions, useTransaction, minimumMessageTime, - maximumMessageTime + maximumMessageTime, + true ); + Preconditions.checkArgument(endPartitions.getPartitionOffsetMap() + .values() + .stream() + .noneMatch(x -> x.equals(OrderedPartitionableRecord.END_OF_SHARD_MARKER))); this.pauseAfterRead = pauseAfterRead != null ? pauseAfterRead : DEFAULT_PAUSE_AFTER_READ; this.endpoint = Preconditions.checkNotNull(endpoint, "endpoint"); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 21e9097719c5..5e1e171778d8 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -22,165 +22,21 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Joiner; -import com.google.common.base.Supplier; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import org.apache.druid.data.input.Committer; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.discovery.DiscoveryDruidNode; -import org.apache.druid.discovery.LookupNodeService; -import org.apache.druid.discovery.NodeType; -import org.apache.druid.indexer.IngestionState; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; -import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; -import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; -import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; -import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; -import org.apache.druid.indexing.common.TaskReport; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.ResetDataSourceMetadataAction; -import org.apache.druid.indexing.common.actions.SegmentAllocateAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.IndexTaskUtils; -import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; -import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; -import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; -import org.apache.druid.indexing.seekablestream.common.StreamPartition; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.collect.Utils; -import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.query.NoopQueryRunner; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryRunner; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.FireDepartmentMetrics; -import org.apache.druid.segment.realtime.appenderator.Appenderator; -import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; -import org.apache.druid.segment.realtime.appenderator.Appenderators; -import org.apache.druid.segment.realtime.appenderator.SegmentIdentifier; -import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata; -import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; -import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.server.security.Access; -import org.apache.druid.server.security.Action; -import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; -import org.apache.druid.server.security.ForbiddenException; -import org.apache.druid.server.security.Resource; -import org.apache.druid.server.security.ResourceAction; -import org.apache.druid.server.security.ResourceType; -import org.apache.druid.timeline.DataSegment; -import org.joda.time.DateTime; -import javax.annotation.Nullable; -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.Consumes; -import javax.ws.rs.DefaultValue; -import javax.ws.rs.GET; -import javax.ws.rs.POST; -import javax.ws.rs.Path; -import javax.ws.rs.Produces; -import javax.ws.rs.QueryParam; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -import java.util.stream.Collectors; public class KinesisIndexTask extends SeekableStreamIndexTask { - public static final long PAUSE_FOREVER = -1L; - - private static final EmittingLogger log = new EmittingLogger(KinesisIndexTask.class); - - private static final long POLL_TIMEOUT = 100; - private static final long POLL_RETRY_MS = 30000; - private static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15; - private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; - - private final AuthorizerMapper authorizerMapper; - - private final Map endOffsets = new ConcurrentHashMap<>(); - private final Map lastOffsets = new ConcurrentHashMap<>(); private final KinesisIOConfig ioConfig; private final KinesisTuningConfig tuningConfig; - private final Lock pauseLock = new ReentrantLock(); - private final Condition hasPaused = pauseLock.newCondition(); - private final Condition shouldResume = pauseLock.newCondition(); - // [pollRetryLock] and [isAwaitingRetry] is used when the Kafka consumer returns an OffsetOutOfRangeException and we - // pause polling from Kafka for POLL_RETRY_MS before trying again. This allows us to signal the sleeping thread and - // resume the main run loop in the case of a pause or stop request from a Jetty thread. - private final Lock pollRetryLock = new ReentrantLock(); - private final Condition isAwaitingRetry = pollRetryLock.newCondition(); - // [statusLock] is used to synchronize the Jetty thread calling stopGracefully() with the main run thread. It prevents - // the main run thread from switching into a publishing state while the stopGracefully() thread thinks it's still in - // a pre-publishing state. This is important because stopGracefully() will try to use the [stopRequested] flag to stop - // the main thread where possible, but this flag is not honored once publishing has begun so in this case we must - // interrupt the thread. The lock ensures that if the run thread is about to transition into publishing state, it - // blocks until after stopGracefully() has set [stopRequested] and then does a final check on [stopRequested] before - // transitioning to publishing state. - private final Object statusLock = new Object(); - private final RowIngestionMeters rowIngestionMeters; - private ObjectMapper mapper; - - // The pause lock and associated conditions are to support coordination between the Jetty threads and the main - // ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully - // the ingestion loop has been stopped at the returned offsets and will not ingest any more data until resumed. The - // fields are used as follows (every step requires acquiring [pauseLock]): - // Pausing: - // - In pause(), [pauseRequested] is set to true and then execution waits for [status] to change to PAUSED, with the - // condition checked when [hasPaused] is signalled. - // - In possiblyPause() called from the main loop, if [pauseRequested] is true, [status] is set to PAUSED, - // [hasPaused] is signalled, and execution pauses until [pauseRequested] becomes false, either by being set or by - // the [pauseMillis] timeout elapsing. [pauseRequested] is checked when [shouldResume] is signalled. - // Resuming: - // - In resume(), [pauseRequested] is set to false, [shouldResume] is signalled, and execution waits for [status] to - // change to something other than PAUSED, with the condition checked when [shouldResume] is signalled. - // - In possiblyPause(), when [shouldResume] is signalled, if [pauseRequested] has become false the pause loop ends, - // [status] is changed to STARTING and [shouldResume] is signalled. - private volatile Appenderator appenderator = null; - private volatile FireDepartmentMetrics fireDepartmentMetrics = null; - private volatile DateTime startTime; - private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread) - private volatile Thread runThread = null; - private volatile boolean stopRequested = false; - private volatile boolean publishOnStop = false; - private IngestionState ingestionState; - - private volatile boolean pauseRequested = false; - private volatile long pauseMillis = 0; @JsonCreator public KinesisIndexTask( @@ -208,441 +64,12 @@ public KinesisIndexTask( "index_kinesis" ); - this.authorizerMapper = authorizerMapper; - this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters(); - this.ingestionState = IngestionState.NOT_STARTED; this.ioConfig = ioConfig; this.tuningConfig = tuningConfig; - this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionSequenceNumberMap()); } @Override - public TaskStatus run(final TaskToolbox toolbox) throws Exception - { - log.info("Starting up!"); - startTime = DateTimes.nowUtc(); - mapper = toolbox.getObjectMapper(); - status = Status.STARTING; - - if (chatHandlerProvider.isPresent()) { - log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); - chatHandlerProvider.get().register(getId(), this, false); - } else { - log.warn("No chat handler detected"); - } - - runThread = Thread.currentThread(); - - // Set up FireDepartmentMetrics - final FireDepartment fireDepartmentForMetrics = new FireDepartment( - dataSchema, - new RealtimeIOConfig(null, null, null), - null - ); - fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); - toolbox.getMonitorScheduler() - .addMonitor(TaskRealtimeMetricsMonitorBuilder.build(this, fireDepartmentForMetrics, rowIngestionMeters)); - - final LookupNodeService lookupNodeService = getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER) == null ? - toolbox.getLookupNodeService() : - new LookupNodeService(getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER)); - - final DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( - toolbox.getDruidNode(), - NodeType.PEON, - ImmutableMap.of( - toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), - lookupNodeService.getName(), lookupNodeService - ) - ); - - try ( - final RecordSupplier recordSupplier = getRecordSupplier() - ) { - toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); - toolbox.getDataSegmentServerAnnouncer().announce(); - - final Appenderator appenderator0 = newAppenderator(fireDepartmentMetrics, toolbox); - final StreamAppenderatorDriver driver = newDriver(appenderator0, toolbox, fireDepartmentMetrics); - - appenderator = appenderator0; - - final String topic = ioConfig.getStartPartitions().getStream(); - - // Start up, set up initial offsets. - final Object restoredMetadata = driver.startJob(); - if (restoredMetadata == null) { - log.info( - "no restored metadata found for [%s], using starting sequences[%s] from ioConfig", - topic, - ioConfig.getStartPartitions() - ); - lastOffsets.putAll(ioConfig.getStartPartitions().getPartitionSequenceNumberMap()); - } else { - log.info("found meatadata [%s] for [%s]", restoredMetadata, topic); - @SuppressWarnings("unchecked") - final Map restoredMetadataMap = (Map) restoredMetadata; - final SeekableStreamPartitions restoredNextPartitions = toolbox - .getObjectMapper() - .convertValue( - restoredMetadataMap.get(METADATA_NEXT_PARTITIONS), - toolbox.getObjectMapper().getTypeFactory().constructParametrizedType( - SeekableStreamPartitions.class, - SeekableStreamPartitions.class, - String.class, - String.class - ) - ); - lastOffsets.putAll(restoredNextPartitions.getPartitionSequenceNumberMap()); - - // Sanity checks. - if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) { - throw new ISE( - "WTF?! Restored stream[%s] but expected stream[%s]", - restoredNextPartitions.getStream(), - ioConfig.getStartPartitions().getStream() - ); - } - - if (!lastOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet())) { - throw new ISE( - "WTF?! Restored partitions[%s] but expected partitions[%s]", - lastOffsets.keySet(), - ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet() - ); - } - } - - // Filter out partitions with END_OF_SHARD markers since these partitions have already been fully read. This - // should have been done by the supervisor already so this is defensive. - int numPreFilterPartitions = lastOffsets.size(); - if (lastOffsets.entrySet().removeIf(x -> OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(x.getValue()))) { - log.info( - "Removed [%d] partitions from assignment which have already been closed", - numPreFilterPartitions - lastOffsets.size() - ); - } - - // Set up sequenceNames. - final Map sequenceNames = new HashMap<>(); - for (String partitionNum : lastOffsets.keySet()) { - sequenceNames.put(partitionNum, StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), partitionNum)); - } - - // Set up committer. - final Supplier committerSupplier = () -> { - final Map snapshot = ImmutableMap.copyOf(lastOffsets); - - return new Committer() - { - @Override - public Object getMetadata() - { - return ImmutableMap.of( - METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>( - ioConfig.getStartPartitions().getStream(), - snapshot - ) - ); - } - - @Override - public void run() - { - // Do nothing. - } - }; - }; - - Set assignment = assignPartitions(recordSupplier, topic); - seekToStartingRecords(recordSupplier, topic, assignment, toolbox); - - Map contiguousOffsetCheck = new HashMap<>(lastOffsets); - boolean verifiedAllStartingOffsets = false; - - // Main loop. - // Could eventually support leader/follower mode (for keeping replicas more in sync) - boolean stillReading = !assignment.isEmpty(); - status = Status.READING; - try { - while (stillReading) { - if (possiblyPause(assignment)) { - assignment = assignPartitions(recordSupplier, topic); - - if (assignment.isEmpty()) { - log.info("All partitions have been fully read"); - publishOnStop = true; - stopRequested = true; - } - } - - if (stopRequested) { - break; - } - - List> records = recordSupplier.poll(POLL_TIMEOUT); - - if (records.size() == 0) { - continue; - } - - for (OrderedPartitionableRecord record : records) { - - // for the first message we receive, check that we were given a message with a sequenceNumber that matches our - // expected starting sequenceNumber - if (!verifiedAllStartingOffsets && contiguousOffsetCheck.containsKey(record.getPartitionId())) { - if (!contiguousOffsetCheck.get(record.getPartitionId()).equals(record.getSequenceNumber())) { - throw new ISE( - "Starting sequenceNumber [%s] does not match expected [%s] for partition [%s]", - record.getSequenceNumber(), - contiguousOffsetCheck.get(record.getPartitionId()), - record.getPartitionId() - ); - } - - log.info( - "Verified starting sequenceNumber [%s] for partition [%s]", - record.getSequenceNumber(), record.getPartitionId() - ); - - contiguousOffsetCheck.remove(record.getPartitionId()); - if (contiguousOffsetCheck.isEmpty()) { - verifiedAllStartingOffsets = true; - log.info("Verified starting offsets for all partitions"); - } - - if (ioConfig.getExclusiveStartSequenceNumberPartitions() != null - && ioConfig.getExclusiveStartSequenceNumberPartitions().contains(record.getPartitionId())) { - log.info( - "Skipping starting sequenceNumber for partition [%s] marked exclusive", - record.getPartitionId() - ); - - continue; - } - } - - if (log.isTraceEnabled()) { - log.trace( - "Got topic[%s] partition[%s] offset[%s].", - record.getStream(), - record.getPartitionId(), - record.getSequenceNumber() - ); - } - - if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(record.getSequenceNumber())) { - lastOffsets.put(record.getPartitionId(), record.getSequenceNumber()); - - } else if (SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffsets.get(record.getPartitionId())) - || record.getSequenceNumber().compareTo(endOffsets.get(record.getPartitionId())) <= 0) { - - try { - final List valueBytess = record.getData(); - - final List rows; - if (valueBytess == null || valueBytess.isEmpty()) { - rows = Utils.nullableListOf((InputRow) null); - } else { - rows = new ArrayList<>(); - for (byte[] valueBytes : valueBytess) { - rows.addAll(parser.parseBatch(ByteBuffer.wrap(valueBytes))); - } - } - - boolean isPersistRequired = false; - final Map> segmentsToMoveOut = new HashMap<>(); - - for (final InputRow row : rows) { - if (row != null && withinMinMaxRecordTime(row)) { - final String sequenceName = sequenceNames.get(record.getPartitionId()); - final AppenderatorDriverAddResult addResult = driver.add( - row, - sequenceName, - committerSupplier, - false, - false - ); - - if (addResult.isOk()) { - // If the number of rows in the segment exceeds the threshold after adding a row, - // move the segment out from the active segments of BaseAppenderatorDriver to make a new segment. - if (addResult.getNumRowsInSegment() > tuningConfig.getMaxRowsPerSegment()) { - segmentsToMoveOut.computeIfAbsent(sequenceName, k -> new HashSet<>()) - .add(addResult.getSegmentIdentifier()); - } - isPersistRequired |= addResult.isPersistRequired(); - } else { - // Failure to allocate segment puts determinism at risk, bail out to be safe. - // May want configurable behavior here at some point. - // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks. - throw new ISE("Could not allocate segment for row with timestamp[%s]", row.getTimestamp()); - } - - if (addResult.getParseException() != null) { - handleParseException(addResult.getParseException(), record); - } else { - rowIngestionMeters.incrementProcessed(); - } - } else { - rowIngestionMeters.incrementThrownAway(); - } - } - - if (isPersistRequired) { - driver.persist(committerSupplier.get()); - } - segmentsToMoveOut.forEach((key, value) -> driver.moveSegmentOut( - key, - new ArrayList(value) - )); - } - catch (ParseException e) { - handleParseException(e, record); - } - - lastOffsets.put(record.getPartitionId(), record.getSequenceNumber()); - - - } - if ((lastOffsets.get(record.getPartitionId()).equals(endOffsets.get(record.getPartitionId())) - || OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(lastOffsets.get(record.getPartitionId()))) - && assignment.remove(record.getPartitionId())) { - - log.info("Finished reading stream[%s], partition[%s].", record.getStream(), record.getPartitionId()); - assignPartitions(recordSupplier, topic, assignment); - stillReading = !assignment.isEmpty(); - } - } - } - } - catch (Exception e) { - log.error(e, "Encountered exception while running task."); - final String errorMsg = Throwables.getStackTraceAsString(e); - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(errorMsg)); - return TaskStatus.failure( - getId(), - errorMsg - ); - } - finally { - log.info("Persisting all pending data"); - driver.persist(committerSupplier.get()); // persist pending data - } - - synchronized (statusLock) { - if (stopRequested && !publishOnStop) { - throw new InterruptedException("Stopping without publishing"); - } - - status = Status.PUBLISHING; - } - - final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { - final SeekableStreamPartitions finalPartitions = toolbox.getObjectMapper().convertValue( - ((Map) commitMetadata).get(METADATA_NEXT_PARTITIONS), - toolbox.getObjectMapper().getTypeFactory().constructParametrizedType( - SeekableStreamPartitions.class, - SeekableStreamPartitions.class, - String.class, - String.class - ) - ); - - // Sanity check, we should only be publishing things that match our desired end state. - if (!endOffsets.equals(finalPartitions.getPartitionSequenceNumberMap())) { - throw new ISE( - "WTF?! Driver attempted to publish invalid metadata[%s], final sequences are [%s]", - commitMetadata, - endOffsets - ); - } - - final SegmentTransactionalInsertAction action; - - if (ioConfig.isUseTransaction()) { - action = new SegmentTransactionalInsertAction( - segments, - new KinesisDataSourceMetadata(ioConfig.getStartPartitions()), - new KinesisDataSourceMetadata(finalPartitions) - ); - } else { - action = new SegmentTransactionalInsertAction(segments, null, null); - } - - log.info("Publishing with isTransaction[%s].", ioConfig.isUseTransaction()); - - return toolbox.getTaskActionClient().submit(action); - }; - - // Supervised Kinesis tasks are killed by KinesisSupervisor if they are stuck during publishing segments or waiting - // for hand off. See KinesisSupervisorIOConfig.completionTimeout. - final SegmentsAndMetadata published = driver.publish( - publisher, - committerSupplier.get(), - sequenceNames.values() - ).get(); - - final SegmentsAndMetadata handedOff; - if (tuningConfig.getHandoffConditionTimeout() == 0) { - handedOff = driver.registerHandoff(published) - .get(); - } else { - handedOff = driver.registerHandoff(published) - .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); - } - - if (handedOff == null) { - throw new ISE("Transaction failure publishing segments, aborting"); - } else { - log.info( - "Published segments[%s] with metadata[%s].", - Joiner.on(", ").join( - Iterables.transform( - handedOff.getSegments(), - DataSegment::getIdentifier - ) - ), - handedOff.getCommitMetadata() - ); - } - } - catch (InterruptedException | RejectedExecutionException e) { - final String errorMsg = Throwables.getStackTraceAsString(e); - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(errorMsg)); - - if (appenderator != null) { - appenderator.closeNow(); - } - - // handle the InterruptedException that gets wrapped in a RejectedExecutionException - if (e instanceof RejectedExecutionException - && (e.getCause() == null || !(e.getCause() instanceof InterruptedException))) { - throw e; - } - - // if we were interrupted because we were asked to stop, handle the exception and return success, else rethrow - if (!stopRequested) { - Thread.currentThread().interrupt(); - throw e; - } - - log.info("The task was asked to stop before completing"); - } - finally { - if (chatHandlerProvider.isPresent()) { - chatHandlerProvider.get().unregister(getId()); - } - } - - toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); - toolbox.getDataSegmentServerAnnouncer().unannounce(); - - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(null)); - return success(); - } - - private RecordSupplier getRecordSupplier() + protected RecordSupplier getRecordSupplier() { int fetchThreads = tuningConfig.getFetchThreads() != null ? tuningConfig.getFetchThreads() @@ -661,699 +88,22 @@ private RecordSupplier getRecordSupplier() tuningConfig.getRecordBufferSize(), tuningConfig.getRecordBufferOfferTimeout(), tuningConfig.getRecordBufferFullWait(), - tuningConfig.getFetchSequenceNumberTimeout() - ); - } - - private Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) - { - final int maxRowsInMemoryPerPartition = (tuningConfig.getMaxRowsInMemory() / - ioConfig.getStartPartitions().getPartitionSequenceNumberMap().size()); - return Appenderators.createRealtime( - dataSchema, - tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), - metrics, - toolbox.getSegmentPusher(), - toolbox.getObjectMapper(), - toolbox.getIndexIO(), - toolbox.getIndexMergerV9(), - toolbox.getQueryRunnerFactoryConglomerate(), - toolbox.getSegmentAnnouncer(), - toolbox.getEmitter(), - toolbox.getQueryExecutorService(), - toolbox.getCache(), - toolbox.getCacheConfig(), - toolbox.getCachePopulatorStats() - ); - } - - private StreamAppenderatorDriver newDriver( - final Appenderator appenderator, - final TaskToolbox toolbox, - final FireDepartmentMetrics metrics - ) - { - return new StreamAppenderatorDriver( - appenderator, - new ActionBasedSegmentAllocator( - toolbox.getTaskActionClient(), - dataSchema, - (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( - schema.getDataSource(), - row.getTimestamp(), - schema.getGranularitySpec().getQueryGranularity(), - schema.getGranularitySpec().getSegmentGranularity(), - sequenceName, - previousSegmentId, - skipSegmentLineageCheck - ) - ), - toolbox.getSegmentHandoffNotifierFactory(), - new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), - toolbox.getDataSegmentKiller(), - toolbox.getObjectMapper(), - metrics - ); - } - - private Set assignPartitions(RecordSupplier recordSupplier, String topic) - { - // Initialize consumer assignment. - final Set assignment = new HashSet<>(); - for (Map.Entry entry : lastOffsets.entrySet()) { - final String endOffset = endOffsets.get(entry.getKey()); - if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(endOffset) - || SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) - || KinesisSequenceNumber.of(entry.getValue()).compareTo(KinesisSequenceNumber.of(endOffset)) < 0) { - assignment.add(entry.getKey()); - } else if (entry.getValue().equals(endOffset)) { - log.info("Finished reading partition[%s].", entry.getKey()); - } else { - throw new ISE( - "WTF?! Cannot start from offset[%s] > endOffset[%s]", - entry.getValue(), - endOffset - ); - } - } - - assignPartitions(recordSupplier, topic, assignment); - - return assignment; - } - - private void seekToStartingRecords( - RecordSupplier recordSupplier, - String topic, - Set assignment, - TaskToolbox toolbox - ) - { - // Seek to starting offsets. - for (final String partition : assignment) { - final String offset = lastOffsets.get(partition); - final StreamPartition streamPartition = StreamPartition.of(topic, partition); - - if (!tuningConfig.isSkipSequenceNumberAvailabilityCheck()) { - try { - String earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); - if (earliestSequenceNumber == null - || KinesisSequenceNumber.of(earliestSequenceNumber).compareTo(KinesisSequenceNumber.of(offset)) > 0) { - if (tuningConfig.isResetOffsetAutomatically()) { - log.info("Attempting to reset offsets automatically for all partitions"); - try { - sendResetRequestAndWait( - assignment.stream() - .collect(Collectors.toMap((x) -> new StreamPartition<>(topic, x), lastOffsets::get)), - toolbox - ); - } - catch (IOException e) { - throw new ISE(e, "Exception while attempting to automatically reset offsets"); - } - } else { - throw new ISE( - "Starting sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) and resetOffsetAutomatically is not enabled", - offset, - partition, - earliestSequenceNumber - ); - } - } - } - catch (TimeoutException e) { - throw new ISE(e, "Timeout while fetching earliest sequence number for partition [%s]", partition); - } - } - - log.info("Seeking partition[%s] to sequenceNumber[%s].", partition, offset); - - // We will seek to and start reading from the last offset that we read on the previous run so that we can confirm - // that the sequenceNumbers match, but we will discard the event instead of indexing it so we don't read it twice. - recordSupplier.seek(streamPartition, offset); - } - } - - /** - * Checks if the pauseRequested flag was set and if so blocks: - * a) if pauseMillis == PAUSE_FOREVER, until pauseRequested is cleared - * b) if pauseMillis != PAUSE_FOREVER, until pauseMillis elapses -or- pauseRequested is cleared - *

- * If pauseMillis is changed while paused, the new pause timeout will be applied. This allows adjustment of the - * pause timeout (making a timed pause into an indefinite pause and vice versa is valid) without having to resume - * and ensures that the loop continues to stay paused without ingesting any new events. You will need to signal - * shouldResume after adjusting pauseMillis for the new value to take effect. - *

- * Sets paused = true and signals paused so callers can be notified when the pause command has been accepted. - *

- * Additionally, pauses if all partitions assignments have been read and pauseAfterRead flag is set. - * - * @return true if a pause request was handled, false otherwise - */ - private boolean possiblyPause(Set assignment) throws InterruptedException - { - pauseLock.lockInterruptibly(); - try { - /* - if (ioConfig.isPauseAfterRead() && assignment.isEmpty()) { - pauseMillis = PAUSE_FOREVER; - pauseRequested = true; - } - */ - if (pauseRequested) { - status = Status.PAUSED; - hasPaused.signalAll(); - - while (pauseRequested) { - log.info("Pausing ingestion until resumed"); - shouldResume.await(); - } - - status = Status.READING; - shouldResume.signalAll(); - log.info("Ingestion loop resumed"); - return true; - } - } - finally { - pauseLock.unlock(); - } - - return false; - } - - private boolean withinMinMaxRecordTime(final InputRow row) - { - final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent() - && ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp()); - - final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent() - && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp()); - - if (!Intervals.ETERNITY.contains(row.getTimestamp())) { - final String errorMsg = StringUtils.format( - "Encountered row with timestamp that cannot be represented as a long: [%s]", - row - ); - throw new ParseException(errorMsg); - } - - if (log.isDebugEnabled()) { - if (beforeMinimumMessageTime) { - log.debug( - "CurrentTimeStamp[%s] is before MinimumMessageTime[%s]", - row.getTimestamp(), - ioConfig.getMinimumMessageTime().get() - ); - } else if (afterMaximumMessageTime) { - log.debug( - "CurrentTimeStamp[%s] is after MaximumMessageTime[%s]", - row.getTimestamp(), - ioConfig.getMaximumMessageTime().get() - ); - } - } - - return !beforeMinimumMessageTime && !afterMaximumMessageTime; - } - - private void handleParseException(ParseException pe, OrderedPartitionableRecord record) - { - if (pe.isFromPartiallyValidRow()) { - rowIngestionMeters.incrementProcessedWithError(); - } else { - rowIngestionMeters.incrementUnparseable(); - } - - if (tuningConfig.isLogParseExceptions()) { - log.error( - pe, - "Encountered parse exception on row from partition[%s] sequenceNumber[%s]", - record.getPartitionId(), - record.getSequenceNumber() - ); - } - - if (savedParseExceptions != null) { - savedParseExceptions.add(pe); - } - - if (rowIngestionMeters.getUnparseable() + rowIngestionMeters.getProcessedWithError() - > tuningConfig.getMaxParseExceptions()) { - log.error("Max parse exceptions exceeded, terminating task..."); - throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); - } - } - - private static void assignPartitions( - final RecordSupplier recordSupplier, - final String topic, - final Set partitions - ) - { - recordSupplier.assign(partitions.stream().map(x -> StreamPartition.of(topic, x)).collect(Collectors.toSet())); - } - - private Map getTaskCompletionReports(@Nullable String errorMsg) - { - return TaskReport.buildTaskReports( - new IngestionStatsAndErrorsTaskReport( - getId(), - new IngestionStatsAndErrorsTaskReportData( - ingestionState, - getTaskCompletionUnparseableEvents(), - getTaskCompletionRowStats(), - errorMsg - ) - ) - ); - } - - private void sendResetRequestAndWait( - Map, String> outOfRangePartitions, - TaskToolbox taskToolbox - ) - throws IOException - { - Map partitionOffsetMap = outOfRangePartitions - .entrySet().stream().collect(Collectors.toMap(x -> x.getKey().getPartitionId(), Map.Entry::getValue)); - - boolean result = taskToolbox - .getTaskActionClient() - .submit( - new ResetDataSourceMetadataAction( - getDataSource(), - new KinesisDataSourceMetadata( - new SeekableStreamPartitions<>( - ioConfig.getStartPartitions().getStream(), - partitionOffsetMap - ) - ) - ) - ); - - if (result) { - log.makeAlert("Resetting Kinesis offsets for datasource [%s]", getDataSource()) - .addData("partitions", partitionOffsetMap.keySet()) - .emit(); - // wait for being killed by supervisor - try { - pause(-1); - } - catch (InterruptedException e) { - throw new RuntimeException("Got interrupted while pausing task"); - } - } else { - log.makeAlert("Failed to send reset request for partitions [%s]", partitionOffsetMap.keySet()).emit(); - } - } - - private Map getTaskCompletionUnparseableEvents() - { - Map unparseableEventsMap = new HashMap<>(); - List buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions( - savedParseExceptions - ); - if (buildSegmentsParseExceptionMessages != null) { - unparseableEventsMap.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegmentsParseExceptionMessages); - } - return unparseableEventsMap; - } - - private Map getTaskCompletionRowStats() - { - Map metrics = new HashMap<>(); - metrics.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getTotals() - ); - return metrics; - } - - public Response pause(final long timeout) throws InterruptedException - { - if (!(status == Status.PAUSED || status == Status.READING)) { - return Response.status(Response.Status.BAD_REQUEST) - .entity(StringUtils.format("Can't pause, task is not in a pausable state (state: [%s])", status)) - .build(); - } - - pauseLock.lockInterruptibly(); - try { - pauseMillis = timeout <= 0 ? PAUSE_FOREVER : timeout; - pauseRequested = true; - - pollRetryLock.lockInterruptibly(); - try { - isAwaitingRetry.signalAll(); - } - finally { - pollRetryLock.unlock(); - } - - if (isPaused()) { - shouldResume.signalAll(); // kick the monitor so it re-awaits with the new pauseMillis - } - - long nanos = TimeUnit.SECONDS.toNanos(2); - while (!isPaused()) { - if (nanos <= 0L) { - return Response.status(Response.Status.ACCEPTED) - .entity("Request accepted but task has not yet paused") - .build(); - } - nanos = hasPaused.awaitNanos(nanos); - } - } - finally { - pauseLock.unlock(); - } - - try { - return Response.ok().entity(mapper.writeValueAsString(getCurrentOffsets())).build(); - } - catch (JsonProcessingException e) { - throw Throwables.propagate(e); - } - } - - private boolean isPaused() - { - return status == Status.PAUSED; - } - - public Map getCurrentOffsets() - { - return lastOffsets; - } - - @Override - public boolean canRestore() - { - return true; - } - - @Override - public QueryRunner getQueryRunner(Query query) - { - if (appenderator == null) { - // Not yet initialized, no data yet, just return a noop runner. - return new NoopQueryRunner<>(); - } - - return (query1, responseContext) -> query1.run(appenderator, responseContext); - } - - @POST - @Path("/stop") - public Response stop(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.WRITE); - stopGracefully(); - return Response.status(Response.Status.OK).build(); - } - - /** - * Authorizes action to be performed on this task's datasource - * - * @return authorization result - */ - private Access authorizationCheck(final HttpServletRequest req, Action action) - { - ResourceAction resourceAction = new ResourceAction( - new Resource(dataSchema.getDataSource(), ResourceType.DATASOURCE), - action + tuningConfig.getFetchSequenceNumberTimeout(), + tuningConfig.getMaxRecordsPerPoll() ); - - Access access = AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); - if (!access.isAllowed()) { - throw new ForbiddenException(access.toString()); - } - - return access; } @Override - public void stopGracefully() - { - log.info("Stopping gracefully (status: [%s])", status); - stopRequested = true; - - synchronized (statusLock) { - if (status == Status.PUBLISHING) { - runThread.interrupt(); - return; - } - } - - try { - if (pauseLock.tryLock(LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { - try { - if (pauseRequested) { - pauseRequested = false; - shouldResume.signalAll(); - } - } - finally { - pauseLock.unlock(); - } - } else { - log.warn("While stopping: failed to acquire pauseLock before timeout, interrupting run thread"); - runThread.interrupt(); - return; - } - - if (pollRetryLock.tryLock(LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { - try { - isAwaitingRetry.signalAll(); - } - finally { - pollRetryLock.unlock(); - } - } else { - log.warn("While stopping: failed to acquire pollRetryLock before timeout, interrupting run thread"); - runThread.interrupt(); - } - } - catch (Exception e) { - Throwables.propagate(e); - } - } - - @GET - @Path("/status") - @Produces(MediaType.APPLICATION_JSON) - public Status getStatusHTTP(@Context final HttpServletRequest req) + protected SeekableStreamIndexTaskRunner createTaskRunner() { - authorizationCheck(req, Action.READ); - return status; - } - - public Status getStatus() - { - return status; - } - - @GET - @Path("/offsets/current") - @Produces(MediaType.APPLICATION_JSON) - public Map getCurrentOffsets(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.READ); - return getCurrentOffsets(); - } - - @GET - @Path("/offsets/end") - @Produces(MediaType.APPLICATION_JSON) - public Map getEndOffsetsHTTP(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.READ); - return getEndOffsets(); - } - - public Map getEndOffsets() - { - return endOffsets; - } - - @POST - @Path("/offsets/end") - @Consumes(MediaType.APPLICATION_JSON) - @Produces(MediaType.APPLICATION_JSON) - public Response setEndOffsetsHTTP( - Map offsets, - @QueryParam("resume") @DefaultValue("false") final boolean resume, - @Context final HttpServletRequest req - ) throws InterruptedException - { - authorizationCheck(req, Action.WRITE); - return setEndOffsets(offsets, resume); - } - - public Response setEndOffsets(Map offsets, final boolean resume) throws InterruptedException - { - if (offsets == null) { - return Response.status(Response.Status.BAD_REQUEST) - .entity("Request body must contain a map of { partition:endOffset }") - .build(); - } else if (!endOffsets.keySet().containsAll(offsets.keySet())) { - return Response.status(Response.Status.BAD_REQUEST) - .entity( - StringUtils.format( - "Request contains partitions not being handled by this task, my partitions: %s", - endOffsets.keySet() - ) - ) - .build(); - } - - pauseLock.lockInterruptibly(); - try { - if (!isPaused()) { - return Response.status(Response.Status.BAD_REQUEST) - .entity("Task must be paused before changing the end offsets") - .build(); - } - - for (Map.Entry entry : offsets.entrySet()) { - if (entry.getValue().compareTo(lastOffsets.get(entry.getKey())) < 0) { - return Response.status(Response.Status.BAD_REQUEST) - .entity( - StringUtils.format( - "End offset must be >= current offset for partition [%s] (current: %s)", - entry.getKey(), - lastOffsets.get(entry.getKey()) - ) - ) - .build(); - } - } - - endOffsets.putAll(offsets); - log.info("endOffsets changed to %s", endOffsets); - } - finally { - pauseLock.unlock(); - } - - resume(); - - return Response.ok(endOffsets).build(); - } - - public void resume() throws InterruptedException - { - pauseLock.lockInterruptibly(); - try { - pauseRequested = false; - shouldResume.signalAll(); - - long nanos = TimeUnit.SECONDS.toNanos(5); - while (isPaused()) { - if (nanos <= 0L) { - throw new RuntimeException("Resume command was not accepted within 5 seconds"); - } - nanos = shouldResume.awaitNanos(nanos); - } - } - finally { - pauseLock.unlock(); - } - } - - /** - * Signals the ingestion loop to pause. - * - * @param timeout how long to pause for before resuming in milliseconds, <= 0 means indefinitely - * - * @return one of the following Responses: 400 Bad Request if the task has started publishing; 202 Accepted if the - * method has timed out and returned before the task has paused; 200 OK with a map of the current partition offsets - * in the response body if the task successfully paused - */ - @POST - @Path("/pause") - @Produces(MediaType.APPLICATION_JSON) - public Response pauseHTTP( - @QueryParam("timeout") @DefaultValue("0") final long timeout, @Context final HttpServletRequest req - ) throws InterruptedException - { - authorizationCheck(req, Action.WRITE); - return pause(timeout); - } - - @POST - @Path("/resume") - public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException - { - authorizationCheck(req, Action.WRITE); - resume(); - return Response.status(Response.Status.OK).build(); - } - - @GET - @Path("/time/start") - @Produces(MediaType.APPLICATION_JSON) - public DateTime getStartTime(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.READ); - return startTime; - } - - @GET - @Path("/rowStats") - @Produces(MediaType.APPLICATION_JSON) - public Response getRowStats( - @Context final HttpServletRequest req - ) - { - authorizationCheck(req, Action.READ); - Map returnMap = new HashMap<>(); - Map totalsMap = new HashMap<>(); - Map averagesMap = new HashMap<>(); - - totalsMap.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getTotals() - ); - averagesMap.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getMovingAverages() + return new KinesisIndexTaskRunner( + this, + parser, + authorizerMapper, + chatHandlerProvider, + savedParseExceptions, + rowIngestionMetersFactory ); - - returnMap.put("movingAverages", averagesMap); - returnMap.put("totals", totalsMap); - return Response.ok(returnMap).build(); - } - - @GET - @Path("/unparseableEvents") - @Produces(MediaType.APPLICATION_JSON) - public Response getUnparseableEvents( - @Context final HttpServletRequest req - ) - { - authorizationCheck(req, Action.READ); - List events = IndexTaskUtils.getMessagesFromSavedParseExceptions(savedParseExceptions); - return Response.ok(events).build(); - } - - @VisibleForTesting - RowIngestionMeters getRowIngestionMeters() - { - return rowIngestionMeters; - } - - @VisibleForTesting - Appenderator getAppenderator() - { - return appenderator; - } - - @VisibleForTesting - FireDepartmentMetrics getFireDepartmentMetrics() - { - return fireDepartmentMetrics; } @Override diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java new file mode 100644 index 000000000000..f9ba0b3998e7 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.utils.CircularBuffer; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.CopyOnWriteArrayList; + +public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner +{ + private static final EmittingLogger log = new EmittingLogger(KinesisIndexTaskRunner.class); + private static final long POLL_TIMEOUT = 100; + private volatile CopyOnWriteArrayList sequences; + + + public KinesisIndexTaskRunner( + KinesisIndexTask task, + InputRowParser parser, + AuthorizerMapper authorizerMapper, + Optional chatHandlerProvider, + CircularBuffer savedParseExceptions, + RowIngestionMetersFactory rowIngestionMetersFactory + ) + { + super( + task, + parser, + authorizerMapper, + chatHandlerProvider, + savedParseExceptions, + rowIngestionMetersFactory, + false + ); + + + } + + + @Override + protected String getNextSequenceNumber( + RecordSupplier recordSupplier, StreamPartition partition, String sequenceNumber + ) + { + String sequence = recordSupplier.getPosition(partition); + return sequence == null ? sequenceNumber : sequence; + } + + @Nonnull + @Override + protected List> getRecords( + RecordSupplier recordSupplier, TaskToolbox toolbox + ) + { + return recordSupplier.poll(POLL_TIMEOUT); + } + + @Override + protected SeekableStreamPartitions createSeekableStreamPartitions( + ObjectMapper mapper, + Object obeject + ) + { + return mapper.convertValue(obeject, mapper.getTypeFactory().constructParametrizedType( + SeekableStreamPartitions.class, + SeekableStreamPartitions.class, + String.class, + String.class + )); + } + + @Override + protected OrderedSequenceNumber createSequencenNumber(String sequenceNumber) + { + return KinesisSequenceNumber.of(sequenceNumber); + } + + @Override + protected SeekableStreamDataSourceMetadata createDataSourceMetadata( + SeekableStreamPartitions partitions + ) + { + return new KinesisDataSourceMetadata(partitions); + } + + @Override + protected OrderedSequenceNumber createSequenceNumber(String sequenceNumber) + { + return KinesisSequenceNumber.of(sequenceNumber); + } + + @Nullable + @Override + protected TreeMap> getCheckPointsFromContext( + TaskToolbox toolbox, + SeekableStreamIndexTask task + ) throws IOException + { + final String checkpointsString = task.getContextValue("checkpoints"); + if (checkpointsString != null) { + log.info("Checkpoints [%s]", checkpointsString); + return toolbox.getObjectMapper().readValue( + checkpointsString, + new TypeReference>>() + { + } + ); + } else { + return null; + } + } + + +} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index e29e20db3f28..c9f7a23f4770 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -34,6 +34,7 @@ import com.amazonaws.services.kinesis.model.ShardIteratorType; import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder; import com.amazonaws.util.AwsHostNameUtils; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Queues; import org.apache.druid.common.aws.AWSCredentialsUtils; @@ -41,16 +42,19 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -60,9 +64,9 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; public class KinesisRecordSupplier implements RecordSupplier @@ -70,29 +74,25 @@ public class KinesisRecordSupplier implements RecordSupplier private static final EmittingLogger log = new EmittingLogger(KinesisRecordSupplier.class); private static final long PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS = 3000; private static final long EXCEPTION_RETRY_DELAY_MS = 10000; - private final String endpoint; - private static final int DEFAULT_POLL_RECORD_LIMIT = 20; private class PartitionResource { private final StreamPartition streamPartition; private final AmazonKinesis kinesisProxy; - private final ScheduledExecutorService scheduledExec; private final Object startLock = new Object(); + private String currIterator; // tracks current position private volatile String shardIterator; private volatile boolean started; private volatile boolean stopRequested; public PartitionResource( StreamPartition streamPartition, - AmazonKinesis kinesisProxy, - ScheduledExecutorService scheduledExec + AmazonKinesis kinesisProxy ) { this.streamPartition = streamPartition; this.kinesisProxy = kinesisProxy; - this.scheduledExec = scheduledExec; } public void start() @@ -122,13 +122,14 @@ public void stop() streamPartition.getStream(), streamPartition.getPartitionId() ); - stopRequested = true; } + private Runnable getRecordRunnable() { return () -> { + if (stopRequested) { started = false; stopRequested = false; @@ -137,6 +138,7 @@ private Runnable getRecordRunnable() return; } + try { if (shardIterator == null) { @@ -220,13 +222,23 @@ private Runnable getRecordRunnable() log.error(e, "getRecordRunnable exception, retrying in [%,dms]", EXCEPTION_RETRY_DELAY_MS); rescheduleRunnable(EXCEPTION_RETRY_DELAY_MS); } + }; } private void rescheduleRunnable(long delayMillis) { if (started && !stopRequested) { - scheduledExec.schedule(getRecordRunnable(), delayMillis, TimeUnit.MILLISECONDS); + try { + scheduledExec.schedule(getRecordRunnable(), delayMillis, TimeUnit.MILLISECONDS); + } + catch (RejectedExecutionException e) { + log.info( + "Rejecting fetch records runnable submission, worker for partition[%s] is not in a valid state", + streamPartition.getPartitionId() + ); + + } } else { log.info("Worker for partition[%s] has been stopped", streamPartition.getPartitionId()); } @@ -239,13 +251,16 @@ private void rescheduleRunnable(long delayMillis) private final int recordBufferOfferTimeout; private final int recordBufferFullWait; private final int fetchSequenceNumberTimeout; + private final int maxRecordsPerPoll; + private final int fetchThreads; + private final int recordBufferSize; private final AmazonKinesisClientBuilder kinesisBuilder; - private final ScheduledExecutorService scheduledExec; + private ScheduledExecutorService scheduledExec; private final Map kinesisProxies = new ConcurrentHashMap<>(); private final Map, PartitionResource> partitionResources = new ConcurrentHashMap<>(); - private final BlockingQueue> records; + private BlockingQueue> records; private volatile boolean checkPartitionsStarted = false; private volatile boolean closed = false; @@ -263,7 +278,8 @@ public KinesisRecordSupplier( int recordBufferSize, int recordBufferOfferTimeout, int recordBufferFullWait, - int fetchSequenceNumberTimeout + int fetchSequenceNumberTimeout, + int maxRecordsPerPoll ) { this.recordsPerFetch = recordsPerFetch; @@ -272,7 +288,9 @@ public KinesisRecordSupplier( this.recordBufferOfferTimeout = recordBufferOfferTimeout; this.recordBufferFullWait = recordBufferFullWait; this.fetchSequenceNumberTimeout = fetchSequenceNumberTimeout; - this.endpoint = endpoint; + this.maxRecordsPerPoll = maxRecordsPerPoll; + this.fetchThreads = fetchThreads; + this.recordBufferSize = recordBufferSize; AWSCredentialsProvider awsCredentialsProvider = AWSCredentialsUtils.defaultAWSCredentialsProviderChain( new ConstructibleAWSCredentialsConfig(awsAccessKeyId, awsSecretAccessKey) @@ -304,8 +322,6 @@ public KinesisRecordSupplier( ) )); - records = new LinkedBlockingQueue<>(recordBufferSize); - log.info( "Creating fetch thread pool of size [%d] (Runtime.availableProcessors=%d)", fetchThreads, @@ -316,8 +332,11 @@ public KinesisRecordSupplier( fetchThreads, Execs.makeThreadFactory("KinesisRecordSupplier-Worker-%d") ); + + records = new LinkedBlockingQueue<>(recordBufferSize); } + @Override public void assign(Set> collection) { @@ -326,7 +345,7 @@ public void assign(Set> collection) collection.forEach( streamPartition -> partitionResources.putIfAbsent( streamPartition, - new PartitionResource(streamPartition, getKinesisProxy(streamPartition.getStream()), scheduledExec) + new PartitionResource(streamPartition, getKinesisProxy(streamPartition.getStream())) ) ); @@ -338,34 +357,30 @@ public void assign(Set> collection) entry.getValue().stop(); } } + } @Override public void seek(StreamPartition partition, String sequenceNumber) { checkIfClosed(); + filterBufferAndResetFetchRunnable(ImmutableSet.of(partition)); seekInternal(partition, sequenceNumber, ShardIteratorType.AT_SEQUENCE_NUMBER); } - @Override - public void seekAfter(StreamPartition partition, String sequenceNumber) - { - checkIfClosed(); - seekInternal(partition, sequenceNumber, ShardIteratorType.AFTER_SEQUENCE_NUMBER); - } - @Override public void seekToEarliest(Set> partitions) { checkIfClosed(); + filterBufferAndResetFetchRunnable(partitions); partitions.forEach(partition -> seekInternal(partition, null, ShardIteratorType.TRIM_HORIZON)); - } @Override public void seekToLatest(Set> partitions) { checkIfClosed(); + filterBufferAndResetFetchRunnable(partitions); partitions.forEach(partition -> seekInternal(partition, null, ShardIteratorType.LATEST)); } @@ -390,39 +405,81 @@ public List> poll(long timeout) Queues.drain( records, polledRecords, - Math.max(records.size(), DEFAULT_POLL_RECORD_LIMIT), + Math.min(Math.max(records.size(), 1), maxRecordsPerPoll), timeout, TimeUnit.MILLISECONDS ); - return polledRecords.stream() - .filter(x -> partitionResources.containsKey(x.getStreamPartition())) - .collect(Collectors.toList()); + + polledRecords = polledRecords.stream() + .filter(x -> partitionResources.containsKey(x.getStreamPartition())) + .collect(Collectors.toList()); + + // update currIterator in each PartitionResource + // first, build a map of shardId -> latest record we've polled + // since polledRecords is ordered from earliest to latest, the final ordering of partitionSequenceMap + // is guranteed to be latest + Map> partitionSequenceMap = new HashMap<>(); + polledRecords.forEach(x -> partitionSequenceMap.put(x.getPartitionId(), x)); + + // then get the next shardIterator for each shard and update currIterator + partitionSequenceMap.forEach((shardId, record) -> partitionResources.get(record.getStreamPartition()).currIterator = + record.getSequenceNumber().equals(OrderedPartitionableRecord.END_OF_SHARD_MARKER) ? + null : + getKinesisProxy(shardId) + .getShardIterator( + record.getStream(), + record.getPartitionId(), + ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(), + record.getSequenceNumber() + ) + .getShardIterator()); + + + return polledRecords; } catch (InterruptedException e) { - log.warn(e, "InterruptedException"); + log.warn(e, "Interrupted while polling"); return Collections.emptyList(); } + } @Override - public String getLatestSequenceNumber(StreamPartition partition) throws TimeoutException + public String getLatestSequenceNumber(StreamPartition partition) { checkIfClosed(); return getSequenceNumberInternal(partition, ShardIteratorType.LATEST); } @Override - public String getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException + public String getEarliestSequenceNumber(StreamPartition partition) { checkIfClosed(); return getSequenceNumberInternal(partition, ShardIteratorType.TRIM_HORIZON); } + @Nullable @Override - public String position(StreamPartition partition) + public String getPosition(StreamPartition partition) { checkIfClosed(); - return null; + if (partitionResources.containsKey(partition)) { + String iter = partitionResources.get(partition).currIterator; + if (iter == null) { + log.warn( + "attempting to get position in shard[%s], stream[%s] with null sharditerator, is shard closed or did you forget to seek?", + partition.getPartitionId(), + partition.getStream() + ); + } + return getSequenceNumberInternal(partition, iter); + } else { + throw new IAE( + "attempting to get position in unassigned shard[%s], stream[%s]", + partition.getPartitionId(), + partition.getStream() + ); + } } @Override @@ -482,23 +539,58 @@ private void seekInternal(StreamPartition partition, String sequenceNumb sequenceNumber != null ? sequenceNumber : iteratorEnum.toString() ); - resource.shardIterator = getKinesisProxy(partition.getStream()).getShardIterator( + AmazonKinesis kinesis = getKinesisProxy(partition.getStream()); + + resource.shardIterator = kinesis.getShardIterator( partition.getStream(), partition.getPartitionId(), iteratorEnum.toString(), sequenceNumber ).getShardIterator(); + resource.currIterator = resource.shardIterator; + + resource.start(); + + checkPartitionsStarted = true; + } + + private void filterBufferAndResetFetchRunnable(Set> partitions) + { + scheduledExec.shutdown(); + + try { + if (!scheduledExec.awaitTermination(EXCEPTION_RETRY_DELAY_MS, TimeUnit.MILLISECONDS)) { + scheduledExec.shutdownNow(); + } + } + catch (InterruptedException e) { + log.info(e, "InterruptedException while shutting down"); + } + + scheduledExec = Executors.newScheduledThreadPool( + fetchThreads, + Execs.makeThreadFactory("KinesisRecordSupplier-Worker-%d") + ); + + // filter records in buffer and only retain ones whose partition was not seeked + BlockingQueue> newQ = new LinkedBlockingQueue<>(recordBufferSize); + records + .parallelStream() + .filter(x -> !partitions.contains(x.getStreamPartition())) + .forEachOrdered(newQ::offer); + + records = newQ; + + // restart fetching threads + partitionResources.values().forEach(x -> x.started = false); checkPartitionsStarted = true; } private String getSequenceNumberInternal(StreamPartition partition, ShardIteratorType iteratorEnum) - throws TimeoutException { - long timeoutMillis = System.currentTimeMillis() + fetchSequenceNumberTimeout; AmazonKinesis kinesis = getKinesisProxy(partition.getStream()); String shardIterator = null; - try { shardIterator = kinesis.getShardIterator( partition.getStream(), @@ -510,6 +602,15 @@ private String getSequenceNumberInternal(StreamPartition partition, Shar log.warn("Caught ResourceNotFoundException: %s", e.getMessage()); } + return getSequenceNumberInternal(partition, shardIterator); + } + + private String getSequenceNumberInternal(StreamPartition partition, String shardIterator) + { + long timeoutMillis = System.currentTimeMillis() + fetchSequenceNumberTimeout; + AmazonKinesis kinesis = getKinesisProxy(partition.getStream()); + + while (shardIterator != null && System.currentTimeMillis() < timeoutMillis) { if (closed) { @@ -548,12 +649,15 @@ private String getSequenceNumberInternal(StreamPartition partition, Shar return OrderedPartitionableRecord.END_OF_SHARD_MARKER; } - throw new TimeoutException( - StringUtils.format( - "Timeout while retrieving sequence number for partition[%s]", - partition.getPartitionId() - ) + + // if we reach here, it usually means either the shard has no more records, or records have not been + // added to this shard + log.warn( + "timed out while trying to fetch position for shard[%s], likely no more records in shard", + partition.getPartitionId() ); + return null; + } private void checkIfClosed() @@ -580,4 +684,10 @@ private static byte[] toByteArray(final ByteBuffer buffer) return retVal; } } + + @VisibleForTesting + public int bufferSize() + { + return records.size(); + } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java index 79f9e4ad9707..a33349976503 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.kinesis; +import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; @@ -30,14 +31,23 @@ public class KinesisSequenceNumber extends OrderedSequenceNumber { + // this flag is used to indicate either END_OF_SHARD_MARKER + // or NO_END_SEQUENCE_NUMBER so that they can be properly compared + // with other sequence numbers + private final boolean isMaxSequenceNumber; private final BigInteger intSequence; private KinesisSequenceNumber(@NotNull String sequenceNumber, boolean isExclusive) { super(sequenceNumber, isExclusive); - this.intSequence = OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(sequenceNumber) - ? new BigInteger("-1") - : new BigInteger(sequenceNumber); + if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(sequenceNumber) + || SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(sequenceNumber)) { + isMaxSequenceNumber = true; + this.intSequence = null; + } else { + isMaxSequenceNumber = false; + this.intSequence = new BigInteger(sequenceNumber); + } } public static KinesisSequenceNumber of(String sequenceNumber) @@ -63,7 +73,16 @@ public boolean equals(Object o) @Override public int compareTo(@NotNull OrderedSequenceNumber o) { - return this.intSequence.compareTo(new BigInteger(o.get())); + KinesisSequenceNumber num = (KinesisSequenceNumber) o; + if (isMaxSequenceNumber && num.isMaxSequenceNumber) { + return 0; + } else if (isMaxSequenceNumber) { + return 1; + } else if (num.isMaxSequenceNumber) { + return -1; + } else { + return this.intSequence.compareTo(new BigInteger(o.get())); + } } @Override diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java index c648975150ea..767aec18ebad 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java @@ -37,7 +37,8 @@ public class KinesisTuningConfig extends SeekableStreamTuningConfig private static final int DEFAULT_RECORD_BUFFER_SIZE = 10000; private static final int DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT = 5000; private static final int DEFAULT_RECORD_BUFFER_FULL_WAIT = 5000; - private static final int DEFAULT_FETCH_SEQUENCE_NUMBER_TIMEOUT = 60000; + private static final int DEFAULT_FETCH_SEQUENCE_NUMBER_TIMEOUT = 10000; + private static final int DEFAULT_MAX_RECORDS_PER_POLL = 5; private final int recordBufferSize; @@ -45,12 +46,14 @@ public class KinesisTuningConfig extends SeekableStreamTuningConfig private final int recordBufferFullWait; private final int fetchSequenceNumberTimeout; private final Integer fetchThreads; + private final int maxRecordsPerPoll; @JsonCreator public KinesisTuningConfig( @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, + @JsonProperty("maxTotalRows") Long maxTotalRows, @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @JsonProperty("maxPendingPersists") Integer maxPendingPersists, @@ -68,14 +71,16 @@ public KinesisTuningConfig( @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, - @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, + @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll, + @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod ) { super( maxRowsInMemory, maxBytesInMemory, maxRowsPerSegment, - null, + maxTotalRows, intermediatePersistPeriod, basePersistDirectory, maxPendingPersists, @@ -86,7 +91,7 @@ public KinesisTuningConfig( resetOffsetAutomatically, skipSequenceNumberAvailabilityCheck, segmentWriteOutMediumFactory, - null, + intermediateHandoffPeriod, logParseExceptions, maxParseExceptions, maxSavedParseExceptions @@ -99,6 +104,7 @@ public KinesisTuningConfig( this.fetchSequenceNumberTimeout = fetchSequenceNumberTimeout == null ? DEFAULT_FETCH_SEQUENCE_NUMBER_TIMEOUT : fetchSequenceNumberTimeout; this.fetchThreads = fetchThreads; // we handle this being null later + this.maxRecordsPerPoll = maxRecordsPerPoll == null ? DEFAULT_MAX_RECORDS_PER_POLL : maxRecordsPerPoll; Preconditions.checkArgument( !(super.isResetOffsetAutomatically() && super.isSkipSequenceNumberAvailabilityCheck()), @@ -113,6 +119,7 @@ public KinesisTuningConfig copyOf() getMaxRowsInMemory(), getMaxBytesInMemory(), getMaxRowsPerSegment(), + getMaxTotalRows(), getIntermediatePersistPeriod(), getBasePersistDirectory(), 0, @@ -130,7 +137,9 @@ public KinesisTuningConfig copyOf() getSegmentWriteOutMediumFactory(), isLogParseExceptions(), getMaxParseExceptions(), - getMaxSavedParseExceptions() + getMaxSavedParseExceptions(), + getMaxRecordsPerPoll(), + getIntermediateHandoffPeriod() ); } @@ -164,6 +173,12 @@ public Integer getFetchThreads() return fetchThreads; } + @JsonProperty + public int getMaxRecordsPerPoll() + { + return maxRecordsPerPoll; + } + @Override public KinesisTuningConfig withBasePersistDirectory(File dir) { @@ -171,6 +186,7 @@ public KinesisTuningConfig withBasePersistDirectory(File dir) getMaxRowsInMemory(), getMaxBytesInMemory(), getMaxRowsPerSegment(), + getMaxTotalRows(), getIntermediatePersistPeriod(), dir, 0, @@ -188,7 +204,9 @@ public KinesisTuningConfig withBasePersistDirectory(File dir) getSegmentWriteOutMediumFactory(), isLogParseExceptions(), getMaxParseExceptions(), - getMaxSavedParseExceptions() + getMaxSavedParseExceptions(), + getMaxRecordsPerPoll(), + getIntermediateHandoffPeriod() ); } @@ -277,6 +295,7 @@ public String toString() ", logParseExceptions=" + isLogParseExceptions() + ", maxParseExceptions=" + getMaxParseExceptions() + ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + + ", maxRecordsPerPoll=" + maxRecordsPerPoll + '}'; } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 8812a36760fd..52334af7c17d 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -52,8 +52,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.joda.time.DateTime; -import javax.annotation.Nullable; -import javax.validation.constraints.NotNull; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -65,8 +63,8 @@ /** * Supervisor responsible for managing the KinesisIndexTask for a single dataSource. At a high level, the class accepts a - * {@link KinesisSupervisorSpec} which includes the Kafka topic and configuration as well as an ingestion spec which will - * be used to generate the indexing tasks. The run loop periodically refreshes its view of the Kafka topic's partitions + * {@link KinesisSupervisorSpec} which includes the Kinesis stream and configuration as well as an ingestion spec which will + * be used to generate the indexing tasks. The run loop periodically refreshes its view of the Kinesis stream's partitions * and the list of running indexing tasks and ensures that all partitions are being read from and that there are enough * tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of * Kinesis sequences. @@ -97,26 +95,12 @@ public KinesisSupervisor( mapper, spec, rowIngestionMetersFactory, - true, - false + true ); this.spec = spec; } - - @Override - public void checkpoint( - @Nullable Integer taskGroupId, - @Deprecated String baseSequenceName, - DataSourceMetadata previousCheckPoint, - DataSourceMetadata currentCheckPoint - ) - { - // not supported right now - throw new UnsupportedOperationException("kinesis supervisor does not yet support checkpoints"); - } - @Override protected SeekableStreamIOConfig createIoConfig( int groupId, @@ -197,7 +181,8 @@ protected RecordSupplier setupRecordSupplier() taskTuningConfig.getRecordBufferSize(), taskTuningConfig.getRecordBufferOfferTimeout(), taskTuningConfig.getRecordBufferFullWait(), - taskTuningConfig.getFetchSequenceNumberTimeout() + taskTuningConfig.getFetchSequenceNumberTimeout(), + taskTuningConfig.getMaxRecordsPerPoll() ); } @@ -326,15 +311,6 @@ protected int getNoticesQueueSize() return super.getNoticesQueueSize(); } - @Override - protected boolean checkSequenceAvailability(@NotNull String partition, @NotNull String sequenceFromMetadata) - throws TimeoutException - { - String earliestSequence = super.getOffsetFromStreamForPartition(partition, true); - return earliestSequence != null - && KinesisSequenceNumber.of(earliestSequence).compareTo(KinesisSequenceNumber.of(sequenceFromMetadata)) <= 0; - } - @Override protected String getNotSetMarker() { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java index 1b2729dd5e68..dcfb7a083693 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java @@ -169,11 +169,4 @@ public String toString() ", deaggregate=" + deaggregate + '}'; } - - @JsonProperty - public String getStream() - { - return getId(); - } - } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java index ae872215f44c..06da8d024ebf 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -86,6 +86,9 @@ public KinesisSupervisorSpec( null, null, null, + null, + null, + null, null ), ioConfig, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 7abe07cf02ed..8bfa7762ffec 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -42,6 +42,7 @@ public KinesisSupervisorTuningConfig( @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, + @JsonProperty("maxTotalRows") Long maxTotalRows, @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @JsonProperty("maxPendingPersists") Integer maxPendingPersists, @@ -64,13 +65,16 @@ public KinesisSupervisorTuningConfig( @JsonProperty("fetchThreads") Integer fetchThreads, @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, - @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, + @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll, + @JsonProperty("intermediateHandoffPeriod") Period intermediateHandoffPeriod ) { super( maxRowsInMemory, maxBytesInMemory, maxRowsPerSegment, + maxTotalRows, intermediatePersistPeriod, basePersistDirectory, maxPendingPersists, @@ -88,7 +92,9 @@ public KinesisSupervisorTuningConfig( segmentWriteOutMediumFactory, logParseExceptions, maxParseExceptions, - maxSavedParseExceptions + maxSavedParseExceptions, + maxRecordsPerPoll, + intermediateHandoffPeriod ); this.workerThreads = workerThreads; diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java index 70862cb7fae3..f1e3b0fca65e 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadataTest.java @@ -125,8 +125,8 @@ public void testMinus() ); } - private static KinesisDataSourceMetadata KM(String stream, Map offsets) + private static KinesisDataSourceMetadata KM(String stream, Map sequences) { - return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, offsets)); + return new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, sequences)); } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index c3aabba34ac5..e20d2944074d 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -89,6 +89,7 @@ public void testSerdeWithDefaults() throws Exception Assert.assertNull(config.getAwsAssumedRoleArn()); Assert.assertNull(config.getAwsExternalId()); Assert.assertFalse(config.isDeaggregate()); + Assert.assertTrue(config.isSkipOffsetGaps()); } @Test @@ -149,6 +150,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals("role", config.getAwsAssumedRoleArn()); Assert.assertEquals("awsexternalid", config.getAwsExternalId()); Assert.assertTrue(config.isDeaggregate()); + Assert.assertTrue(config.isSkipOffsetGaps()); } @Test diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java index 64b00c6e9a18..82959767750c 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java @@ -30,7 +30,7 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IndexTaskClient; import org.apache.druid.indexing.common.TaskInfoProvider; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; @@ -150,7 +150,7 @@ public void testNoTaskLocation() throws IOException Assert.assertFalse(client.resume(TEST_ID)); Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID)); Assert.assertEquals(ImmutableMap.of(), client.pause(TEST_ID)); - Assert.assertEquals(SeekableStreamIndexTask.Status.NOT_STARTED, client.getStatus(TEST_ID)); + Assert.assertEquals(Status.NOT_STARTED, client.getStatus(TEST_ID)); Assert.assertNull(client.getStartTime(TEST_ID)); Assert.assertEquals(ImmutableMap.of(), client.getCurrentOffsets(TEST_ID, true)); Assert.assertEquals(ImmutableMap.of(), client.getEndOffsets(TEST_ID)); @@ -421,7 +421,7 @@ public void testGetStartTime() throws Exception @Test public void testGetStatus() throws Exception { - SeekableStreamIndexTask.Status status = SeekableStreamIndexTask.Status.READING; + Status status = Status.READING; Capture captured = Capture.newInstance(); EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); @@ -435,7 +435,7 @@ public void testGetStatus() throws Exception ); replayAll(); - SeekableStreamIndexTask.Status results = client.getStatus(TEST_ID); + Status results = client.getStatus(TEST_ID); verifyAll(); Request request = captured.getValue(); @@ -806,13 +806,13 @@ public void testGetStatusAsync() throws Exception replayAll(); List expectedUrls = new ArrayList<>(); - List> futures = new ArrayList<>(); + List> futures = new ArrayList<>(); for (String testId : TEST_IDS) { expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "status"))); futures.add(client.getStatusAsync(testId)); } - List responses = Futures.allAsList(futures).get(); + List responses = Futures.allAsList(futures).get(); verifyAll(); List requests = captured.getValues(); @@ -822,7 +822,7 @@ public void testGetStatusAsync() throws Exception for (int i = 0; i < numRequests; i++) { Assert.assertEquals(HttpMethod.GET, requests.get(i).getMethod()); Assert.assertTrue("unexpectedURL", expectedUrls.contains(requests.get(i).getUrl())); - Assert.assertEquals(SeekableStreamIndexTask.Status.READING, responses.get(i)); + Assert.assertEquals(Status.READING, responses.get(i)); } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 49dcf71c8dcb..c8319c022a6c 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -83,7 +83,7 @@ import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; @@ -172,14 +172,19 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Random; import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.Executor; import java.util.concurrent.Executors; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; @RunWith(LocalstackDockerTestRunner.class) @@ -246,6 +251,7 @@ public class KinesisIndexTaskTest "1", JB("2049", "f", "y", "10", "20.0", "notanumber") ), + generateRequestEntry("1", StringUtils.toUtf8("END_OF_PARTITION")), generateRequestEntry( "123123", JB("2012", "g", "y", "10", "20.0", "1.0") @@ -253,7 +259,8 @@ public class KinesisIndexTaskTest generateRequestEntry( "123123", JB("2011", "h", "y", "10", "20.0", "1.0") - ) + ), + generateRequestEntry("123123", StringUtils.toUtf8("END_OF_PARTITION")) ); @@ -270,8 +277,10 @@ public class KinesisIndexTaskTest private boolean resetOffsetAutomatically = false; private boolean doHandoff = true; private int maxRowsInMemory = 1000; + private Integer maxRowsPerSegment = null; private Long maxTotalRows = null; private Period intermediateHandoffPeriod = null; + private int maxRecordsPerPoll; private TaskToolboxFactory toolboxFactory; private IndexerMetadataStorageCoordinator metadataStorageCoordinator; @@ -363,7 +372,7 @@ public static void setupClass() EmittingLogger.registerEmitter(emitter); taskExec = MoreExecutors.listeningDecorator( Executors.newCachedThreadPool( - Execs.makeThreadFactory("kafka-task-test-%d") + Execs.makeThreadFactory("kinesis-task-test-%d") ) ); } @@ -379,6 +388,7 @@ public void setupTest() throws IOException, InterruptedException doHandoff = true; stream = getStreamName(); reportsFile = File.createTempFile("KinesisIndexTaskTestReports-" + System.currentTimeMillis(), "json"); + maxRecordsPerPoll = 1; // sleep required because of kinesalite Thread.sleep(500); @@ -424,7 +434,7 @@ public void testRunAfterDataInserted() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) )), true, null, @@ -449,9 +459,9 @@ public void testRunAfterDataInserted() throws Exception Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics - Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); @@ -462,7 +472,7 @@ public void testRunAfterDataInserted() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -502,7 +512,7 @@ public void testRunBeforeDataInserted() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - OrderedPartitionableRecord.END_OF_SHARD_MARKER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), true, null, @@ -523,24 +533,30 @@ public void testRunBeforeDataInserted() throws Exception final ListenableFuture future = runTask(task); - // Wait for the task to start reading - while (task.getStatus() != SeekableStreamIndexTask.Status.READING) { + + // Wait for the task to start readingtestRunConfli + while (task.getRunner().getStatus() != Status.READING) { Thread.sleep(10); } // Insert data List res2 = insertData(kinesis, generateRecordsRequests(stream)); - // force shard 0 to close - kinesis.splitShard(stream, shardId0, "somenewshardpls234234234"); + while (task.getRunner().getRowIngestionMeters().getProcessed() < 3) { + Thread.sleep(10); + } + + // force shard to close + kinesis.splitShard(stream, shardId0, "somerandomshardidhah1213123"); // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + // Check metrics - Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata SegmentDescriptor desc1 = SD(task, "2011/P1D", 0); @@ -565,6 +581,380 @@ public void testRunBeforeDataInserted() throws Exception Assert.assertEquals(ImmutableList.of("z"), readSegmentColumn("dim1", desc3)); } + @Test(timeout = 120_000L) + public void testIncrementalHandOff() throws Exception + { + final String baseSequenceName = "sequence0"; + // as soon as any segment has more than one record, incremental publishing should happen + maxRowsPerSegment = 2; + maxRecordsPerPoll = 1; + + // Insert data + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0), + shardId0, + getSequenceNumber(res, shardId0, 0) + ) + ); + // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering + // of events fetched across two partitions from Kafka + final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 5), + shardId0, + getSequenceNumber(res, shardId0, 0) + ) + ); + final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 4), + shardId0, + getSequenceNumber(res, shardId0, 2) + ) + ); + final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 10), + shardId0, + getSequenceNumber(res, shardId0, 2) + ) + ); + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + baseSequenceName, + startPartitions, + endPartitions, + true, + null, + null, + null, + LocalstackTestRunner.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + final ListenableFuture future = runTask(task); + while (task.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); + Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets) || checkpoint2.getPartitionOffsetMap() + .equals(currentOffsets)); + task.getRunner().setEndOffsets(currentOffsets, false); + + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + Assert.assertEquals(1, checkpointRequestsHash.size()); + Assert.assertTrue( + checkpointRequestsHash.contains( + Objects.hash( + DATA_SCHEMA.getDataSource(), + 0, + new KinesisDataSourceMetadata(startPartitions), + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, currentOffsets)) + ) + ) + ); + + // Check metrics + Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); + SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc4 = SD(task, "2011/P1D", 0); + SegmentDescriptor desc5 = SD(task, "2011/P1D", 1); + SegmentDescriptor desc6 = SD(task, "2012/P1D", 0); + SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 10), + shardId0, + getSequenceNumber(res, shardId0, 2) + ))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); + Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4)) + && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) || + (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4)) + && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5)))); + Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6)); + Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); + } + + @Test(timeout = 120_000L) + public void testIncrementalHandOffMaxTotalRows() throws Exception + { + final String baseSequenceName = "sequence0"; + // incremental publish should happen every 3 records + maxRowsPerSegment = Integer.MAX_VALUE; + maxTotalRows = 3L; + + // Insert data + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream, 0, 13)); + final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0) + ) + ); + // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering + // of events fetched across two partitions from Kafka + final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 3) + ) + ); + final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 10) + ) + ); + final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 11) + ) + ); + + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + baseSequenceName, + startPartitions, + endPartitions, + true, + null, + null, + null, + LocalstackTestRunner.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + while (task.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); + + Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); + task.getRunner().setEndOffsets(currentOffsets, false); + + while (task.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + + final Map nextOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); + + Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), nextOffsets); + + task.getRunner().setEndOffsets(nextOffsets, false); + + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + Assert.assertEquals(2, checkpointRequestsHash.size()); + Assert.assertTrue( + checkpointRequestsHash.contains( + Objects.hash( + DATA_SCHEMA.getDataSource(), + 0, + new KinesisDataSourceMetadata(startPartitions), + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, currentOffsets)) + ) + ) + ); + Assert.assertTrue( + checkpointRequestsHash.contains( + Objects.hash( + DATA_SCHEMA.getDataSource(), + 0, + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, currentOffsets)), + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, nextOffsets)) + ) + ) + ); + + // Check metrics + Assert.assertEquals(6, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); + SegmentDescriptor desc3 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc4 = SD(task, "2011/P1D", 0); + SegmentDescriptor desc5 = SD(task, "2049/P1D", 0); + SegmentDescriptor desc7 = SD(task, "2013/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc7), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 11) + ))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc4)); + Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc5)); + Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); + } + + @Test(timeout = 120_000L) + public void testTimeBasedIncrementalHandOff() throws Exception + { + + final String baseSequenceName = "sequence0"; + // as soon as any segment hits maxRowsPerSegment or intermediateHandoffPeriod, incremental publishing should happen + maxRowsPerSegment = Integer.MAX_VALUE; + intermediateHandoffPeriod = new Period().withSeconds(0); + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream, 0, 13)); + + final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0) + ) + ); + // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering + // of events fetched across two partitions from Kafka + final SeekableStreamPartitions checkpoint = new SeekableStreamPartitions<>( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 1) + ) + ); + + final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + stream, + ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + ) + ); + + + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + baseSequenceName, + startPartitions, + endPartitions, + true, + null, + null, + null, + LocalstackTestRunner.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + // task will pause for checkpointing + while (task.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); + Assert.assertEquals(checkpoint.getPartitionSequenceNumberMap(), currentOffsets); + task.getRunner().setEndOffsets(currentOffsets, false); + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + Assert.assertEquals(1, checkpointRequestsHash.size()); + Assert.assertTrue( + checkpointRequestsHash.contains( + Objects.hash( + DATA_SCHEMA.getDataSource(), + 0, + new KinesisDataSourceMetadata(startPartitions), + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>( + stream, + checkpoint.getPartitionSequenceNumberMap() + )) + ) + ) + ); + + // Check metrics + Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + ))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); + } + @Test(timeout = 120_000L) public void testRunWithMinimumMessageTime() throws Exception @@ -582,7 +972,7 @@ public void testRunWithMinimumMessageTime() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) )), true, null, @@ -603,7 +993,7 @@ public void testRunWithMinimumMessageTime() throws Exception final ListenableFuture future = runTask(task); // Wait for the task to start reading - while (task.getStatus() != SeekableStreamIndexTask.Status.READING) { + while (task.getRunner().getStatus() != Status.READING) { Thread.sleep(10); } @@ -611,9 +1001,9 @@ public void testRunWithMinimumMessageTime() throws Exception Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics - Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(2, task.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); @@ -625,7 +1015,7 @@ public void testRunWithMinimumMessageTime() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -652,7 +1042,7 @@ public void testRunWithMaximumMessageTime() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) )), true, null, @@ -673,7 +1063,7 @@ public void testRunWithMaximumMessageTime() throws Exception final ListenableFuture future = runTask(task); // Wait for the task to start reading - while (task.getStatus() != SeekableStreamIndexTask.Status.READING) { + while (task.getRunner().getStatus() != Status.READING) { Thread.sleep(10); } @@ -681,9 +1071,9 @@ public void testRunWithMaximumMessageTime() throws Exception Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics - Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(2, task.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); @@ -696,7 +1086,7 @@ public void testRunWithMaximumMessageTime() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -732,7 +1122,7 @@ public void testRunWithTransformSpec() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) )), true, null, @@ -753,7 +1143,7 @@ public void testRunWithTransformSpec() throws Exception final ListenableFuture future = runTask(task); // Wait for the task to start reading - while (task.getStatus() != SeekableStreamIndexTask.Status.READING) { + while (task.getRunner().getStatus() != Status.READING) { Thread.sleep(10); } @@ -761,9 +1151,9 @@ public void testRunWithTransformSpec() throws Exception Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics - Assert.assertEquals(1, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(4, task.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata SegmentDescriptor desc1 = SD(task, "2009/P1D", 0); @@ -774,7 +1164,7 @@ public void testRunWithTransformSpec() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -825,14 +1215,145 @@ public void testRunOnNothing() throws Exception Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics - Assert.assertEquals(0, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); } + @Test(timeout = 60_000L) + public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception + { + handoffConditionTimeout = 5_000; + + // Insert data + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + "sequence0", + new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 5) + )), + true, + null, + null, + null, + LocalstackTestRunner.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 5) + ))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + + @Test(timeout = 60_000L) + public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exception + { + doHandoff = false; + handoffConditionTimeout = 100; + + // Insert data + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream)); + + final KinesisIndexTask task = createTask( + null, + new KinesisIOConfig( + "sequence0", + new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 5) + )), + true, + null, + null, + null, + LocalstackTestRunner.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 5) + ))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + @Test(timeout = 120_000L) public void testReportParseExceptions() throws Exception { @@ -879,9 +1400,9 @@ public void testReportParseExceptions() throws Exception Assert.assertEquals(TaskState.FAILED, future.get().getStatusCode()); // Check metrics - Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(1, task.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); @@ -909,7 +1430,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 12) + getSequenceNumber(res, shardId1, 13) )), true, null, @@ -929,6 +1450,8 @@ public void testMultipleParseExceptionsSuccess() throws Exception final ListenableFuture future = runTask(task); + kinesis.splitShard(stream, shardId1, "somerandomshardidhah1213123"); + TaskStatus status = future.get(); // Wait for task to exit @@ -936,9 +1459,9 @@ public void testMultipleParseExceptionsSuccess() throws Exception Assert.assertEquals(null, status.getErrorMsg()); // Check metrics - Assert.assertEquals(4, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(3, task.getRowIngestionMeters().getProcessedWithError()); - Assert.assertEquals(4, task.getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessedWithError()); + Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable()); // Check published metadata SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); @@ -952,7 +1475,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 12) + getSequenceNumber(res, shardId1, 13) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1008,7 +1531,7 @@ public void testMultipleParseExceptionsFailure() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 9) + getSequenceNumber(res, shardId1, 10) )), true, null, @@ -1035,10 +1558,10 @@ public void testMultipleParseExceptionsFailure() throws Exception IndexTaskTest.checkTaskStatusErrorMsgForParseExceptionsExceeded(status); // Check metrics - Assert.assertEquals(3, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRowIngestionMeters().getProcessedWithError()); - Assert.assertEquals(3, task.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessedWithError()); + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); @@ -1085,7 +1608,7 @@ public void testRunReplicas() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) )), true, null, @@ -1112,7 +1635,7 @@ public void testRunReplicas() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) )), true, null, @@ -1138,12 +1661,12 @@ public void testRunReplicas() throws Exception Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); // Check metrics - Assert.assertEquals(3, task1.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(3, task2.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); @@ -1155,7 +1678,7 @@ public void testRunReplicas() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1183,7 +1706,7 @@ public void testRunConflicting() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) )), true, null, @@ -1210,7 +1733,7 @@ public void testRunConflicting() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 9) + getSequenceNumber(res, shardId1, 10) )), true, null, @@ -1237,12 +1760,12 @@ public void testRunConflicting() throws Exception Assert.assertEquals(TaskState.FAILED, future2.get().getStatusCode()); // Check metrics - Assert.assertEquals(3, task1.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(3, task2.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(4, task2.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(4, task2.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata, should all be from the first task SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); @@ -1254,7 +1777,7 @@ public void testRunConflicting() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1282,7 +1805,7 @@ public void testRunConflictingWithoutTransactions() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) )), false, null, @@ -1309,7 +1832,7 @@ public void testRunConflictingWithoutTransactions() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 9) + getSequenceNumber(res, shardId1, 10) )), false, null, @@ -1342,12 +1865,12 @@ public void testRunConflictingWithoutTransactions() throws Exception Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); // Check metrics - Assert.assertEquals(3, task1.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(3, task2.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(4, task2.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task2.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(4, task2.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1); @@ -1381,9 +1904,9 @@ public void testRunOneTaskTwoPartitions() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4), + getSequenceNumber(res, shardId1, 5), shardId0, - getSequenceNumber(res, shardId0, 1) + getSequenceNumber(res, shardId0, 2) )), true, null, @@ -1403,30 +1926,31 @@ public void testRunOneTaskTwoPartitions() throws Exception final ListenableFuture future = runTask(task); + while (countEvents(task) < 5) { + Thread.sleep(10); + } + + kinesis.splitShard(stream, shardId0, "somerandomshardidhah1213123"); // Wait for tasks to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); // Check metrics - Assert.assertEquals(5, task.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(5, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); - // desc3 will not be created in KinesisIndexTask (0.12.x) as it does not create per Kafka partition Druid segments - SegmentDescriptor desc3 = SD(task, "2011/P1D", 1); SegmentDescriptor desc4 = SD(task, "2012/P1D", 0); - Assert.assertEquals(isIncrementalHandoffSupported - ? ImmutableSet.of(desc1, desc2, desc4) - : ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc4), publishedDescriptors()); Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4), + getSequenceNumber(res, shardId1, 5), shardId0, - getSequenceNumber(res, shardId0, 1) + getSequenceNumber(res, shardId0, 2) ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1437,12 +1961,8 @@ public void testRunOneTaskTwoPartitions() throws Exception // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically Assert.assertEquals( - isIncrementalHandoffSupported - ? ImmutableSet.of(ImmutableList.of("d", "e", "h")) - : ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), - isIncrementalHandoffSupported - ? ImmutableSet.of(readSegmentColumn("dim1", desc2)) - : ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3)) + ImmutableSet.of(ImmutableList.of("d", "e", "h")), + ImmutableSet.of(readSegmentColumn("dim1", desc2)) ); } @@ -1463,7 +1983,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + getSequenceNumber(res, shardId1, 5) )), true, null, @@ -1490,7 +2010,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 1) + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), true, null, @@ -1511,17 +2031,23 @@ public void testRunTwoTasksTwoPartitions() throws Exception final ListenableFuture future1 = runTask(task1); final ListenableFuture future2 = runTask(task2); + while (countEvents(task2) < 2) { + Thread.sleep(10); + } + + kinesis.splitShard(stream, shardId0, "somerandomshardidhah1213123"); + // Wait for tasks to exit Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); // Check metrics - Assert.assertEquals(3, task1.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(2, task2.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(2, task2.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); @@ -1534,9 +2060,9 @@ public void testRunTwoTasksTwoPartitions() throws Exception new KinesisDataSourceMetadata( new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4), + getSequenceNumber(res, shardId1, 5), shardId0, - getSequenceNumber(res, shardId0, 1) + OrderedPartitionableRecord.END_OF_SHARD_MARKER ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1545,12 +2071,8 @@ public void testRunTwoTasksTwoPartitions() throws Exception Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically Assert.assertEquals( - isIncrementalHandoffSupported - ? ImmutableSet.of(ImmutableList.of("d", "e", "h")) - : ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), - isIncrementalHandoffSupported - ? ImmutableSet.of(readSegmentColumn("dim1", desc2)) - : ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3)) + ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), + ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3)) ); Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4)); } @@ -1604,7 +2126,7 @@ public void testRestore() throws Exception Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); - List res2 = insertData(kinesis, generateRecordsRequests(stream, 4, 5)); + insertData(kinesis, generateRecordsRequests(stream, 4, 5)); // Start a new task final KinesisIndexTask task2 = createTask( @@ -1617,7 +2139,7 @@ public void testRestore() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - OrderedPartitionableRecord.END_OF_SHARD_MARKER + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER )), true, null, @@ -1637,23 +2159,24 @@ public void testRestore() throws Exception final ListenableFuture future2 = runTask(task2); - while (countEvents(task2) < 1) { + while (countEvents(task2) < 3) { Thread.sleep(25); } - // force shard to close + Assert.assertEquals(3, countEvents(task2)); + kinesis.splitShard(stream, shardId1, "somerandomshardidhah1213123"); // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); // Check metrics - Assert.assertEquals(2, task1.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task1.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task1.getRowIngestionMeters().getThrownAway()); - Assert.assertEquals(1, task2.getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task2.getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(2, task1.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); + Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); @@ -1673,6 +2196,176 @@ public void testRestore() throws Exception Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } + @Test(timeout = 120_000L) + public void testRunWithPauseAndResume() throws Exception + { + // Insert data + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream, 0, 5)); + + final KinesisIndexTask task = createTask( + "task1", + new KinesisIOConfig( + "sequence0", + new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 2) + )), + new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + true, + null, + null, + null, + LocalstackTestRunner.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + + while (countEvents(task) != 3) { + Thread.sleep(25); + } + + Assert.assertEquals(3, countEvents(task)); + Assert.assertEquals(Status.READING, task.getRunner().getStatus()); + + task.getRunner().pause(); + + while (task.getRunner().getStatus() != Status.PAUSED) { + Thread.sleep(10); + } + Assert.assertEquals(Status.PAUSED, task.getRunner().getStatus()); + + Map currentOffsets = task.getRunner().getCurrentOffsets(); + + try { + future.get(10, TimeUnit.SECONDS); + Assert.fail("Task completed when it should have been paused"); + } + catch (TimeoutException e) { + // carry on.. + } + + Assert.assertEquals(currentOffsets, task.getRunner().getCurrentOffsets()); + + task.getRunner().setEndOffsets(currentOffsets, true); + + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + Assert.assertEquals(task.getRunner().getEndOffsets(), task.getRunner().getCurrentOffsets()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>( + stream, + ImmutableMap.of( + shardId1, + currentOffsets.get(shardId1) + ) + )), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + + + @Test(timeout = 60_000L) + public void testRunContextSequenceAheadOfStartingOffsets() throws Exception + { + // This tests the case when a replacement task is created in place of a failed test + // which has done some incremental handoffs, thus the context will contain starting + // sequence sequences from which the task should start reading and ignore the start sequences + // Insert data + AmazonKinesis kinesis = getKinesisClientInstance(); + List res = insertData(kinesis, generateRecordsRequests(stream, 0, 6)); + + final TreeMap> sequences = new TreeMap<>(); + // Here the sequence number is 1 meaning that one incremental handoff was done by the failed task + // and this task should start reading from stream 2 for partition 0 + sequences.put(1, ImmutableMap.of(shardId1, getSequenceNumber(res, shardId1, 2))); + final Map context = new HashMap<>(); + context.put("checkpoints", objectMapper.writerWithType(new TypeReference>>() + { + }).writeValueAsString(sequences)); + + + final KinesisIndexTask task = createTask( + "task1", + new KinesisIOConfig( + "sequence0", + new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 0) + )), + new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 5) + )), + true, + null, + null, + null, + LocalstackTestRunner.getEndpointKinesis(), + null, + null, + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + null, + null, + null, + false + ), + context + ); + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2010/P1D", 0); + SegmentDescriptor desc2 = SD(task, "2011/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); + Assert.assertEquals( + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( + shardId1, + getSequenceNumber(res, shardId1, 5) + ))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + } + private ListenableFuture runTask(final Task task) { try { @@ -1745,7 +2438,8 @@ private KinesisIndexTask createTask( final KinesisTuningConfig tuningConfig = new KinesisTuningConfig( 1000, null, - null, + maxRowsPerSegment, + maxTotalRows, new Period("P1Y"), null, null, @@ -1758,12 +2452,14 @@ private KinesisIndexTask createTask( null, null, null, - null, + 5000, null, null, logParseExceptions, maxParseExceptions, - maxSavedParseExceptions + maxSavedParseExceptions, + maxRecordsPerPoll, + intermediateHandoffPeriod ); final Map context = null; final KinesisIndexTask task = new KinesisIndexTask( @@ -1791,7 +2487,8 @@ private KinesisIndexTask createTask( final KinesisTuningConfig tuningConfig = new KinesisTuningConfig( maxRowsInMemory, null, - null, + maxRowsPerSegment, + maxTotalRows, new Period("P1Y"), null, null, @@ -1809,11 +2506,11 @@ private KinesisIndexTask createTask( null, logParseExceptions, maxParseExceptions, - maxSavedParseExceptions + maxSavedParseExceptions, + maxRecordsPerPoll, + intermediateHandoffPeriod ); - if (isIncrementalHandoffSupported) { - context.put(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true); - } + context.put(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true); final KinesisIndexTask task = new KinesisIndexTask( taskId, @@ -2099,6 +2796,21 @@ private List readSegmentColumn(final String column, final SegmentDescrip return values; } + private void stopShards(Set shards, AmazonKinesis kinesis) + { + shards.forEach(s -> kinesis.splitShard(stream, s, "somerandomshardidhah" + getRandomShardName())); + } + + private String getRandomShardName() + { + Random random = ThreadLocalRandom.current(); + StringBuilder s = new StringBuilder(); + for (int i = 0; i < 10; i++) { + s.append((char) random.nextInt(27) + 'a'); + } + return s.toString(); + } + public long countEvents(final Task task) { @@ -2167,13 +2879,13 @@ private AmazonKinesis getKinesisClientInstance() throws InterruptedException return kinesis; } - private static String getSequenceNumber(List entries, String shardId, int offset) + private static String getSequenceNumber(List entries, String shardId, int stream) { List sortedEntries = entries.stream() .filter(e -> e.getShardId().equals(shardId)) .sorted(Comparator.comparing(e -> KinesisSequenceNumber.of(e.getSequenceNumber()))) .collect(Collectors.toList()); - return sortedEntries.get(offset).getSequenceNumber(); + return sortedEntries.get(stream).getSequenceNumber(); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index d4e2115bad32..657c3523dd9f 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -39,6 +39,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -46,9 +47,9 @@ import java.nio.ByteBuffer; import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.Set; -import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; @RunWith(LocalstackDockerTestRunner.class) @@ -65,6 +66,8 @@ public class KinesisRecordSupplierTest private static String shardId1 = "shardId-000000000001"; private static String shardId0 = "shardId-000000000000"; private static int streamPosFix = 0; + private static int pollRetry = 10; + private static KinesisRecordSupplier recordSupplier; private static final List records = ImmutableList.of( generateRequestEntry( "1", @@ -180,6 +183,13 @@ public void setupTest() stream = getStreamName(); } + @After + public void tearDownTest() + { + recordSupplier.close(); + recordSupplier = null; + } + @Test public void testSupplierSetup() throws InterruptedException { @@ -191,7 +201,7 @@ public void testSupplierSetup() throws InterruptedException StreamPartition.of(stream, shardId1) ); - KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( + recordSupplier = new KinesisRecordSupplier( LocalstackTestRunner.getEndpointKinesis(), TestUtils.TEST_ACCESS_KEY, TestUtils.TEST_SECRET_KEY, @@ -204,7 +214,8 @@ public void testSupplierSetup() throws InterruptedException 100, 5000, 5000, - 60000 + 60000, + 5 ); Assert.assertTrue(recordSupplier.getAssignment().isEmpty()); @@ -214,8 +225,6 @@ public void testSupplierSetup() throws InterruptedException Assert.assertEquals(partitions, recordSupplier.getAssignment()); Assert.assertEquals(ImmutableSet.of(shardId1, shardId0), recordSupplier.getPartitionIds(stream)); Assert.assertEquals(Collections.emptyList(), recordSupplier.poll(100)); - - recordSupplier.close(); } @Test @@ -237,11 +246,11 @@ public void testPoll() throws InterruptedException StreamPartition.of(stream, shardId1) ); - KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( + recordSupplier = new KinesisRecordSupplier( LocalstackTestRunner.getEndpointKinesis(), TestUtils.TEST_ACCESS_KEY, TestUtils.TEST_SECRET_KEY, - 1, + 100, 0, 2, null, @@ -250,18 +259,21 @@ public void testPoll() throws InterruptedException 100, 5000, 5000, - 60000 + 60000, + 100 ); recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); List> polledRecords = recordSupplier.poll(poll_timeout_millis); + for (int i = 0; polledRecords.size() != initialRecords.size() && i < pollRetry; i++) { + polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); + Thread.sleep(200); + } Assert.assertEquals(partitions, recordSupplier.getAssignment()); Assert.assertEquals(initialRecords.size(), polledRecords.size()); Assert.assertTrue(polledRecords.containsAll(initialRecords)); - - recordSupplier.close(); } @Test @@ -283,7 +295,7 @@ public void testPollAfterMoreDataAdded() throws InterruptedException StreamPartition.of(stream, shardId1) ); - KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( + recordSupplier = new KinesisRecordSupplier( LocalstackTestRunner.getEndpointKinesis(), TestUtils.TEST_ACCESS_KEY, TestUtils.TEST_SECRET_KEY, @@ -296,13 +308,18 @@ public void testPollAfterMoreDataAdded() throws InterruptedException 100, 5000, 5000, - 60000 + 60000, + 5 ); recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); List> polledRecords = recordSupplier.poll(poll_timeout_millis); + for (int i = 0; polledRecords.size() != initialRecords.size() && i < pollRetry; i++) { + polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); + Thread.sleep(200); + } List insertDataResults2 = insertData(kinesis, generateRecordsRequests(stream, 5, 12)); insertDataResults2.forEach(entry -> initialRecords.add(new OrderedPartitionableRecord<>( @@ -312,16 +329,17 @@ public void testPollAfterMoreDataAdded() throws InterruptedException null ))); - polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); + for (int i = 0; polledRecords.size() != initialRecords.size() && i < pollRetry; i++) { + polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); + Thread.sleep(200); + } Assert.assertEquals(initialRecords.size(), polledRecords.size()); Assert.assertTrue(polledRecords.containsAll(initialRecords)); - - recordSupplier.close(); } @Test - public void testSeek() throws InterruptedException, TimeoutException + public void testSeek() throws InterruptedException { AmazonKinesis kinesis = getKinesisClientInstance(); List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); @@ -333,7 +351,7 @@ public void testSeek() throws InterruptedException, TimeoutException shard1 ); - KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( + recordSupplier = new KinesisRecordSupplier( LocalstackTestRunner.getEndpointKinesis(), TestUtils.TEST_ACCESS_KEY, TestUtils.TEST_SECRET_KEY, @@ -346,7 +364,8 @@ public void testSeek() throws InterruptedException, TimeoutException 100, 5000, 5000, - 60000 + 60000, + 5 ); recordSupplier.assign(partitions); @@ -376,17 +395,19 @@ public void testSeek() throws InterruptedException, TimeoutException .collect(Collectors.toSet()); List> polledRecords = recordSupplier.poll(poll_timeout_millis); + for (int i = 0; polledRecords.size() != 8 && i < pollRetry; i++) { + polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); + Thread.sleep(200); + } Assert.assertEquals(8, polledRecords.size()); Assert.assertTrue(polledRecords.containsAll(initialRecords1)); Assert.assertTrue(polledRecords.containsAll(initialRecords2)); - recordSupplier.close(); - } @Test - public void testSeekToLatest() throws InterruptedException, TimeoutException + public void testSeekToLatest() throws InterruptedException { AmazonKinesis kinesis = getKinesisClientInstance(); List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); @@ -398,7 +419,7 @@ public void testSeekToLatest() throws InterruptedException, TimeoutException shard1 ); - KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( + recordSupplier = new KinesisRecordSupplier( LocalstackTestRunner.getEndpointKinesis(), TestUtils.TEST_ACCESS_KEY, TestUtils.TEST_SECRET_KEY, @@ -411,7 +432,8 @@ public void testSeekToLatest() throws InterruptedException, TimeoutException 100, 5000, 5000, - 60000 + 60000, + 5 ); recordSupplier.assign(partitions); @@ -421,12 +443,10 @@ public void testSeekToLatest() throws InterruptedException, TimeoutException recordSupplier.seekToLatest(partitions); Assert.assertEquals(Collections.emptyList(), recordSupplier.poll(poll_timeout_millis)); - - recordSupplier.close(); } @Test(expected = ISE.class) - public void testSeekUnassigned() throws InterruptedException, TimeoutException + public void testSeekUnassigned() throws InterruptedException { AmazonKinesis kinesis = getKinesisClientInstance(); List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); @@ -437,7 +457,7 @@ public void testSeekUnassigned() throws InterruptedException, TimeoutException shard1 ); - KinesisRecordSupplier recordSupplier = new KinesisRecordSupplier( + recordSupplier = new KinesisRecordSupplier( LocalstackTestRunner.getEndpointKinesis(), TestUtils.TEST_ACCESS_KEY, TestUtils.TEST_SECRET_KEY, @@ -450,7 +470,8 @@ public void testSeekUnassigned() throws InterruptedException, TimeoutException 100, 5000, 5000, - 60000 + 60000, + 5 ); recordSupplier.assign(partitions); @@ -458,7 +479,182 @@ public void testSeekUnassigned() throws InterruptedException, TimeoutException Assert.assertEquals(insertDataResults.get(0).getSequenceNumber(), recordSupplier.getEarliestSequenceNumber(shard1)); recordSupplier.seekToEarliest(Collections.singleton(shard0)); + } - recordSupplier.close(); + @Test + public void testPollAfterSeek() throws InterruptedException + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); + + Set> partitions = ImmutableSet.of( + StreamPartition.of(stream, shardId1) + ); + + recordSupplier = new KinesisRecordSupplier( + LocalstackTestRunner.getEndpointKinesis(), + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + 10, + 0, + 2, + null, + null, + false, + 100, + 5000, + 5000, + 60000, + 1 + ); + recordSupplier.assign(partitions); + recordSupplier.seek(StreamPartition.of(stream, shardId1), getSequenceNumber(insertDataResults, shardId1, 5)); + + for (int i = 0; recordSupplier.bufferSize() < 2 && i < pollRetry; i++) { + Thread.sleep(200); + } + OrderedPartitionableRecord firstRecord = recordSupplier.poll(poll_timeout_millis).get(0); + + Assert.assertEquals( + getSequenceNumber(insertDataResults, shardId1, 5), + firstRecord.getSequenceNumber() + ); + + recordSupplier.seek(StreamPartition.of(stream, shardId1), getSequenceNumber(insertDataResults, shardId1, 7)); + for (int i = 0; recordSupplier.bufferSize() < 2 && i < pollRetry; i++) { + Thread.sleep(200); + } + + OrderedPartitionableRecord record2 = recordSupplier.poll(poll_timeout_millis).get(0); + + Assert.assertNotNull(record2); + Assert.assertEquals(stream, record2.getStream()); + Assert.assertEquals(shardId1, record2.getPartitionId()); + Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 7), record2.getSequenceNumber()); + + recordSupplier.seek(StreamPartition.of(stream, shardId1), getSequenceNumber(insertDataResults, shardId1, 2)); + for (int i = 0; recordSupplier.bufferSize() < 2 && i < pollRetry; i++) { + Thread.sleep(200); + } + OrderedPartitionableRecord record3 = recordSupplier.poll(poll_timeout_millis).get(0); + + Assert.assertNotNull(record3); + Assert.assertEquals(stream, record3.getStream()); + Assert.assertEquals(shardId1, record3.getPartitionId()); + Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 2), record3.getSequenceNumber()); + } + + @Test + public void testPosition() throws InterruptedException + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); + + StreamPartition partition1 = StreamPartition.of(stream, shardId1); + Set> partitions = ImmutableSet.of( + partition1 + ); + + recordSupplier = new KinesisRecordSupplier( + LocalstackTestRunner.getEndpointKinesis(), + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + 100, + 0, + 2, + null, + null, + false, + 100, + 5000, + 5000, + 60000, + 1 + ); + recordSupplier.assign(partitions); + recordSupplier.seekToEarliest(partitions); + + Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 0), recordSupplier.getPosition(partition1)); + + recordSupplier.seek(partition1, getSequenceNumber(insertDataResults, shardId1, 3)); + Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 3), recordSupplier.getPosition(partition1)); + + Assert.assertEquals(new OrderedPartitionableRecord<>( + stream, + shardId1, + getSequenceNumber(insertDataResults, shardId1, 3), + null + ), recordSupplier.poll(poll_timeout_millis).get(0)); + + Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 4), recordSupplier.getPosition(partition1)); + + Assert.assertEquals( + getSequenceNumber(insertDataResults, shardId1, 4), + recordSupplier.poll(poll_timeout_millis).get(0).getSequenceNumber() + ); + + Assert.assertEquals( + getSequenceNumber(insertDataResults, shardId1, 5), + recordSupplier.poll(poll_timeout_millis).get(0).getSequenceNumber() + ); + + Assert.assertEquals( + getSequenceNumber(insertDataResults, shardId1, 6), + recordSupplier.poll(poll_timeout_millis).get(0).getSequenceNumber() + ); + + Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 7), recordSupplier.getPosition(partition1)); + } + + @Test + public void testPositionAfterPollBatch() throws InterruptedException + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); + + StreamPartition partition1 = StreamPartition.of(stream, shardId1); + Set> partitions = ImmutableSet.of( + partition1 + ); + + recordSupplier = new KinesisRecordSupplier( + LocalstackTestRunner.getEndpointKinesis(), + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + 100, + 0, + 2, + null, + null, + false, + 100, + 5000, + 5000, + 60000, + 3 + ); + recordSupplier.assign(partitions); + recordSupplier.seekToEarliest(partitions); + + Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 0), recordSupplier.getPosition(partition1)); + + int i = 0; + while (recordSupplier.bufferSize() < 3 && i++ < pollRetry) { + Thread.sleep(100); + } + + Assert.assertEquals(3, recordSupplier.poll(poll_timeout_millis).size()); + + Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 3), recordSupplier.getPosition(partition1)); + } + + + private static String getSequenceNumber(List entries, String shardId, int sequence) + { + List sortedEntries = entries.stream() + .filter(e -> e.getShardId().equals(shardId)) + .sorted(Comparator.comparing(e -> KinesisSequenceNumber.of(e.getSequenceNumber()))) + .collect(Collectors.toList()); + return sortedEntries.get(sequence).getSequenceNumber(); } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java index 7b22fefc771d..2251f22f6773 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java @@ -74,7 +74,7 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(10000, config.getRecordBufferSize()); Assert.assertEquals(5000, config.getRecordBufferOfferTimeout()); Assert.assertEquals(5000, config.getRecordBufferFullWait()); - Assert.assertEquals(60000, config.getFetchSequenceNumberTimeout()); + Assert.assertEquals(10000, config.getFetchSequenceNumberTimeout()); Assert.assertNull(config.getFetchThreads()); Assert.assertFalse(config.isSkipSequenceNumberAvailabilityCheck()); Assert.assertFalse(config.isResetOffsetAutomatically()); @@ -165,6 +165,7 @@ public void testCopyOf() 1, (long) 3, 2, + 100L, new Period("PT3S"), new File("/tmp/xxx"), 4, @@ -182,6 +183,8 @@ public void testCopyOf() null, null, null, + null, + null, null ); KinesisTuningConfig copy = original.copyOf(); @@ -189,6 +192,7 @@ public void testCopyOf() Assert.assertEquals(1, copy.getMaxRowsInMemory()); Assert.assertEquals(3, copy.getMaxBytesInMemory()); Assert.assertEquals(2, copy.getMaxRowsPerSegment()); + Assert.assertEquals(100L, (long) copy.getMaxTotalRows()); Assert.assertEquals(new Period("PT3S"), copy.getIntermediatePersistPeriod()); Assert.assertEquals(new File("/tmp/xxx"), copy.getBasePersistDirectory()); Assert.assertEquals(0, copy.getMaxPendingPersists()); @@ -203,5 +207,7 @@ public void testCopyOf() Assert.assertEquals(2, (int) copy.getFetchThreads()); Assert.assertFalse(copy.isSkipSequenceNumberAvailabilityCheck()); Assert.assertTrue(copy.isResetOffsetAutomatically()); + Assert.assertEquals(5, copy.getMaxRecordsPerPoll()); + Assert.assertEquals(new Period().withDays(Integer.MAX_VALUE), copy.getIntermediateHandoffPeriod()); } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index ce87d4ea199e..2bfee014be13 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -28,6 +28,7 @@ import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry; import com.amazonaws.services.kinesis.model.PutRecordsResult; import com.amazonaws.services.kinesis.model.PutRecordsResultEntry; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.base.Throwables; @@ -64,7 +65,7 @@ import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.java.util.common.DateTimes; @@ -108,7 +109,9 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; +import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import static org.easymock.EasyMock.expect; @@ -237,6 +240,7 @@ public void setupTest() 1000, null, 50000, + null, new Period("P1Y"), new File("/test"), null, @@ -255,6 +259,8 @@ public void setupTest() null, null, null, + 5000, + null, null, null, null, @@ -527,8 +533,8 @@ public void testEarlyMessageRejectionPeriod() throws Exception @Test /** - * Test generating the starting offsets from the partition data stored in druid_dataSource which contains the - * offsets of the last built segments. + * Test generating the starting sequences from the partition data stored in druid_dataSource which contains the + * sequences of the last built segments. */ public void testDatasourceMetadata() throws Exception { @@ -709,7 +715,7 @@ public void testKillIncompatibleTasks() throws Exception EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes(); EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) @@ -734,11 +740,9 @@ public void testKillIncompatibleTasks() throws Exception getSequenceNumber(res, shardId1, 0) )); - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(2); - */ replayAll(); supervisor.start(); @@ -746,7 +750,6 @@ public void testKillIncompatibleTasks() throws Exception verifyAll(); } - // TODO: delete redundant tasks @Test public void testKillBadPartitionAssignment() throws Exception { @@ -851,7 +854,7 @@ public void testKillBadPartitionAssignment() throws Exception EasyMock.expect(taskStorage.getTask("id4")).andReturn(Optional.of(id4)).anyTimes(); EasyMock.expect(taskStorage.getTask("id5")).andReturn(Optional.of(id5)).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) @@ -869,14 +872,12 @@ public void testKillBadPartitionAssignment() throws Exception checkpoints1.put(0, ImmutableMap.of(shardId1, getSequenceNumber(res, shardId1, 0))); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(1); - */ taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); @@ -902,7 +903,7 @@ public void testRequeueTaskWhenFailed() throws Exception EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) @@ -927,14 +928,12 @@ public void testRequeueTaskWhenFailed() throws Exception shardId0, getSequenceNumber(res, shardId0, 0) )); - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .anyTimes(); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .anyTimes(); - */ taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1029,7 +1028,7 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); EasyMock.expect(taskClient.getStatusAsync("id1")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( @@ -1045,11 +1044,9 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception shardId0, getSequenceNumber(res, shardId0, 0) )); - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(2); - */ taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -1070,14 +1067,12 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception EasyMock.reset(taskClient); // for the newly created replica task - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - */ EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes(); @@ -1089,7 +1084,7 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception EasyMock.expect(taskStorage.getTask(iHaveFailed.getId())).andReturn(Optional.of((Task) iHaveFailed)).anyTimes(); EasyMock.expect(taskStorage.getTask(runningTaskId)).andReturn(Optional.of(captured.getValue())).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(runningTaskId)) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); EasyMock.expect(taskClient.getStartTimeAsync(runningTaskId)).andReturn(Futures.immediateFuture(now)).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.capture(aNewTaskCapture))).andReturn(true); EasyMock.replay(taskStorage); @@ -1130,7 +1125,7 @@ public void testQueueNextTasksOnSuccess() throws Exception EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) @@ -1155,7 +1150,7 @@ public void testQueueNextTasksOnSuccess() throws Exception EasyMock.reset(taskClient); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) .anyTimes(); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) @@ -1173,14 +1168,12 @@ public void testQueueNextTasksOnSuccess() throws Exception getSequenceNumber(res, shardId1, 0) )); // there would be 4 tasks, 2 for each task group - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); - */ EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); @@ -1196,7 +1189,7 @@ public void testQueueNextTasksOnSuccess() throws Exception supervisor.runInternal(); verifyAll(); - // test that a task succeeding causes a new task to be re-queued with the next offset range and causes any replica + // test that a task succeeding causes a new task to be re-queued with the next stream range and causes any replica // tasks to be shutdown Capture newTasksCapture = Capture.newInstance(CaptureType.ALL); Capture shutdownTaskIdCapture = Capture.newInstance(); @@ -1273,7 +1266,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception } EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)) + .andReturn(Futures.immediateFuture(Status.READING)) .anyTimes(); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) @@ -1318,14 +1311,12 @@ public void testBeginPublishAndQueueNextTasks() throws Exception shardId0, getSequenceNumber(res, shardId0, 0) )); - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); - */ EasyMock.replay(taskStorage, taskRunner, taskClient, taskQueue); @@ -1398,7 +1389,7 @@ public void testDiscoverExistingPublishingTask() throws Exception ) ).anyTimes(); EasyMock.expect(taskClient.getStatusAsync("id1")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + .andReturn(Futures.immediateFuture(Status.PUBLISHING)); EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture(ImmutableMap.of( shardId1, @@ -1473,7 +1464,7 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); - // check that the new task was created with starting offsets matching where the publishing task finished + // check that the new task was created with starting sequences matching where the publishing task finished Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); Assert.assertEquals( getSequenceNumber(res, shardId1, 2), @@ -1540,7 +1531,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() ) ).anyTimes(); EasyMock.expect(taskClient.getStatusAsync("id1")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + .andReturn(Futures.immediateFuture(Status.PUBLISHING)); EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture(ImmutableMap.of( shardId1, @@ -1604,7 +1595,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); - // check that the new task was created with starting offsets matching where the publishing task finished + // check that the new task was created with starting sequences matching where the publishing task finished Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); Assert.assertEquals( getSequenceNumber(res, shardId1, 2), @@ -1698,9 +1689,9 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception ) ).anyTimes(); EasyMock.expect(taskClient.getStatusAsync("id1")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + .andReturn(Futures.immediateFuture(Status.PUBLISHING)); EasyMock.expect(taskClient.getStatusAsync("id2")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture(ImmutableMap.of( @@ -1733,11 +1724,9 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception shardId0, getSequenceNumber(res, shardId0, 1) )); - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - */ replayAll(); @@ -1833,14 +1822,12 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception )); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); - */ EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); @@ -1850,7 +1837,7 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception .anyTimes(); EasyMock.expect(taskStorage.getTask(task.getId())).andReturn(Optional.of(task)).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(task.getId())) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.NOT_STARTED)); + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)); EasyMock.expect(taskClient.getStartTimeAsync(task.getId())) .andReturn(Futures.immediateFailedFuture(new RuntimeException())); taskQueue.shutdown(task.getId()); @@ -1903,14 +1890,12 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception )); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); - */ captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); @@ -1922,7 +1907,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception } EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)) + .andReturn(Futures.immediateFuture(Status.READING)) .anyTimes(); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) @@ -1995,14 +1980,12 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception )); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints2)) .times(2); - */ captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes(); @@ -2014,7 +1997,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception } EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)) + .andReturn(Futures.immediateFuture(Status.READING)) .anyTimes(); EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFuture(DateTimes.nowUtc().minusMinutes(2))) @@ -2172,11 +2155,11 @@ public void testStopGracefully() throws Exception ) ).anyTimes(); EasyMock.expect(taskClient.getStatusAsync("id1")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + .andReturn(Futures.immediateFuture(Status.PUBLISHING)); EasyMock.expect(taskClient.getStatusAsync("id2")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); EasyMock.expect(taskClient.getStatusAsync("id3")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( @@ -2194,14 +2177,12 @@ public void testStopGracefully() throws Exception shardId0, getSequenceNumber(res, shardId0, 1) )); - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - */ taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2317,7 +2298,7 @@ public void testResetDataSourceMetadata() throws Exception supervisor.resetInternal(resetMetadata); } catch (NullPointerException npe) { - // Expected as there will be an attempt to EasyMock.reset partitionGroups offsets to NOT_SET + // Expected as there will be an attempt to EasyMock.reset partitionGroups sequences to NOT_SET // however there would be no entries in the map as we have not put nay data in kafka Assert.assertTrue(npe.getCause() == null); } @@ -2459,11 +2440,11 @@ public void testResetRunningTasks() throws Exception ) ).anyTimes(); EasyMock.expect(taskClient.getStatusAsync("id1")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + .andReturn(Futures.immediateFuture(Status.PUBLISHING)); EasyMock.expect(taskClient.getStatusAsync("id2")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); EasyMock.expect(taskClient.getStatusAsync("id3")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( @@ -2486,14 +2467,12 @@ public void testResetRunningTasks() throws Exception getSequenceNumber(res, shardId0, 1) )); - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - */ taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2517,8 +2496,7 @@ public void testNoDataIngestionTasks() throws Exception { final DateTime startTime = DateTimes.nowUtc(); supervisor = getSupervisor(2, 1, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + //not adding any events Task id1 = createKinesisIndexTask( "id1", @@ -2526,12 +2504,9 @@ public void testNoDataIngestionTasks() throws Exception 0, new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - - getSequenceNumber(res, shardId1, 0), - + "0", shardId0, - - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -2549,12 +2524,9 @@ public void testNoDataIngestionTasks() throws Exception 0, new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - - getSequenceNumber(res, shardId1, 3), - + "10", shardId0, - - getSequenceNumber(res, shardId0, 1) + "20" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -2572,12 +2544,9 @@ public void testNoDataIngestionTasks() throws Exception 0, new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - - getSequenceNumber(res, shardId1, 3), - + "10", shardId0, - - getSequenceNumber(res, shardId0, 1) + "20" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -2604,11 +2573,11 @@ public void testNoDataIngestionTasks() throws Exception ) ).anyTimes(); EasyMock.expect(taskClient.getStatusAsync("id1")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); EasyMock.expect(taskClient.getStatusAsync("id2")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); EasyMock.expect(taskClient.getStatusAsync("id3")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); @@ -2616,14 +2585,10 @@ public void testNoDataIngestionTasks() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of( shardId1, - - getSequenceNumber(res, shardId1, 3), - + "10", shardId0, - - getSequenceNumber(res, shardId0, 1) + "20" )); - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); @@ -2633,7 +2598,6 @@ public void testNoDataIngestionTasks() throws Exception EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - */ taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2653,6 +2617,369 @@ public void testNoDataIngestionTasks() throws Exception verifyAll(); } + + @Test(timeout = 60_000L) + public void testCheckpointForInactiveTaskGroup() + throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException, ExecutionException, + TimeoutException, JsonProcessingException + { + supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); + //not adding any events + final Task id1 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + "0", + shardId0, + "0" + )), + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + final Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 0, + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + "10", + shardId0, + "20" + )), + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + final Task id3 = createKinesisIndexTask( + "id3", + DATASOURCE, + 0, + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + "10", + shardId0, + "20" + )), + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + final TaskLocation location1 = new TaskLocation("testHost", 1234, -1); + final TaskLocation location2 = new TaskLocation("testHost2", 145, -1); + Collection workItems = new ArrayList<>(); + workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); + workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); + workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); + + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); + 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 KinesisDataSourceMetadata( + null) + ).anyTimes(); + expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING)); + expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(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> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of( + shardId1, + "10", + shardId0, + "20" + )); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(1); + + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + + final Map fakeCheckpoints = Collections.emptyMap(); + supervisor.moveTaskGroupToPendingCompletion(0); + supervisor.checkpoint( + 0, + ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, checkpoints.get(0))), + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, fakeCheckpoints)) + ); + + while (supervisor.getNoticesQueueSize() > 0) { + Thread.sleep(100); + } + + verifyAll(); + + Assert.assertNull(serviceEmitter.getStackTrace(), serviceEmitter.getStackTrace()); + Assert.assertNull(serviceEmitter.getExceptionMessage(), 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 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + "0", + shardId0, + "0" + )), + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + final Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 0, + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + "10", + shardId0, + "20" + )), + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + final Task id3 = createKinesisIndexTask( + "id3", + DATASOURCE, + 0, + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + "10", + shardId0, + "20" + )), + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER, + shardId0, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + 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 KinesisDataSourceMetadata( + null) + ).anyTimes(); + + replayAll(); + + supervisor.start(); + + supervisor.checkpoint( + 0, + ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, Collections.emptyMap())), + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, Collections.emptyMap())) + ); + + while (supervisor.getNoticesQueueSize() > 0) { + Thread.sleep(100); + } + + verifyAll(); + + while (serviceEmitter.getStackTrace() == null) { + Thread.sleep(100); + } + + Assert.assertTrue(serviceEmitter.getStackTrace() + .startsWith("org.apache.druid.java.util.common.ISE: WTH?! cannot find")); + Assert.assertEquals( + "WTH?! cannot find taskGroup [0] among all activelyReadingTaskGroups [{}]", + serviceEmitter.getExceptionMessage() + ); + Assert.assertEquals(ISE.class, serviceEmitter.getExceptionClass()); + } + + @Test(timeout = 60_000L) + public void testCheckpointWithNullTaskGroupId() + throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException + { + supervisor = getSupervisor(1, 3, true, "PT1S", null, null, false); + //not adding any events + final Task id1 = createKinesisIndexTask( + "id1", + DATASOURCE, + 0, + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + "0" + )), + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + final Task id2 = createKinesisIndexTask( + "id2", + DATASOURCE, + 0, + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + "0" + )), + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + null, + null + ); + + final Task id3 = createKinesisIndexTask( + "id3", + DATASOURCE, + 0, + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + "0" + )), + new SeekableStreamPartitions<>("stream", ImmutableMap.of( + shardId1, + SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + )), + 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 KinesisDataSourceMetadata( + null) + ).anyTimes(); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + expect(taskClient.getStatusAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(Status.READING)) + .anyTimes(); + final TreeMap> checkpoints = new TreeMap<>(); + checkpoints.put(0, ImmutableMap.of(shardId1, "0")); + expect(taskClient.getCheckpointsAsync(EasyMock.anyString(), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(checkpoints)) + .times(3); + expect(taskClient.getStartTimeAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())) + .anyTimes(); + expect(taskClient.pauseAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(ImmutableMap.of(shardId1, "10"))) + .anyTimes(); + expect(taskClient.setEndOffsetsAsync( + EasyMock.anyString(), + EasyMock.eq(ImmutableMap.of("0", "10")), + EasyMock.anyBoolean() + )) + .andReturn(Futures.immediateFuture(true)) + .anyTimes(); + + replayAll(); + + supervisor.start(); + + supervisor.runInternal(); + + final TreeMap> newCheckpoints = new TreeMap<>(); + newCheckpoints.put(0, ImmutableMap.of(shardId1, "10")); + supervisor.checkpoint( + null, + ((KinesisIndexTask) id1).getIOConfig().getBaseSequenceName(), + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, checkpoints.get(0))), + new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, newCheckpoints.get(0))) + ); + + while (supervisor.getNoticesQueueSize() > 0) { + Thread.sleep(100); + } + + verifyAll(); + } + + @Test public void testSuspendedNoRunningTasks() throws Exception { @@ -2772,11 +3099,11 @@ public void testSuspendedRunningTasks() throws Exception ) ).anyTimes(); EasyMock.expect(taskClient.getStatusAsync("id1")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.PUBLISHING)); + .andReturn(Futures.immediateFuture(Status.PUBLISHING)); EasyMock.expect(taskClient.getStatusAsync("id2")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); EasyMock.expect(taskClient.getStatusAsync("id3")) - .andReturn(Futures.immediateFuture(SeekableStreamIndexTask.Status.READING)); + .andReturn(Futures.immediateFuture(Status.READING)); EasyMock.expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( @@ -2794,14 +3121,12 @@ public void testSuspendedRunningTasks() throws Exception shardId0, getSequenceNumber(res, shardId0, 1) )); - /* EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) .times(1); - */ taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); @@ -3100,13 +3425,13 @@ private static byte[] JB(String timestamp, String dim1, String dim2, String dimL } } - private static String getSequenceNumber(List entries, String shardId, int offset) + private static String getSequenceNumber(List entries, String shardId, int stream) { List sortedEntries = entries.stream() .filter(e -> e.getShardId().equals(shardId)) .sorted(Comparator.comparing(e -> KinesisSequenceNumber.of(e.getSequenceNumber()))) .collect(Collectors.toList()); - return sortedEntries.get(offset).getSequenceNumber(); + return sortedEntries.get(stream).getSequenceNumber(); } private KinesisIndexTask createKinesisIndexTask( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java index 3591924f8732..96d2894accb1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java @@ -78,7 +78,7 @@ public DataSourceMetadata plus(DataSourceMetadata other) final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) { - // Same topic, merge offsets. + // Same stream, merge sequences. final Map newMap = new HashMap<>(); for (Map.Entry entry : seekableStreamPartitions.getPartitionSequenceNumberMap().entrySet()) { @@ -91,7 +91,7 @@ public DataSourceMetadata plus(DataSourceMetadata other) return createConcreteDataSourceMetaData(seekableStreamPartitions.getStream(), newMap); } else { - // Different topic, prefer "other". + // Different stream, prefer "other". return other; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java index a369cfbef496..6d701d2203c9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java @@ -32,6 +32,7 @@ public abstract class SeekableStreamIOConfig implements IOConfig { private static final boolean DEFAULT_USE_TRANSACTION = true; + private static final boolean DEFAULT_SKIP_OFFSET_GAPS = false; @Nullable private final Integer taskGroupId; @@ -41,6 +42,8 @@ public abstract class SeekableStreamIOConfig implem private final boolean useTransaction; private final Optional minimumMessageTime; private final Optional maximumMessageTime; + private final boolean skipOffsetGaps; + @JsonCreator public SeekableStreamIOConfig( @@ -50,7 +53,8 @@ public SeekableStreamIOConfig( @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, @JsonProperty("useTransaction") Boolean useTransaction, @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, - @JsonProperty("maximumMessageTime") DateTime maximumMessageTime + @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, + @JsonProperty("skipOffsetGaps") Boolean skipOffsetGaps ) { this.taskGroupId = taskGroupId; @@ -60,6 +64,7 @@ public SeekableStreamIOConfig( this.useTransaction = useTransaction != null ? useTransaction : DEFAULT_USE_TRANSACTION; this.minimumMessageTime = Optional.fromNullable(minimumMessageTime); this.maximumMessageTime = Optional.fromNullable(maximumMessageTime); + this.skipOffsetGaps = skipOffsetGaps != null ? skipOffsetGaps : DEFAULT_SKIP_OFFSET_GAPS; Preconditions.checkArgument( startPartitions.getStream().equals(endPartitions.getStream()), @@ -117,6 +122,12 @@ public Optional getMinimumMessageTime() return minimumMessageTime; } + @JsonProperty + public boolean isSkipOffsetGaps() + { + return skipOffsetGaps; + } + // exclusive starting sequence partitions are used only for kinesis where the starting // sequence number for certain partitions are discarded because they've already been // read by a previous task diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index e53c0f08dc64..fa92c0b4b827 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -22,21 +22,35 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; +import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.query.NoopQueryRunner; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.appenderator.Appenderator; +import org.apache.druid.segment.realtime.appenderator.Appenderators; +import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; @@ -51,6 +65,9 @@ public abstract class SeekableStreamIndexTask extends AbstractTask implements ChatHandler { + private final EmittingLogger log = new EmittingLogger(this.getClass()); + public static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15; + private static final Random RANDOM = ThreadLocalRandom.current(); protected final DataSchema dataSchema; protected final InputRowParser parser; @@ -58,7 +75,11 @@ public abstract class SeekableStreamIndexTask exten protected final SeekableStreamIOConfig ioConfig; protected final Optional chatHandlerProvider; protected final String type; + protected final Map context; + protected final AuthorizerMapper authorizerMapper; + protected final RowIngestionMetersFactory rowIngestionMetersFactory; protected CircularBuffer savedParseExceptions; + private final SeekableStreamIndexTaskRunner runner; @JsonCreator public SeekableStreamIndexTask( @@ -92,8 +113,15 @@ public SeekableStreamIndexTask( } else { savedParseExceptions = null; } + this.context = context; + this.authorizerMapper = authorizerMapper; + this.rowIngestionMetersFactory = rowIngestionMetersFactory; + this.runner = createTaskRunner(); } + protected abstract SeekableStreamIndexTaskRunner createTaskRunner(); + + private static String makeTaskId(String dataSource, int randomBits, String type) { final StringBuilder suffix = new StringBuilder(8); @@ -103,6 +131,90 @@ private static String makeTaskId(String dataSource, int randomBits, String type) return Joiner.on("_").join(type, dataSource, suffix); } + public StreamAppenderatorDriver newDriver( + final Appenderator appenderator, + final TaskToolbox toolbox, + final FireDepartmentMetrics metrics + ) + { + return new StreamAppenderatorDriver( + appenderator, + new ActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + dataSchema, + (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck + ) + ), + toolbox.getSegmentHandoffNotifierFactory(), + new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), + toolbox.getDataSegmentKiller(), + toolbox.getObjectMapper(), + metrics + ); + } + + public Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) + { + return Appenderators.createRealtime( + dataSchema, + tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), + metrics, + toolbox.getSegmentPusher(), + toolbox.getObjectMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMergerV9(), + toolbox.getQueryRunnerFactoryConglomerate(), + toolbox.getSegmentAnnouncer(), + toolbox.getEmitter(), + toolbox.getQueryExecutorService(), + toolbox.getCache(), + toolbox.getCacheConfig(), + toolbox.getCachePopulatorStats() + ); + } + + public boolean withinMinMaxRecordTime(final InputRow row) + { + final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent() + && ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp()); + + final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent() + && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp()); + + if (!Intervals.ETERNITY.contains(row.getTimestamp())) { + final String errorMsg = StringUtils.format( + "Encountered row with timestamp that cannot be represented as a long: [%s]", + row + ); + throw new ParseException(errorMsg); + } + + if (log.isDebugEnabled()) { + if (beforeMinimumMessageTime) { + log.debug( + "CurrentTimeStamp[%s] is before MinimumMessageTime[%s]", + row.getTimestamp(), + ioConfig.getMinimumMessageTime().get() + ); + } else if (afterMaximumMessageTime) { + log.debug( + "CurrentTimeStamp[%s] is after MaximumMessageTime[%s]", + row.getTimestamp(), + ioConfig.getMaximumMessageTime().get() + ); + } + } + + return !beforeMinimumMessageTime && !afterMaximumMessageTime; + } + @Override public int getPriority() { @@ -140,24 +252,46 @@ public SeekableStreamIOConfig getIOConfig() } @Override - public abstract TaskStatus run(TaskToolbox toolbox) throws Exception; + public TaskStatus run(final TaskToolbox toolbox) + { + return runner.run(toolbox); + } @Override - public abstract boolean canRestore(); + public boolean canRestore() + { + return true; + } @Override - public abstract void stopGracefully(); + public void stopGracefully() + { + runner.stopGracefully(); + } @Override - public abstract QueryRunner getQueryRunner(Query query); + public QueryRunner getQueryRunner(Query query) + { + if (runner.getAppenderator() == null) { + // Not yet initialized, no data yet, just return a noop runner. + return new NoopQueryRunner<>(); + } + + return (queryPlus, responseContext) -> queryPlus.run(runner.getAppenderator(), responseContext); + } + + protected abstract RecordSupplier getRecordSupplier(); + + @VisibleForTesting + public Appenderator getAppenderator() + { + return runner.getAppenderator(); + } - public enum Status + @VisibleForTesting + public SeekableStreamIndexTaskRunner getRunner() { - NOT_STARTED, - STARTING, - READING, - PAUSED, - PUBLISHING + return runner; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index f48af510021c..5493e45825fc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -121,7 +121,7 @@ public Map pause(final String id) } while (true) { - if (getStatus(id) == SeekableStreamIndexTask.Status.PAUSED) { + if (getStatus(id) == SeekableStreamIndexTaskRunner.Status.PAUSED) { return getCurrentOffsets(id, true); } @@ -152,16 +152,16 @@ public Map pause(final String id) } } - public SeekableStreamIndexTask.Status getStatus(final String id) + public SeekableStreamIndexTaskRunner.Status getStatus(final String id) { log.debug("GetStatus task[%s]", id); try { final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "status", null, true); - return deserialize(response.getContent(), SeekableStreamIndexTask.Status.class); + return deserialize(response.getContent(), SeekableStreamIndexTaskRunner.Status.class); } catch (NoTaskLocationException e) { - return SeekableStreamIndexTask.Status.NOT_STARTED; + return SeekableStreamIndexTaskRunner.Status.NOT_STARTED; } catch (IOException e) { throw new RuntimeException(e); @@ -349,7 +349,7 @@ public ListenableFuture> getMovingAveragesAsync(final String } - public ListenableFuture getStatusAsync(final String id) + public ListenableFuture getStatusAsync(final String id) { return doAsync(() -> getStatus(id)); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 50bd9599f9da..fdb6b72e4333 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -19,52 +19,1930 @@ package org.apache.druid.indexing.seekablestream; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import org.apache.druid.data.input.Committer; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.discovery.DiscoveryDruidNode; +import org.apache.druid.discovery.LookupNodeService; +import org.apache.druid.discovery.NodeType; +import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; +import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.CheckPointDataSourceMetadataAction; +import org.apache.druid.indexing.common.actions.ResetDataSourceMetadataAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.stats.RowIngestionMeters; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.IndexTaskUtils; +import org.apache.druid.indexing.common.task.RealtimeIndexTask; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.collect.Utils; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.indexing.RealtimeIOConfig; +import org.apache.druid.segment.realtime.FireDepartment; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; +import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; +import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata; +import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; +import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.segment.realtime.firehose.ChatHandler; +import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.utils.CircularBuffer; +import org.joda.time.DateTime; +import javax.annotation.Nullable; +import javax.annotation.ParametersAreNonnullByDefault; +import javax.servlet.http.HttpServletRequest; +import javax.validation.constraints.NotNull; +import javax.ws.rs.Consumes; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; /** * Interface for abstracting the indexing task run logic. Only used by Kafka indexing tasks, * but will also be used by Kinesis indexing tasks once implemented * * @param Partition Number Type - * @param Sequence Number Type + * @param Sequence Number Type */ -public interface SeekableStreamIndexTaskRunner extends ChatHandler +public abstract class SeekableStreamIndexTaskRunner implements ChatHandler { - Appenderator getAppenderator(); + private final EmittingLogger log = new EmittingLogger(this.getClass()); + private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; + private static final String METADATA_PUBLISH_PARTITIONS = "publishPartitions"; - TaskStatus run(TaskToolbox toolbox); + private final Map endOffsets; + private final Map currOffsets = new ConcurrentHashMap<>(); + private final Map lastPersistedOffsets = new ConcurrentHashMap<>(); - void stopGracefully(); + private final Set publishingSequences = Sets.newConcurrentHashSet(); + private final List> publishWaitList = new ArrayList<>(); + private final List> handOffWaitList = new ArrayList<>(); + private final Map initialOffsetsSnapshot = new HashMap<>(); - @VisibleForTesting - RowIngestionMeters getRowIngestionMeters(); - @VisibleForTesting - SeekableStreamIndexTask.Status getStatus(); + // The pause lock and associated conditions are to support coordination between the Jetty threads and the main + // ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully + // the ingestion loop has been stopped at the returned sequences and will not ingest any more data until resumed. The + // fields are used as follows (every step requires acquiring [pauseLock]): + // Pausing: + // - In pause(), [pauseRequested] is set to true and then execution waits for [status] to change to PAUSED, with the + // condition checked when [hasPaused] is signalled. + // - In possiblyPause() called from the main loop, if [pauseRequested] is true, [status] is set to PAUSED, + // [hasPaused] is signalled, and execution pauses until [pauseRequested] becomes false, either by being set or by + // the [pauseMillis] timeout elapsing. [pauseRequested] is checked when [shouldResume] is signalled. + // Resuming: + // - In resume(), [pauseRequested] is set to false, [shouldResume] is signalled, and execution waits for [status] to + // change to something other than PAUSED, with the condition checked when [shouldResume] is signalled. + // - In possiblyPause(), when [shouldResume] is signalled, if [pauseRequested] has become false the pause loop ends, + // [status] is changed to STARTING and [shouldResume] is signalled. + private final Lock pauseLock = new ReentrantLock(); + private final Condition hasPaused = pauseLock.newCondition(); + private final Condition shouldResume = pauseLock.newCondition(); + + protected final Lock pollRetryLock = new ReentrantLock(); + protected final Condition isAwaitingRetry = pollRetryLock.newCondition(); + + protected final AtomicBoolean stopRequested = new AtomicBoolean(false); + private final AtomicBoolean publishOnStop = new AtomicBoolean(false); + + // [statusLock] is used to synchronize the Jetty thread calling stopGracefully() with the main run thread. It prevents + // the main run thread from switching into a publishing state while the stopGracefully() thread thinks it's still in + // a pre-publishing state. This is important because stopGracefully() will try to use the [stopRequested] flag to stop + // the main thread where possible, but this flag is not honored once publishing has begun so in this case we must + // interrupt the thread. The lock ensures that if the run thread is about to transition into publishing state, it + // blocks until after stopGracefully() has set [stopRequested] and then does a final check on [stopRequested] before + // transitioning to publishing state. + private final Object statusLock = new Object(); + + + private final SeekableStreamIOConfig ioConfig; + private final SeekableStreamTuningConfig tuningConfig; + private final SeekableStreamIndexTask task; + private final InputRowParser parser; + private final AuthorizerMapper authorizerMapper; + private final Optional chatHandlerProvider; + private final CircularBuffer savedParseExceptions; + private final RowIngestionMeters rowIngestionMeters; + private final String stream; + private final boolean isSkipSegmentLineageCheck; + + private volatile CopyOnWriteArrayList sequences; + private volatile IngestionState ingestionState; + private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread) + private volatile TaskToolbox toolbox; + private volatile Thread runThread; + protected volatile boolean pauseRequested = false; + private volatile long nextCheckpointTime; + + private volatile Appenderator appenderator; + private volatile StreamAppenderatorDriver driver; + private volatile Throwable backgroundThreadException; + private volatile DateTime startTime; + + protected class SequenceMetadata + { + /** + * Lock for accessing {@link #endOffsets} and {@link #checkpointed}. This lock is required because + * {@link #setEndOffsets)} can be called by both the main thread and the HTTP thread. + */ + private final ReentrantLock lock = new ReentrantLock(); + + private final int sequenceId; + private final String sequenceName; + private final Map startOffsets; + private final Map endOffsets; + private final Set assignments; + private final boolean sentinel; + private boolean checkpointed; + + @JsonCreator + public SequenceMetadata( + @JsonProperty("sequenceId") int sequenceId, + @JsonProperty("sequenceName") String sequenceName, + @JsonProperty("startOffsets") Map startOffsets, + @JsonProperty("endOffsets") Map endOffsets, + @JsonProperty("checkpointed") boolean checkpointed + ) + { + Preconditions.checkNotNull(sequenceName); + Preconditions.checkNotNull(startOffsets); + Preconditions.checkNotNull(endOffsets); + this.sequenceId = sequenceId; + this.sequenceName = sequenceName; + this.startOffsets = ImmutableMap.copyOf(startOffsets); + this.endOffsets = new HashMap<>(endOffsets); + this.assignments = new HashSet<>(startOffsets.keySet()); + this.checkpointed = checkpointed; + this.sentinel = false; + } + + @JsonProperty + public int getSequenceId() + { + return sequenceId; + } + + @JsonProperty + public boolean isCheckpointed() + { + lock.lock(); + try { + return checkpointed; + } + finally { + lock.unlock(); + } + } + + @JsonProperty + public String getSequenceName() + { + return sequenceName; + } + + @JsonProperty + public Map getStartOffsets() + { + return startOffsets; + } + + @JsonProperty + public Map getEndOffsets() + { + lock.lock(); + try { + return endOffsets; + } + finally { + lock.unlock(); + } + } + + @JsonProperty + public boolean isSentinel() + { + return sentinel; + } + + void setEndOffsets(Map newEndOffsets) + { + lock.lock(); + try { + endOffsets.putAll(newEndOffsets); + checkpointed = true; + } + finally { + lock.unlock(); + } + } + + void updateAssignments(Map nextPartitionOffset) + { + lock.lock(); + try { + assignments.clear(); + nextPartitionOffset.forEach((key, value) -> { + if (endOffsets.get(key).equals(SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER) + || createSequenceNumber(endOffsets.get(key)).compareTo(createSequenceNumber(nextPartitionOffset.get(key))) + > 0) { + assignments.add(key); + } + }); + } + finally { + lock.unlock(); + } + } + + boolean isOpen() + { + return !assignments.isEmpty(); + } + + boolean canHandle(OrderedPartitionableRecord record) + { + lock.lock(); + try { + final OrderedSequenceNumber partitionEndOffset = createSequencenNumber(endOffsets.get(record.getPartitionId())); + final OrderedSequenceNumber partitionStartOffset = createSequencenNumber(startOffsets.get(record.getPartitionId())); + final OrderedSequenceNumber recordOffset = createSequencenNumber(record.getSequenceNumber()); + return isOpen() + && recordOffset != null + && partitionEndOffset != null + && partitionStartOffset != null + && recordOffset.compareTo(partitionStartOffset) >= 0 + && recordOffset.compareTo(partitionEndOffset) < 0; + } + finally { + lock.unlock(); + } + } + + @Override + public String toString() + { + lock.lock(); + try { + return "SequenceMetadata{" + + "sequenceName='" + sequenceName + '\'' + + ", sequenceId=" + sequenceId + + ", startOffsets=" + startOffsets + + ", endOffsets=" + endOffsets + + ", assignments=" + assignments + + ", sentinel=" + sentinel + + ", checkpointed=" + checkpointed + + '}'; + } + finally { + lock.unlock(); + } + } + + Supplier getCommitterSupplier(String stream, Map lastPersistedOffsets) + { + // Set up committer. + return () -> + new Committer() + { + @Override + public Object getMetadata() + { + lock.lock(); + + try { + Preconditions.checkState( + assignments.isEmpty(), + "This committer can be used only once all the records till sequences [%s] have been consumed, also make" + + " sure to call updateAssignments before using this committer", + endOffsets + ); + + + // merge endOffsets for this sequence with globally lastPersistedOffsets + // This is done because this committer would be persisting only sub set of segments + // corresponding to the current sequence. Generally, lastPersistedOffsets should already + // cover endOffsets but just to be sure take max of sequences and persist that + for (Map.Entry partitionOffset : endOffsets.entrySet()) { + SequenceType newOffsets = partitionOffset.getValue(); + if (lastPersistedOffsets.containsKey(partitionOffset.getKey()) && + createSequencenNumber(lastPersistedOffsets.get(partitionOffset.getKey())).compareTo( + createSequencenNumber(newOffsets)) > 0) { + newOffsets = lastPersistedOffsets.get(partitionOffset.getKey()); + } + lastPersistedOffsets.put( + partitionOffset.getKey(), + newOffsets + ); + } + + // Publish metadata can be different from persist metadata as we are going to publish only + // subset of segments + return ImmutableMap.of( + METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>(stream, lastPersistedOffsets), + METADATA_PUBLISH_PARTITIONS, new SeekableStreamPartitions<>(stream, endOffsets) + ); + } + finally { + lock.unlock(); + } + } + + @Override + public void run() + { + // Do nothing. + } + }; + + } + + TransactionalSegmentPublisher createPublisher(TaskToolbox toolbox, boolean useTransaction) + { + return (segments, commitMetadata) -> { + final SeekableStreamPartitions finalPartitions = createSeekableStreamPartitions( + toolbox.getObjectMapper(), + ((Map) Preconditions + .checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_PUBLISH_PARTITIONS) + ); + + // Sanity check, we should only be publishing things that match our desired end state. + if (!getEndOffsets().equals(finalPartitions.getPartitionOffsetMap())) { + throw new ISE( + "WTF?! Driver for sequence [%s], attempted to publish invalid metadata[%s].", + toString(), + commitMetadata + ); + } + + final SegmentTransactionalInsertAction action; + + if (useTransaction) { + action = new SegmentTransactionalInsertAction( + segments, + createDataSourceMetadata(new SeekableStreamPartitions<>( + finalPartitions.getTopic(), + getStartOffsets() + )), + createDataSourceMetadata(finalPartitions) + ); + } else { + action = new SegmentTransactionalInsertAction(segments, null, null); + } + + log.info("Publishing with isTransaction[%s].", useTransaction); + + return toolbox.getTaskActionClient().submit(action); + }; + } + + } + + public enum Status + { + NOT_STARTED, + STARTING, + READING, + PAUSED, + PUBLISHING + } + + public SeekableStreamIndexTaskRunner( + final SeekableStreamIndexTask task, + final InputRowParser parser, + final AuthorizerMapper authorizerMapper, + final Optional chatHandlerProvider, + final CircularBuffer savedParseExceptions, + final RowIngestionMetersFactory rowIngestionMetersFactory, + final boolean isSkipSegmentLineageCheck + ) + { + this.task = task; + this.ioConfig = task.getIOConfig(); + this.tuningConfig = task.getTuningConfig(); + this.parser = parser; + this.authorizerMapper = authorizerMapper; + this.chatHandlerProvider = chatHandlerProvider; + this.savedParseExceptions = savedParseExceptions; + this.stream = ioConfig.getStartPartitions().getStream(); + this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters(); + this.isSkipSegmentLineageCheck = isSkipSegmentLineageCheck; + + this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndPartitions().getPartitionOffsetMap()); + this.sequences = new CopyOnWriteArrayList<>(); + this.ingestionState = IngestionState.NOT_STARTED; + + + resetNextCheckpointTime(); + } + + public Appenderator getAppenderator() + { + return appenderator; + } + + public TaskStatus run(TaskToolbox toolbox) + { + try { + return runInternal(toolbox); + } + catch (Exception e) { + log.error(e, "Encountered exception while running task."); + final String errorMsg = Throwables.getStackTraceAsString(e); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(errorMsg)); + return TaskStatus.failure( + task.getId(), + errorMsg + ); + } + } + + private TaskStatus runInternal(TaskToolbox toolbox) throws Exception + { + log.info("SeekableStream indexing task starting up!"); + startTime = DateTimes.nowUtc(); + status = Status.STARTING; + this.toolbox = toolbox; + + + if (!restoreSequences()) { + final TreeMap> checkpoints = getCheckPointsFromContext(toolbox, task); + if (checkpoints != null) { + Iterator>> sequenceOffsets = checkpoints.entrySet() + .iterator(); + Map.Entry> previous = sequenceOffsets.next(); + while (sequenceOffsets.hasNext()) { + Map.Entry> current = sequenceOffsets.next(); + sequences.add(new SequenceMetadata( + previous.getKey(), + StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), + previous.getValue(), + current.getValue(), + true + )); + previous = current; + } + sequences.add(new SequenceMetadata( + previous.getKey(), + StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), + previous.getValue(), + endOffsets, + false + )); + } else { + sequences.add(new SequenceMetadata( + 0, + StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0), + ioConfig.getStartPartitions().getPartitionOffsetMap(), + endOffsets, + false + )); + } + } + + log.info("Starting with sequences: %s", sequences); + + if (chatHandlerProvider.isPresent()) { + log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); + chatHandlerProvider.get().register(task.getId(), this, false); + } else { + log.warn("No chat handler detected"); + } + + runThread = Thread.currentThread(); + + // Set up FireDepartmentMetrics + final FireDepartment fireDepartmentForMetrics = new FireDepartment( + task.getDataSchema(), + new RealtimeIOConfig(null, null, null), + null + ); + FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); + toolbox.getMonitorScheduler() + .addMonitor(TaskRealtimeMetricsMonitorBuilder.build(task, fireDepartmentForMetrics, rowIngestionMeters)); + + final String lookupTier = task.getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER); + final LookupNodeService lookupNodeService = lookupTier == null ? + toolbox.getLookupNodeService() : + new LookupNodeService(lookupTier); + + final DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( + toolbox.getDruidNode(), + NodeType.PEON, + ImmutableMap.of( + toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), + lookupNodeService.getName(), lookupNodeService + ) + ); + + Throwable caughtExceptionOuter = null; + try (final RecordSupplier recordSupplier = task.getRecordSupplier()) { + toolbox.getDataSegmentServerAnnouncer().announce(); + toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); + + appenderator = task.newAppenderator(fireDepartmentMetrics, toolbox); + driver = task.newDriver(appenderator, toolbox, fireDepartmentMetrics); + + final String stream = ioConfig.getStartPartitions().getTopic(); + + // Start up, set up initial sequences. + final Object restoredMetadata = driver.startJob(); + if (restoredMetadata == null) { + // no persist has happened so far + // so either this is a brand new task or replacement of a failed task + Preconditions.checkState(sequences.get(0).startOffsets.entrySet().stream().allMatch( + partitionOffsetEntry -> + createSequencenNumber(partitionOffsetEntry.getValue()).compareTo( + createSequencenNumber(ioConfig.getStartPartitions() + .getPartitionOffsetMap() + .get(partitionOffsetEntry.getKey()) + )) >= 0 + ), "Sequence sequences are not compatible with start sequences of task"); + currOffsets.putAll(sequences.get(0).startOffsets); + } else { + @SuppressWarnings("unchecked") + final Map restoredMetadataMap = (Map) restoredMetadata; + final SeekableStreamPartitions restoredNextPartitions = createSeekableStreamPartitions( + toolbox.getObjectMapper(), + restoredMetadataMap.get(METADATA_NEXT_PARTITIONS) + ); + + currOffsets.putAll(restoredNextPartitions.getPartitionOffsetMap()); + + // Sanity checks. + if (!restoredNextPartitions.getTopic().equals(ioConfig.getStartPartitions().getTopic())) { + throw new ISE( + "WTF?! Restored stream[%s] but expected stream[%s]", + restoredNextPartitions.getTopic(), + ioConfig.getStartPartitions().getTopic() + ); + } + + if (!currOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionOffsetMap().keySet())) { + throw new ISE( + "WTF?! Restored partitions[%s] but expected partitions[%s]", + currOffsets.keySet(), + ioConfig.getStartPartitions().getPartitionOffsetMap().keySet() + ); + } + // sequences size can be 0 only when all sequences got published and task stopped before it could finish + // which is super rare + if (sequences.size() == 0 || sequences.get(sequences.size() - 1).isCheckpointed()) { + this.endOffsets.putAll(sequences.size() == 0 + ? currOffsets + : sequences.get(sequences.size() - 1).getEndOffsets()); + log.info("End sequences changed to [%s]", endOffsets); + } + } + + // Filter out partitions with END_OF_SHARD markers since these partitions have already been fully read. This + // should have been done by the supervisor already so this is defensive. + int numPreFilterPartitions = currOffsets.size(); + if (currOffsets.entrySet().removeIf(x -> OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(x.getValue()))) { + log.info( + "Removed [%d] partitions from assignment which have already been closed", + numPreFilterPartitions - currOffsets.size() + ); + } + + // Set up committer. + final Supplier committerSupplier = () -> { + final Map snapshot = ImmutableMap.copyOf(currOffsets); + lastPersistedOffsets.clear(); + lastPersistedOffsets.putAll(snapshot); + + return new Committer() + { + @Override + public Object getMetadata() + { + return ImmutableMap.of( + METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>( + ioConfig.getStartPartitions().getTopic(), + snapshot + ) + ); + } + + @Override + public void run() + { + // Do nothing. + } + }; + }; + + // restart publishing of sequences (if any) + maybePersistAndPublishSequences(committerSupplier); + + Set> assignment = assignPartitions(recordSupplier); + possiblyResetDataSourceMetadata(recordSupplier, assignment); + seekToStartingSequence(recordSupplier, assignment); + + ingestionState = IngestionState.BUILD_SEGMENTS; + + // Main loop. + // Could eventually support leader/follower mode (for keeping replicas more in sync) + boolean stillReading = !assignment.isEmpty(); + status = Status.READING; + Throwable caughtExceptionInner = null; + + initialOffsetsSnapshot.putAll(currOffsets); + + try { + while (stillReading) { + if (possiblyPause()) { + // The partition assignments may have changed while paused by a call to setEndOffsets() so reassign + // partitions upon resuming. This is safe even if the end sequences have not been modified. + assignment = assignPartitions(recordSupplier); + possiblyResetDataSourceMetadata(recordSupplier, assignment); + seekToStartingSequence(recordSupplier, assignment); + + if (assignment.isEmpty()) { + log.info("All partitions have been fully read"); + publishOnStop.set(true); + stopRequested.set(true); + } + } + + // if stop is requested or task's end sequence is set by call to setEndOffsets method with finish set to true + if (stopRequested.get() || sequences.get(sequences.size() - 1).isCheckpointed()) { + status = Status.PUBLISHING; + } + + if (stopRequested.get()) { + break; + } + + if (backgroundThreadException != null) { + throw new RuntimeException(backgroundThreadException); + } + + checkPublishAndHandoffFailure(); + + maybePersistAndPublishSequences(committerSupplier); + + + // calling getReocrd() ensures that excpetions specific to kafka/kinesis like OffsetOutOfRangeException + // are handled in the subclasses + List> records = getRecords(recordSupplier, toolbox); + + stillReading = !assignment.isEmpty(); + + SequenceMetadata sequenceToCheckpoint = null; + for (OrderedPartitionableRecord record : records) { + + // for the first message we receive, check that we were given a message with a sequenceNumber that matches our + // expected starting sequenceNumber + if (!verifyInitialRecordAndSkipExclusiveParition(record, initialOffsetsSnapshot)) { + continue; + } + + if (log.isTraceEnabled()) { + log.trace( + "Got stream[%s] partition[%s] sequence[%s].", + record.getStream(), + record.getPartitionId(), + record.getSequenceNumber() + ); + } + + if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(record.getSequenceNumber())) { + // shard is closed + currOffsets.put(record.getPartitionId(), record.getSequenceNumber()); + } else if (createSequencenNumber(record.getSequenceNumber()).compareTo( + createSequencenNumber(endOffsets.get(record.getPartitionId()))) < 0) { + + + if (!record.getSequenceNumber().equals(currOffsets.get(record.getPartitionId()))) { + if (!ioConfig.isSkipOffsetGaps()) { + throw new ISE( + "WTF?! Got sequence[%s] after sequence[%s] in partition[%s].", + record.getSequenceNumber(), + currOffsets.get(record.getPartitionId()), + record.getPartitionId() + ); + } + } + + try { + final List valueBytess = record.getData(); + final List rows; + if (valueBytess == null || valueBytess.isEmpty()) { + rows = Utils.nullableListOf((InputRow) null); + } else { + rows = new ArrayList<>(); + for (byte[] valueBytes : valueBytess) { + rows.addAll(parser.parseBatch(ByteBuffer.wrap(valueBytes))); + } + } + boolean isPersistRequired = false; + + final SequenceMetadata sequenceToUse = sequences + .stream() + .filter(sequenceMetadata -> sequenceMetadata.canHandle(record)) + .findFirst() + .orElse(null); + + if (sequenceToUse == null) { + throw new ISE( + "WTH?! cannot find any valid sequence for record with partition [%d] and sequence [%d]. Current sequences: %s", + record.getPartitionId(), + record.getSequenceNumber(), + sequences + ); + } + + for (InputRow row : rows) { + if (row != null && task.withinMinMaxRecordTime(row)) { + final AppenderatorDriverAddResult addResult = driver.add( + row, + sequenceToUse.getSequenceName(), + committerSupplier, + // true for kafka, false for kinesis + isSkipSegmentLineageCheck, + // do not allow incremental persists to happen until all the rows from this batch + // of rows are indexed + false + ); + + if (addResult.isOk()) { + // If the number of rows in the segment exceeds the threshold after adding a row, + // move the segment out from the active segments of BaseAppenderatorDriver to make a new segment. + if (addResult.isPushRequired(tuningConfig) && !sequenceToUse.isCheckpointed()) { + sequenceToCheckpoint = sequenceToUse; + } + isPersistRequired |= addResult.isPersistRequired(); + } else { + // Failure to allocate segment puts determinism at risk, bail out to be safe. + // May want configurable behavior here at some point. + // If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks. + throw new ISE("Could not allocate segment for row with timestamp[%s]", row.getTimestamp()); + } + + if (addResult.getParseException() != null) { + handleParseException(addResult.getParseException(), record); + } else { + rowIngestionMeters.incrementProcessed(); + } + } else { + rowIngestionMeters.incrementThrownAway(); + } + } + if (isPersistRequired) { + Futures.addCallback( + driver.persistAsync(committerSupplier.get()), + new FutureCallback() + { + @Override + public void onSuccess(@Nullable Object result) + { + log.info("Persist completed with metadata [%s]", result); + } + + @Override + public void onFailure(@ParametersAreNonnullByDefault Throwable t) + { + log.error("Persist failed, dying"); + backgroundThreadException = t; + } + } + ); + } + } + catch (ParseException e) { + handleParseException(e, record); + } + + + currOffsets.put( + record.getPartitionId(), + getNextSequenceNumber( + recordSupplier, + record.getStreamPartition(), + currOffsets.get(record.getPartitionId()) + ) + ); + + } + + if ((currOffsets.get(record.getPartitionId()).equals(endOffsets.get(record.getPartitionId())) + || currOffsets.get(record.getPartitionId()).equals(OrderedPartitionableRecord.END_OF_SHARD_MARKER)) + && assignment.remove(record.getStreamPartition())) { + log.info("Finished reading stream[%s], partition[%s].", record.getStream(), record.getPartitionId()); + recordSupplier.assign(assignment); + stillReading = !assignment.isEmpty(); + } + } + + if (System.currentTimeMillis() > nextCheckpointTime) { + sequenceToCheckpoint = sequences.get(sequences.size() - 1); + } + + if (sequenceToCheckpoint != null && stillReading) { + Preconditions.checkArgument( + sequences.get(sequences.size() - 1) + .getSequenceName() + .equals(sequenceToCheckpoint.getSequenceName()), + "Cannot checkpoint a sequence [%s] which is not the latest one, sequences %s", + sequenceToCheckpoint, + sequences + ); + requestPause(); + final CheckPointDataSourceMetadataAction checkpointAction = new CheckPointDataSourceMetadataAction( + task.getDataSource(), + ioConfig.getTaskGroupId(), + task.getIOConfig().getBaseSequenceName(), + createDataSourceMetadata(new SeekableStreamPartitions<>( + stream, + sequenceToCheckpoint.getStartOffsets() + )), + createDataSourceMetadata(new SeekableStreamPartitions<>(stream, currOffsets)) + ); + if (!toolbox.getTaskActionClient().submit(checkpointAction)) { + throw new ISE("Checkpoint request with sequences [%s] failed, dying", currOffsets); + } + } + } + ingestionState = IngestionState.COMPLETED; + } + catch (Exception e) { + // (1) catch all exceptions while reading from kafka + caughtExceptionInner = e; + log.error(e, "Encountered exception in run() before persisting."); + throw e; + } + finally { + log.info("Persisting all pending data"); + try { + driver.persist(committerSupplier.get()); // persist pending data + } + catch (Exception e) { + if (caughtExceptionInner != null) { + caughtExceptionInner.addSuppressed(e); + } else { + throw e; + } + } + } + + synchronized (statusLock) { + if (stopRequested.get() && !publishOnStop.get()) { + throw new InterruptedException("Stopping without publishing"); + } + + status = Status.PUBLISHING; + } + + for (SequenceMetadata sequenceMetadata : sequences) { + if (!publishingSequences.contains(sequenceMetadata.getSequenceName())) { + // this is done to prevent checks in sequence specific commit supplier from failing + sequenceMetadata.setEndOffsets(currOffsets); + sequenceMetadata.updateAssignments(currOffsets); + publishingSequences.add(sequenceMetadata.getSequenceName()); + // persist already done in finally, so directly add to publishQueue + publishAndRegisterHandoff(sequenceMetadata); + } + } + + if (backgroundThreadException != null) { + throw new RuntimeException(backgroundThreadException); + } + + // Wait for publish futures to complete. + Futures.allAsList(publishWaitList).get(); + + // Wait for handoff futures to complete. + // Note that every publishing task (created by calling AppenderatorDriver.publish()) has a corresponding + // handoffFuture. handoffFuture can throw an exception if 1) the corresponding publishFuture failed or 2) it + // failed to persist sequences. It might also return null if handoff failed, but was recoverable. + // See publishAndRegisterHandoff() for details. + List handedOffList = Collections.emptyList(); + if (tuningConfig.getHandoffConditionTimeout() == 0) { + handedOffList = Futures.allAsList(handOffWaitList).get(); + } else { + try { + handedOffList = Futures.allAsList(handOffWaitList) + .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); + } + catch (TimeoutException e) { + // Handoff timeout is not an indexing failure, but coordination failure. We simply ignore timeout exception + // here. + log.makeAlert("Timed out after [%d] millis waiting for handoffs", tuningConfig.getHandoffConditionTimeout()) + .addData("TaskId", task.getId()) + .emit(); + } + } + + for (SegmentsAndMetadata handedOff : handedOffList) { + log.info( + "Handoff completed for segments[%s] with metadata[%s].", + Joiner.on(", ").join( + handedOff.getSegments().stream().map(DataSegment::getIdentifier).collect(Collectors.toList()) + ), + Preconditions.checkNotNull(handedOff.getCommitMetadata(), "commitMetadata") + ); + } + + appenderator.close(); + recordSupplier.close(); + } + catch (InterruptedException | RejectedExecutionException e) { + // (2) catch InterruptedException and RejectedExecutionException thrown for the whole ingestion steps including + // the final publishing. + caughtExceptionOuter = e; + try { + Futures.allAsList(publishWaitList).cancel(true); + Futures.allAsList(handOffWaitList).cancel(true); + if (appenderator != null) { + appenderator.closeNow(); + } + } + catch (Exception e2) { + e.addSuppressed(e2); + } + + // handle the InterruptedException that gets wrapped in a RejectedExecutionException + if (e instanceof RejectedExecutionException + && (e.getCause() == null || !(e.getCause() instanceof InterruptedException))) { + throw e; + } + + // if we were interrupted because we were asked to stop, handle the exception and return success, else rethrow + if (!stopRequested.get()) { + Thread.currentThread().interrupt(); + throw e; + } + + log.info("The task was asked to stop before completing"); + } + catch (Exception e) { + // (3) catch all other exceptions thrown for the whole ingestion steps including the final publishing. + caughtExceptionOuter = e; + try { + Futures.allAsList(publishWaitList).cancel(true); + Futures.allAsList(handOffWaitList).cancel(true); + if (appenderator != null) { + appenderator.closeNow(); + } + } + catch (Exception e2) { + e.addSuppressed(e2); + } + throw e; + } + finally { + try { + + if (driver != null) { + driver.close(); + } + if (chatHandlerProvider.isPresent()) { + chatHandlerProvider.get().unregister(task.getId()); + } + + toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); + toolbox.getDataSegmentServerAnnouncer().unannounce(); + } + catch (Exception e) { + if (caughtExceptionOuter != null) { + caughtExceptionOuter.addSuppressed(e); + } else { + throw e; + } + } + } + + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(null)); + return TaskStatus.success(task.getId()); + } + + protected abstract SequenceType getNextSequenceNumber( + RecordSupplier recordSupplier, + StreamPartition partition, + SequenceType sequenceNumber + ); + + + @NotNull + protected abstract List> getRecords( + RecordSupplier recordSupplier, + TaskToolbox toolbox + ) throws Exception; + + private void checkPublishAndHandoffFailure() throws ExecutionException, InterruptedException + { + // Check if any publishFuture failed. + final List> publishFinished = publishWaitList + .stream() + .filter(Future::isDone) + .collect(Collectors.toList()); + + for (ListenableFuture publishFuture : publishFinished) { + // If publishFuture failed, the below line will throw an exception and catched by (1), and then (2) or (3). + publishFuture.get(); + } + + publishWaitList.removeAll(publishFinished); + + // Check if any handoffFuture failed. + final List> handoffFinished = handOffWaitList + .stream() + .filter(Future::isDone) + .collect(Collectors.toList()); + + for (ListenableFuture handoffFuture : handoffFinished) { + // If handoffFuture failed, the below line will throw an exception and catched by (1), and then (2) or (3). + handoffFuture.get(); + } + + handOffWaitList.removeAll(handoffFinished); + } + + + private void publishAndRegisterHandoff(SequenceMetadata sequenceMetadata) + { + log.info("Publishing segments for sequence [%s]", sequenceMetadata); + + final ListenableFuture publishFuture = Futures.transform( + driver.publish( + sequenceMetadata.createPublisher(toolbox, ioConfig.isUseTransaction()), + sequenceMetadata.getCommitterSupplier(stream, lastPersistedOffsets).get(), + Collections.singletonList(sequenceMetadata.getSequenceName()) + ), + (Function) publishedSegmentsAndMetadata -> { + if (publishedSegmentsAndMetadata == null) { + throw new ISE( + "Transaction failure publishing segments for sequence [%s]", + sequenceMetadata + ); + } else { + return publishedSegmentsAndMetadata; + } + } + ); + publishWaitList.add(publishFuture); + + // Create a handoffFuture for every publishFuture. The created handoffFuture must fail if publishFuture fails. + final SettableFuture handoffFuture = SettableFuture.create(); + handOffWaitList.add(handoffFuture); + + Futures.addCallback( + publishFuture, + new FutureCallback() + { + @Override + public void onSuccess(SegmentsAndMetadata publishedSegmentsAndMetadata) + { + log.info( + "Published segments[%s] with metadata[%s].", + publishedSegmentsAndMetadata.getSegments() + .stream() + .map(DataSegment::getIdentifier) + .collect(Collectors.toList()), + Preconditions.checkNotNull(publishedSegmentsAndMetadata.getCommitMetadata(), "commitMetadata") + ); + + sequences.remove(sequenceMetadata); + publishingSequences.remove(sequenceMetadata.getSequenceName()); + try { + persistSequences(); + } + catch (IOException e) { + log.error(e, "Unable to persist state, dying"); + handoffFuture.setException(e); + throw new RuntimeException(e); + } + + Futures.transform( + driver.registerHandoff(publishedSegmentsAndMetadata), + new Function() + { + @Nullable + @Override + public Void apply(@Nullable SegmentsAndMetadata handoffSegmentsAndMetadata) + { + if (handoffSegmentsAndMetadata == null) { + log.warn( + "Failed to handoff segments[%s]", + publishedSegmentsAndMetadata.getSegments() + .stream() + .map(DataSegment::getIdentifier) + .collect(Collectors.toList()) + ); + } + handoffFuture.set(handoffSegmentsAndMetadata); + return null; + } + } + ); + } + + @Override + public void onFailure(@ParametersAreNonnullByDefault Throwable t) + { + log.error(t, "Error while publishing segments for sequence[%s]", sequenceMetadata); + handoffFuture.setException(t); + } + } + ); + } + + private void handleParseException(ParseException pe, OrderedPartitionableRecord record) + { + if (pe.isFromPartiallyValidRow()) { + rowIngestionMeters.incrementProcessedWithError(); + } else { + rowIngestionMeters.incrementUnparseable(); + } + + if (tuningConfig.isLogParseExceptions()) { + log.error( + pe, + "Encountered parse exception on row from partition[%s] sequence[%s]", + record.getPartitionId(), + record.getSequenceNumber() + ); + } + + if (savedParseExceptions != null) { + savedParseExceptions.add(pe); + } + + if (rowIngestionMeters.getUnparseable() + rowIngestionMeters.getProcessedWithError() + > tuningConfig.getMaxParseExceptions()) { + log.error("Max parse exceptions exceeded, terminating task..."); + throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); + } + } + + private boolean verifyInitialRecordAndSkipExclusiveParition( + final OrderedPartitionableRecord record, + final Map intialSequenceSnapshot + ) + { + if (intialSequenceSnapshot.containsKey(record.getPartitionId())) { + if (!intialSequenceSnapshot.get(record.getPartitionId()).equals(record.getSequenceNumber())) { + throw new ISE( + "Starting sequenceNumber [%s] does not match expected [%s] for partition [%s]", + record.getSequenceNumber(), + intialSequenceSnapshot.get(record.getPartitionId()), + record.getPartitionId() + ); + } + + log.info( + "Verified starting sequenceNumber [%s] for partition [%s]", + record.getSequenceNumber(), record.getPartitionId() + ); + + intialSequenceSnapshot.remove(record.getPartitionId()); + if (intialSequenceSnapshot.isEmpty()) { + log.info("Verified starting sequences for all partitions"); + } + + // check exclusive starting sequence + if (ioConfig.getExclusiveStartSequenceNumberPartitions() != null + && ioConfig.getExclusiveStartSequenceNumberPartitions().contains(record.getPartitionId())) { + log.info("Skipping starting sequenceNumber for partition [%s] marked exclusive", record.getPartitionId()); + + return false; + } + } + + return true; + } + + private Set> assignPartitions( + RecordSupplier recordSupplier + ) + { + final Set> assignment = new HashSet<>(); + for (Map.Entry entry : currOffsets.entrySet()) { + final SequenceType endOffset = endOffsets.get(entry.getKey()); + if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(endOffset) + || SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) + || createSequencenNumber(entry.getValue()).compareTo(createSequencenNumber(endOffset)) < 0) { + assignment.add(StreamPartition.of(stream, entry.getKey())); + } else if (entry.getValue().equals(endOffset)) { + log.info("Finished reading partition[%s].", entry.getKey()); + } else { + throw new ISE( + "WTF?! Cannot start from sequence[%,d] > endOffset[%,d]", + entry.getValue(), + endOffset + ); + } + } + + recordSupplier.assign(assignment); + + return assignment; + } + + + private void seekToStartingSequence( + RecordSupplier recordSupplier, Set> partitions + ) + { + for (final StreamPartition partition : partitions) { + final SequenceType sequence = currOffsets.get(partition.getPartitionId()); + log.info("Seeking partition[%s] to sequence[%s].", partition.getPartitionId(), sequence); + recordSupplier.seek(partition, sequence); + } + } + + private void possiblyResetDataSourceMetadata( + RecordSupplier recordSupplier, + Set> assignment + ) + { + for (final StreamPartition streamPartition : assignment) { + SequenceType sequence = currOffsets.get(streamPartition.getPartitionId()); + if (!tuningConfig.isSkipSequenceNumberAvailabilityCheck()) { + SequenceType earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); + if (earliestSequenceNumber == null + || createSequencenNumber(earliestSequenceNumber).compareTo(createSequencenNumber(sequence)) > 0) { + if (tuningConfig.isResetOffsetAutomatically()) { + log.info("Attempting to reset sequences automatically for all partitions"); + try { + sendResetRequestAndWait( + assignment.stream() + .collect(Collectors.toMap(x -> x, x -> currOffsets.get(x.getPartitionId()))), + toolbox + ); + } + catch (IOException e) { + throw new ISE(e, "Exception while attempting to automatically reset sequences"); + } + } else { + throw new ISE( + "Starting sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) and resetOffsetAutomatically is not enabled", + sequence, + streamPartition.getPartitionId(), + earliestSequenceNumber + ); + } + } + } + } + } + + /** + * Checks if the pauseRequested flag was set and if so blocks: + * a) if pauseMillis == PAUSE_FOREVER, until pauseRequested is cleared + * b) if pauseMillis != PAUSE_FOREVER, until pauseMillis elapses -or- pauseRequested is cleared + *

+ * If pauseMillis is changed while paused, the new pause timeout will be applied. This allows adjustment of the + * pause timeout (making a timed pause into an indefinite pause and vice versa is valid) without having to resume + * and ensures that the loop continues to stay paused without ingesting any new events. You will need to signal + * shouldResume after adjusting pauseMillis for the new value to take effect. + *

+ * Sets paused = true and signals paused so callers can be notified when the pause command has been accepted. + *

+ * Additionally, pauses if all partitions assignments have been read and pauseAfterRead flag is set. + * + * @return true if a pause request was handled, false otherwise + */ + private boolean possiblyPause() throws InterruptedException + { + pauseLock.lockInterruptibly(); + try { + if (pauseRequested) { + status = Status.PAUSED; + hasPaused.signalAll(); + + while (pauseRequested) { + log.info("Pausing ingestion until resumed"); + shouldResume.await(); + } + + status = Status.READING; + shouldResume.signalAll(); + log.info("Ingestion loop resumed"); + return true; + } + } + finally { + pauseLock.unlock(); + } + + return false; + } + + protected void sendResetRequestAndWait( + Map, SequenceType> outOfRangePartitions, + TaskToolbox taskToolbox + ) + throws IOException + { + Map partitionOffsetMap = outOfRangePartitions + .entrySet().stream().collect(Collectors.toMap(x -> x.getKey().getPartitionId(), Map.Entry::getValue)); + + boolean result = taskToolbox + .getTaskActionClient() + .submit( + new ResetDataSourceMetadataAction( + task.getDataSource(), + createDataSourceMetadata( + new SeekableStreamPartitions<>( + ioConfig.getStartPartitions().getStream(), + partitionOffsetMap + ) + ) + ) + ); + + if (result) { + log.makeAlert("Resetting sequences for datasource [%s]", task.getDataSource()) + .addData("partitions", partitionOffsetMap.keySet()) + .emit(); + + requestPause(); + } else { + log.makeAlert("Failed to send reset request for partitions [%s]", partitionOffsetMap.keySet()).emit(); + } + } + + + private void maybePersistAndPublishSequences(Supplier committerSupplier) + throws InterruptedException + { + for (SequenceMetadata sequenceMetadata : sequences) { + sequenceMetadata.updateAssignments(currOffsets); + if (!sequenceMetadata.isOpen() && !publishingSequences.contains(sequenceMetadata.getSequenceName())) { + publishingSequences.add(sequenceMetadata.getSequenceName()); + try { + Object result = driver.persist(committerSupplier.get()); + log.info( + "Persist completed with results: [%s], adding sequence [%s] to publish queue", + result, + sequenceMetadata + ); + publishAndRegisterHandoff(sequenceMetadata); + } + catch (InterruptedException e) { + log.warn("Interrupted while persisting sequence [%s]", sequenceMetadata); + throw e; + } + } + } + } + + protected abstract SeekableStreamPartitions createSeekableStreamPartitions( + ObjectMapper mapper, + Object obeject + ); + + protected abstract OrderedSequenceNumber createSequencenNumber(SequenceType sequenceNumber); + + private boolean restoreSequences() throws IOException + { + final File sequencesPersistFile = getSequencesPersistFile(toolbox); + if (sequencesPersistFile.exists()) { + sequences = new CopyOnWriteArrayList<>( + toolbox.getObjectMapper().>readValue( + sequencesPersistFile, + new TypeReference>() + { + } + ) + ); + return true; + } else { + return false; + } + } + + private Map getTaskCompletionReports(@Nullable String errorMsg) + { + return TaskReport.buildTaskReports( + new IngestionStatsAndErrorsTaskReport( + task.getId(), + new IngestionStatsAndErrorsTaskReportData( + ingestionState, + getTaskCompletionUnparseableEvents(), + getTaskCompletionRowStats(), + errorMsg + ) + ) + ); + } + + private Map getTaskCompletionUnparseableEvents() + { + Map unparseableEventsMap = new HashMap<>(); + List buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions( + savedParseExceptions + ); + if (buildSegmentsParseExceptionMessages != null) { + unparseableEventsMap.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegmentsParseExceptionMessages); + } + return unparseableEventsMap; + } + + private Map getTaskCompletionRowStats() + { + Map metrics = new HashMap<>(); + metrics.put( + RowIngestionMeters.BUILD_SEGMENTS, + rowIngestionMeters.getTotals() + ); + return metrics; + } + + public void stopGracefully() + { + log.info("Stopping gracefully (status: [%s])", status); + stopRequested.set(true); + + synchronized (statusLock) { + if (status == Status.PUBLISHING) { + runThread.interrupt(); + return; + } + } + + try { + if (pauseLock.tryLock(SeekableStreamIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { + try { + if (pauseRequested) { + pauseRequested = false; + shouldResume.signalAll(); + } + } + finally { + pauseLock.unlock(); + } + } else { + log.warn("While stopping: failed to acquire pauseLock before timeout, interrupting run thread"); + runThread.interrupt(); + return; + } + + if (pollRetryLock.tryLock(SeekableStreamIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { + try { + isAwaitingRetry.signalAll(); + } + finally { + pollRetryLock.unlock(); + } + } else { + log.warn("While stopping: failed to acquire pollRetryLock before timeout, interrupting run thread"); + runThread.interrupt(); + } + } + catch (Exception e) { + Throwables.propagate(e); + } + } @VisibleForTesting - Map getCurrentOffsets(); + public Response setEndOffsets( + Map sequenceNumbers, + boolean finish // this field is only for internal purposes, shouldn't be usually set by users + ) throws InterruptedException + { + if (sequenceNumbers == null) { + return Response.status(Response.Status.BAD_REQUEST) + .entity("Request body must contain a map of { partition:endOffset }") + .build(); + } else if (!endOffsets.keySet().containsAll(sequenceNumbers.keySet())) { + return Response.status(Response.Status.BAD_REQUEST) + .entity( + StringUtils.format( + "Request contains partitions not being handled by this task, my partitions: %s", + endOffsets.keySet() + ) + ) + .build(); + } else { + try { + pauseLock.lockInterruptibly(); + // Perform all sequence related checks before checking for isPaused() + // and after acquiring pauseLock to correctly guard against duplicate requests + Preconditions.checkState(sequenceNumbers.size() > 0, "WTH?! No Sequences found to set end sequences"); + + final SequenceMetadata latestSequence = sequences.get(sequences.size() - 1); + if ((latestSequence.getStartOffsets().equals(sequenceNumbers) && !finish) || + (latestSequence.getEndOffsets().equals(sequenceNumbers) && finish)) { + log.warn("Ignoring duplicate request, end sequences already set for sequences [%s]", sequenceNumbers); + return Response.ok(sequenceNumbers).build(); + } else if (latestSequence.isCheckpointed()) { + return Response.status(Response.Status.BAD_REQUEST) + .entity(StringUtils.format( + "WTH?! Sequence [%s] has already endOffsets set, cannot set to [%s]", + latestSequence, + sequenceNumbers + )).build(); + } else if (!isPaused()) { + return Response.status(Response.Status.BAD_REQUEST) + .entity("Task must be paused before changing the end sequences") + .build(); + } + + for (Map.Entry entry : sequenceNumbers.entrySet()) { + if (createSequenceNumber(entry.getValue()).compareTo(createSequenceNumber(currOffsets.get(entry.getKey()))) + < 0) { + return Response.status(Response.Status.BAD_REQUEST) + .entity( + StringUtils.format( + "End sequence must be >= current sequence for partition [%s] (current: %s)", + entry.getKey(), + currOffsets.get(entry.getKey()) + ) + ) + .build(); + } + } + + resetNextCheckpointTime(); + latestSequence.setEndOffsets(sequenceNumbers); + + if (finish) { + log.info("Updating endOffsets from [%s] to [%s]", endOffsets, sequenceNumbers); + endOffsets.putAll(sequenceNumbers); + } else { + // create new sequence + final SequenceMetadata newSequence = new SequenceMetadata( + latestSequence.getSequenceId() + 1, + StringUtils.format("%s_%d", ioConfig.getBaseSequenceName(), latestSequence.getSequenceId() + 1), + sequenceNumbers, + endOffsets, + false + ); + sequences.add(newSequence); + initialOffsetsSnapshot.putAll(sequenceNumbers); + } + + persistSequences(); + } + catch (Exception e) { + log.error(e, "Unable to set end sequences, dying"); + backgroundThreadException = e; + // should resume to immediately finish kafka index task as failed + resume(); + return Response.status(Response.Status.INTERNAL_SERVER_ERROR) + .entity(Throwables.getStackTraceAsString(e)) + .build(); + } + finally { + pauseLock.unlock(); + } + } + + resume(); + + return Response.ok(sequenceNumbers).build(); + } @VisibleForTesting - Map getEndOffsets(); + public Response pause() throws InterruptedException + { + if (!(status == Status.PAUSED || status == Status.READING)) { + return Response.status(Response.Status.BAD_REQUEST) + .entity(StringUtils.format("Can't pause, task is not in a pausable state (state: [%s])", status)) + .build(); + } + + pauseLock.lockInterruptibly(); + try { + pauseRequested = true; + + pollRetryLock.lockInterruptibly(); + try { + isAwaitingRetry.signalAll(); + } + finally { + pollRetryLock.unlock(); + } + + if (isPaused()) { + shouldResume.signalAll(); // kick the monitor so it re-awaits with the new pauseMillis + } + + long nanos = TimeUnit.SECONDS.toNanos(2); + while (!isPaused()) { + if (nanos <= 0L) { + return Response.status(Response.Status.ACCEPTED) + .entity("Request accepted but task has not yet paused") + .build(); + } + nanos = hasPaused.awaitNanos(nanos); + } + } + finally { + pauseLock.unlock(); + } + + try { + return Response.ok().entity(toolbox.getObjectMapper().writeValueAsString(getCurrentOffsets())).build(); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } + } @VisibleForTesting - Response setEndOffsets( - Map offsets, - boolean finish // this field is only for internal purposes, shouldn't be usually set by users - ) throws InterruptedException; + public void resume() throws InterruptedException + { + pauseLock.lockInterruptibly(); + try { + pauseRequested = false; + shouldResume.signalAll(); + + long nanos = TimeUnit.SECONDS.toNanos(5); + while (isPaused()) { + if (nanos <= 0L) { + throw new RuntimeException("Resume command was not accepted within 5 seconds"); + } + nanos = shouldResume.awaitNanos(nanos); + } + } + finally { + pauseLock.unlock(); + } + } + + protected abstract SeekableStreamDataSourceMetadata createDataSourceMetadata( + SeekableStreamPartitions partitions + ); + + protected abstract OrderedSequenceNumber createSequenceNumber(SequenceType sequenceNumber); + + private void resetNextCheckpointTime() + { + nextCheckpointTime = DateTimes.nowUtc().plus(tuningConfig.getIntermediateHandoffPeriod()).getMillis(); + } + + private boolean isPaused() + { + return status == Status.PAUSED; + } + + private void requestPause() + { + pauseRequested = true; + } + + private synchronized void persistSequences() throws IOException + { + log.info("Persisting Sequences Metadata [%s]", sequences); + toolbox.getObjectMapper().writerWithType( + new TypeReference>() + { + } + ).writeValue(getSequencesPersistFile(toolbox), sequences); + } + + private static File getSequencesPersistFile(TaskToolbox toolbox) + { + return new File(toolbox.getPersistDir(), "sequences.json"); + } + + @Nullable + protected abstract TreeMap> getCheckPointsFromContext( + TaskToolbox toolbox, + SeekableStreamIndexTask task + ) throws IOException; + + /** + * Authorizes action to be performed on this task's datasource + * + * @return authorization result + */ + private Access authorizationCheck(final HttpServletRequest req, Action action) + { + return IndexTaskUtils.datasourceAuthorizationCheck(req, action, task.getDataSource(), authorizerMapper); + } @VisibleForTesting - Response pause() throws InterruptedException; + public RowIngestionMeters getRowIngestionMeters() + { + return rowIngestionMeters; + } + + @POST + @Path("/stop") + public Response stop(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.WRITE); + stopGracefully(); + return Response.status(Response.Status.OK).build(); + } + + @GET + @Path("/status") + @Produces(MediaType.APPLICATION_JSON) + public Status getStatusHTTP(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return status; + } @VisibleForTesting - void resume() throws InterruptedException; + public Status getStatus() + { + return status; + } + + @GET + @Path("/offsets/current") + @Produces(MediaType.APPLICATION_JSON) + public Map getCurrentOffsets(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return getCurrentOffsets(); + } + + public Map getCurrentOffsets() + { + return currOffsets; + } + + @GET + @Path("/offsets/end") + @Produces(MediaType.APPLICATION_JSON) + public Map getEndOffsetsHTTP(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return getEndOffsets(); + } + + public Map getEndOffsets() + { + return endOffsets; + } + + @POST + @Path("/offsets/end") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response setEndOffsetsHTTP( + Map sequences, + @QueryParam("finish") @DefaultValue("true") final boolean finish, + // this field is only for internal purposes, shouldn't be usually set by users + @Context final HttpServletRequest req + ) throws InterruptedException + { + authorizationCheck(req, Action.WRITE); + return setEndOffsets(sequences, finish); + } + + @GET + @Path("/rowStats") + @Produces(MediaType.APPLICATION_JSON) + public Response getRowStats( + @Context final HttpServletRequest req + ) + { + authorizationCheck(req, Action.READ); + Map returnMap = new HashMap<>(); + Map totalsMap = new HashMap<>(); + Map averagesMap = new HashMap<>(); + + totalsMap.put( + RowIngestionMeters.BUILD_SEGMENTS, + rowIngestionMeters.getTotals() + ); + averagesMap.put( + RowIngestionMeters.BUILD_SEGMENTS, + rowIngestionMeters.getMovingAverages() + ); + + returnMap.put("movingAverages", averagesMap); + returnMap.put("totals", totalsMap); + return Response.ok(returnMap).build(); + } + + @GET + @Path("/unparseableEvents") + @Produces(MediaType.APPLICATION_JSON) + public Response getUnparseableEvents( + @Context final HttpServletRequest req + ) + { + authorizationCheck(req, Action.READ); + List events = IndexTaskUtils.getMessagesFromSavedParseExceptions(savedParseExceptions); + return Response.ok(events).build(); + } + + @GET + @Path("/checkpoints") + @Produces(MediaType.APPLICATION_JSON) + public Map> getCheckpointsHTTP(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return getCheckpoints(); + } + + private Map> getCheckpoints() + { + return new TreeMap<>(sequences.stream() + .collect(Collectors.toMap( + SequenceMetadata::getSequenceId, + SequenceMetadata::getStartOffsets + ))); + } + + /** + * Signals the ingestion loop to pause. + * + * @return one of the following Responses: 400 Bad Request if the task has started publishing; 202 Accepted if the + * method has timed out and returned before the task has paused; 200 OK with a map of the current partition sequences + * in the response body if the task successfully paused + */ + @POST + @Path("/pause") + @Produces(MediaType.APPLICATION_JSON) + public Response pauseHTTP( + @Context final HttpServletRequest req + ) throws InterruptedException + { + authorizationCheck(req, Action.WRITE); + return pause(); + } + + @POST + @Path("/resume") + public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException + { + authorizationCheck(req, Action.WRITE); + resume(); + return Response.status(Response.Status.OK).build(); + } + + @GET + @Path("/time/start") + @Produces(MediaType.APPLICATION_JSON) + public DateTime getStartTime(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.WRITE); + return startTime; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index 4bbb27340188..bb473af06971 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -39,7 +39,7 @@ *

* Redundant getters * are used for proper Jackson serialization/deserialization when processing terminologies - * used by Kafka and kinesis (i.e. topic vs. name) + * used by Kafka and kinesis (i.e. topic vs. stream) * * @param partition id type * @param sequence number type @@ -49,7 +49,7 @@ public class SeekableStreamPartitions public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER"; // stream/topic - private final String name; + private final String stream; // partitionId -> sequence number private final Map map; @@ -61,31 +61,34 @@ public SeekableStreamPartitions( @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap ) { - this.name = stream == null ? topic : stream; + this.stream = stream == null ? topic : stream; this.map = ImmutableMap.copyOf(partitionOffsetMap == null ? partitionSequenceNumberMap : partitionOffsetMap); - Preconditions.checkArgument(this.name != null); + Preconditions.checkArgument(this.stream != null); Preconditions.checkArgument(map != null); } // constructor for backward compatibility - public SeekableStreamPartitions(@NotNull final String id, final Map partitionOffsetMap) + public SeekableStreamPartitions( + @NotNull final String stream, + final Map partitionOffsetMap + ) { - this(id, null, partitionOffsetMap, null); + this(stream, null, partitionOffsetMap, null); } @JsonProperty public String getStream() { - return name; + return stream; } @DoNotCall @JsonProperty public final String getTopic() { - return name; + return stream; } @JsonProperty @@ -111,21 +114,21 @@ public boolean equals(Object o) return false; } SeekableStreamPartitions that = (SeekableStreamPartitions) o; - return Objects.equals(name, that.name) && + return Objects.equals(stream, that.stream) && Objects.equals(map, that.map); } @Override public int hashCode() { - return Objects.hash(name, map); + return Objects.hash(stream, map); } @Override public String toString() { return "SeekableStreamPartitions{" + - "name/topic='" + name + '\'' + + "stream/topic='" + stream + '\'' + ", partitionSequenceNumberMap/partitionOffsetMap=" + map + '}'; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java index e5114ef05558..c1900300e22c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java @@ -71,7 +71,7 @@ public List getData() return data; } - public StreamPartition getStreamPartition() + public StreamPartition getStreamPartition() { return StreamPartition.of(stream, partitionId); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java index ad25d3c128b1..21e6e0318324 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java @@ -21,12 +21,12 @@ import com.google.common.annotations.Beta; +import javax.annotation.Nullable; import javax.validation.constraints.NotNull; import java.io.Closeable; import java.util.Collection; import java.util.List; import java.util.Set; -import java.util.concurrent.TimeoutException; /** * The RecordSupplier interface is a wrapper for the incoming seekable data stream @@ -39,8 +39,9 @@ public interface RecordSupplier extends Closeable { /** - * assigns the given parittions to this RecordSupplier. Previously - * assigned partitions will be replaced + * assigns the given parittions to this RecordSupplier + * and seek to the earliest sequence number. Previously + * assigned partitions will be replaced. * * @param partitions parititions to assign */ @@ -54,14 +55,6 @@ public interface RecordSupplier extends Closeable */ void seek(StreamPartition partition, SequenceType sequenceNumber); - /** - * seek to the sequence number immediately following the given sequenceNumber - * - * @param partition partition to seek - * @param sequenceNumber sequence number to seek - */ - void seekAfter(StreamPartition partition, SequenceType sequenceNumber); - /** * seek a set of partitions to the earliest record position available in the stream * @@ -99,10 +92,9 @@ public interface RecordSupplier extends Closeable * @param partition target partition * * @return latest sequence number - * - * @throws TimeoutException TimeoutException */ - SequenceType getLatestSequenceNumber(StreamPartition partition) throws TimeoutException; + @Nullable + SequenceType getLatestSequenceNumber(StreamPartition partition); /** * get the earliest sequence number in stream @@ -110,19 +102,19 @@ public interface RecordSupplier extends Closeable * @param partition target partition * * @return earliest sequence number - * - * @throws TimeoutException TimeoutException */ - SequenceType getEarliestSequenceNumber(StreamPartition partition) throws TimeoutException; + @Nullable + SequenceType getEarliestSequenceNumber(StreamPartition partition); + /** - * returns the sequence number that the given partition is currently at + * returns the sequence number of the next record * * @param partition target partition * * @return sequence number */ - SequenceType position(StreamPartition partition); + SequenceType getPosition(StreamPartition partition); /** * returns the set of partitions under the given stream diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index cc1ddc94d8fc..5ac3d55d63b1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -60,6 +60,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; @@ -145,18 +146,18 @@ public abstract class SeekableStreamSupervisor * indexing tasks. All the tasks in a TaskGroup should always be doing the same thing (reading the same partitions and * starting from the same sequences) and if [replicas] is configured to be 1, a TaskGroup will contain a single task (the * exception being if the supervisor started up and discovered and adopted some already running tasks). At any given - * 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 [activelyReadingTaskGroups] * map) + zero or more pending-completion task groups (tracked in [pendingCompletionTaskGroups]). */ private class TaskGroup { final int groupId; - // 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 sequences 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 // this task group has completed successfully, at which point this will be destroyed and a new task group will be - // created with new starting offsets. This allows us to create replacement tasks for failed tasks that process the - // same offsets, even if the values in [partitionGroups] has been changed. + // created with new starting sequences. This allows us to create replacement tasks for failed tasks that process the + // same sequences, even if the values in [partitionGroups] has been changed. final ImmutableMap startingSequences; final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); @@ -341,12 +342,12 @@ public void handle() throws ExecutionException, InterruptedException // Find taskGroupId using taskId if it's null. It can be null while rolling update. final int taskGroupId; if (nullableTaskGroupId == null) { - // We search taskId in taskGroups and pendingCompletionTaskGroups sequentially. This should be fine because - // 1) a taskGroup can be moved from taskGroups to pendingCompletionTaskGroups in RunNotice + // We search taskId in activelyReadingTaskGroups and pendingCompletionTaskGroups sequentially. This should be fine because + // 1) a taskGroup can be moved from activelyReadingTaskGroups to pendingCompletionTaskGroups in RunNotice // (see checkTaskDuration()). // 2) Notices are proceesed by a single thread. So, CheckpointNotice and RunNotice cannot be processed at the // same time. - final java.util.Optional maybeGroupId = taskGroups + final java.util.Optional maybeGroupId = activelyReadingTaskGroups .entrySet() .stream() .filter(entry -> { @@ -373,7 +374,7 @@ public void handle() throws ExecutionException, InterruptedException // check for consistency // if already received request for this sequenceName and dataSourceMetadata combination then return - final TaskGroup taskGroup = taskGroups.get(taskGroupId); + final TaskGroup taskGroup = activelyReadingTaskGroups.get(taskGroupId); if (isValidTaskGroup(taskGroupId, taskGroup)) { final TreeMap> checkpoints = taskGroup.checkpointSequences; @@ -382,7 +383,7 @@ public void handle() throws ExecutionException, InterruptedException int index = checkpoints.size(); for (int sequenceId : checkpoints.descendingKeySet()) { Map checkpoint = checkpoints.get(sequenceId); - // We have already verified the topic of the current checkpoint is same with that in ioConfig. + // We have already verified the stream of the current checkpoint is same with that in ioConfig. // See checkpoint(). if (checkpoint.equals(previousCheckpoint.getSeekableStreamPartitions() .getPartitionSequenceNumberMap() @@ -396,7 +397,7 @@ public void handle() throws ExecutionException, InterruptedException } else if (index < checkpoints.size()) { // if the found checkpoint is not the latest one then already checkpointed by a replica Preconditions.checkState(index == checkpoints.size() - 1, "checkpoint consistency failure"); - log.info("Already checkpointed with offsets [%s]", checkpoints.lastEntry().getValue()); + log.info("Already checkpointed with sequences [%s]", checkpoints.lastEntry().getValue()); return; } final Map newCheckpoint = checkpointTaskGroup(taskGroup, false).get(); @@ -420,7 +421,9 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) 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); + throw new ISE("WTH?! cannot find taskGroup [%s] among all activelyReadingTaskGroups [%s]", taskGroupId, + activelyReadingTaskGroups + ); } } @@ -430,21 +433,21 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) // Map<{group ID}, {actively reading task group}>; see documentation for TaskGroup class - private final ConcurrentHashMap taskGroups = new ConcurrentHashMap<>(); + private final ConcurrentHashMap activelyReadingTaskGroups = new ConcurrentHashMap<>(); - // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [taskGroups] to here so - // we can monitor its status while we queue new tasks to read the next range of offsets. This is a list since we could + // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [activelyReadingTaskGroups] to here so + // we can monitor its status while we queue new tasks to read the next range of sequences. This is a list since we could // have multiple sets of tasks publishing at once if time-to-publish > taskDuration. // Map<{group ID}, List<{pending completion task groups}>> private final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); - // The starting offset for a new partition in [partitionGroups] is initially set to getNotSetMarker(). When a new task group - // is created and is assigned partitions, if the offset in [partitionGroups] is getNotSetMarker() it will take the starting - // offset value from the metadata store, and if it can't find it there, from stream. Once a task begins - // publishing, the offset in partitionGroups will be updated to the ending offset of the publishing-but-not-yet- + // The starting sequence for a new partition in [partitionGroups] is initially set to getNotSetMarker(). When a new task group + // is created and is assigned partitions, if the sequence in [partitionGroups] is getNotSetMarker() it will take the starting + // sequence value from the metadata store, and if it can't find it there, from stream. Once a task begins + // publishing, the sequence in partitionGroups will be updated to the ending sequence of the publishing-but-not-yet- // completed task, which will cause the next set of tasks to begin reading from where the previous task left - // off. If that previous task now fails, we will set the offset in [partitionGroups] back to getNotSetMarker() which will - // cause successive tasks to again grab their starting offset from metadata store. This mechanism allows us to + // off. If that previous task now fails, we will set the sequence in [partitionGroups] back to getNotSetMarker() which will + // cause successive tasks to again grab their starting sequence from metadata store. This mechanism allows us to // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task // failures during publishing. // Map<{group ID}, Map<{partition ID}, {startingOffset}>> @@ -480,7 +483,6 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) private final Object recordSupplierLock = new Object(); private final boolean useExclusiveStartingSequence; - private final boolean isCheckpointSupported; private boolean listenerRegistered = false; private long lastRunTime; private int initRetryCounter = 0; @@ -500,8 +502,7 @@ public SeekableStreamSupervisor( final ObjectMapper mapper, final SeekableStreamSupervisorSpec spec, final RowIngestionMetersFactory rowIngestionMetersFactory, - final boolean useExclusiveStartingSequence, - final boolean isCheckpointSupported + final boolean useExclusiveStartingSequence ) { this.taskStorage = taskStorage; @@ -511,7 +512,6 @@ public SeekableStreamSupervisor( this.spec = spec; this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.useExclusiveStartingSequence = useExclusiveStartingSequence; - this.isCheckpointSupported = isCheckpointSupported; this.dataSource = spec.getDataSchema().getDataSource(); this.ioConfig = spec.getIoConfig(); @@ -663,7 +663,7 @@ public void stop(boolean stopGracefully) taskRunner.get().unregisterListener(supervisorId); } - // Stopping gracefully will synchronize the end offsets of the tasks and signal them to publish, and will block + // Stopping gracefully will synchronize the end sequences of the tasks and signal them to publish, and will block // until the tasks have acknowledged or timed out. We want this behavior when we're explicitly shut down through // the API, but if we shut down for other reasons (e.g. we lose leadership) we want to just stop and leave the // tasks as they are. @@ -819,7 +819,7 @@ private SupervisorReport> taskReports = new ArrayList<>(); try { - for (TaskGroup taskGroup : taskGroups.values()) { + for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) { for (Entry entry : taskGroup.tasks.entrySet()) { String taskId = entry.getKey(); @Nullable @@ -916,8 +916,8 @@ private Map> getCurrentTotalStats() final List> futures = new ArrayList<>(); final List> groupAndTaskIds = new ArrayList<>(); - for (int groupId : taskGroups.keySet()) { - TaskGroup group = taskGroups.get(groupId); + for (int groupId : activelyReadingTaskGroups.keySet()) { + TaskGroup group = activelyReadingTaskGroups.get(groupId); for (String taskId : group.taskIds()) { futures.add( Futures.transform( @@ -987,7 +987,7 @@ protected void addTaskGroupToActivelyReadingTaskGroup( exclusiveStartingSequencePartitions ); group.tasks.putAll(tasks.stream().collect(Collectors.toMap(x -> x, x -> new TaskData()))); - if (taskGroups.putIfAbsent(taskGroupId, group) != null) { + if (activelyReadingTaskGroups.putIfAbsent(taskGroupId, group) != null) { throw new ISE( "trying to add taskGroup with ID [%s] to actively reading task groups, but group already exists.", taskGroupId @@ -1087,7 +1087,7 @@ public void statusChanged(String taskId, TaskStatus status) @VisibleForTesting protected void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException { - for (TaskGroup taskGroup : taskGroups.values()) { + for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) { for (Entry entry : taskGroup.tasks.entrySet()) { if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) { @@ -1108,8 +1108,8 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) // Reset everything boolean result = indexerMetadataStorageCoordinator.deleteDataSourceMetadata(dataSource); log.info("Reset dataSource[%s] - dataSource metadata entry deleted? [%s]", dataSource, result); - taskGroups.values().forEach(this::killTasksInGroup); - taskGroups.clear(); + activelyReadingTaskGroups.values().forEach(this::killTasksInGroup); + activelyReadingTaskGroups.clear(); partitionGroups.clear(); } else { @@ -1123,7 +1123,7 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) @SuppressWarnings("unchecked") final SeekableStreamDataSourceMetadata resetMetadata = (SeekableStreamDataSourceMetadata) dataSourceMetadata; - if (resetMetadata.getSeekableStreamPartitions().getStream().equals(ioConfig.getId())) { + if (resetMetadata.getSeekableStreamPartitions().getStream().equals(ioConfig.getStream())) { // metadata can be null final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); if (metadata != null && !checkSourceMetadataMatch(metadata)) { @@ -1147,7 +1147,7 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) : currentMetadata.getSeekableStreamPartitions() .getPartitionSequenceNumberMap() .get(resetPartitionOffset.getKey()); - final TaskGroup partitionTaskGroup = taskGroups.get( + final TaskGroup partitionTaskGroup = activelyReadingTaskGroups.get( getTaskGroupIdForPartition(resetPartitionOffset.getKey()) ); final boolean isSameOffset = partitionTaskGroup != null @@ -1181,17 +1181,17 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) resetMetadata.getSeekableStreamPartitions().getPartitionSequenceNumberMap().keySet().forEach(partition -> { final int groupId = getTaskGroupIdForPartition(partition); killTaskGroupForPartitions(ImmutableSet.of(partition)); - taskGroups.remove(groupId); - partitionGroups.get(groupId).replaceAll((partitionId, offset) -> getNotSetMarker()); + activelyReadingTaskGroups.remove(groupId); + partitionGroups.get(groupId).replaceAll((partitionId, sequence) -> getNotSetMarker()); }); } else { throw new ISE("Unable to reset metadata"); } } else { log.warn( - "Reset metadata topic [%s] and supervisor's stream name [%s] do not match", + "Reset metadata stream [%s] and supervisor's stream name [%s] do not match", resetMetadata.getSeekableStreamPartitions().getStream(), - ioConfig.getId() + ioConfig.getStream() ); } } @@ -1223,9 +1223,9 @@ private void killTaskGroupForPartitions(Set partitions) { for (PartitionType partition : partitions) { int taskGroupId = getTaskGroupIdForPartition(partition); - killTasksInGroup(taskGroups.get(taskGroupId)); + killTasksInGroup(activelyReadingTaskGroups.get(taskGroupId)); partitionGroups.remove(taskGroupId); - taskGroups.remove(taskGroupId); + activelyReadingTaskGroups.remove(taskGroupId); } } @@ -1274,22 +1274,22 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti final Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null); if (taskGroupId != null) { - // check to see if we already know about this task, either in [taskGroups] or in [pendingCompletionTaskGroups] - // and if not add it to taskGroups or pendingCompletionTaskGroups (if status = PUBLISHING) - TaskGroup taskGroup = taskGroups.get(taskGroupId); + // check to see if we already know about this task, either in [activelyReadingTaskGroups] or in [pendingCompletionTaskGroups] + // and if not add it to activelyReadingTaskGroups or pendingCompletionTaskGroups (if status = PUBLISHING) + TaskGroup taskGroup = activelyReadingTaskGroups.get(taskGroupId); if (!isTaskInPendingCompletionGroups(taskId) && (taskGroup == null || !taskGroup.tasks.containsKey(taskId))) { futureTaskIds.add(taskId); futures.add( Futures.transform( - taskClient.getStatusAsync(taskId), new Function() + taskClient.getStatusAsync(taskId), new Function() { @Override - public Boolean apply(SeekableStreamIndexTask.Status status) + public Boolean apply(SeekableStreamIndexTaskRunner.Status status) { try { log.debug("Task [%s], status [%s]", taskId, status); - if (status == SeekableStreamIndexTask.Status.PUBLISHING) { + if (status == SeekableStreamIndexTaskRunner.Status.PUBLISHING) { seekableStreamIndexTask.getIOConfig() .getStartPartitions() .getPartitionSequenceNumberMap() @@ -1303,13 +1303,13 @@ public Boolean apply(SeekableStreamIndexTask.Status status) .getPartitionSequenceNumberMap() )); - // update partitionGroups with the publishing task's offsets (if they are greater than what is + // update partitionGroups with the publishing task's sequences (if they are greater than what is // existing) so that the next tasks will start reading from where this task left off Map publishingTaskEndOffsets = taskClient.getEndOffsets(taskId); for (Entry entry : publishingTaskEndOffsets.entrySet()) { PartitionType partition = entry.getKey(); - SequenceType offset = entry.getValue(); + SequenceType sequence = entry.getValue(); ConcurrentHashMap partitionOffsets = partitionGroups.get( getTaskGroupIdForPartition(partition) ); @@ -1317,10 +1317,10 @@ public Boolean apply(SeekableStreamIndexTask.Status status) boolean succeeded; do { succeeded = true; - SequenceType previousOffset = partitionOffsets.putIfAbsent(partition, offset); + SequenceType previousOffset = partitionOffsets.putIfAbsent(partition, sequence); if (previousOffset != null - && (makeSequenceNumber(previousOffset).compareTo(makeSequenceNumber(offset))) < 0) { - succeeded = partitionOffsets.replace(partition, previousOffset, offset); + && (makeSequenceNumber(previousOffset).compareTo(makeSequenceNumber(sequence))) < 0) { + succeeded = partitionOffsets.replace(partition, previousOffset, sequence); } } while (!succeeded); } @@ -1358,7 +1358,7 @@ public Boolean apply(SeekableStreamIndexTask.Status status) } return false; } else { - final TaskGroup taskGroup = taskGroups.computeIfAbsent( + final TaskGroup taskGroup = activelyReadingTaskGroups.computeIfAbsent( taskGroupId, k -> { log.info("Creating a new task group for taskGroupId[%d]", taskGroupId); @@ -1413,9 +1413,8 @@ public Boolean apply(SeekableStreamIndexTask.Status status) log.debug("Found [%d] seekablestream indexing tasks for dataSource [%s]", taskCount, dataSource); // make sure the checkpoints are consistent with each other and with the metadata store - if (isCheckpointSupported) { - verifyAndMergeCheckpoints(taskGroupsToVerify.values()); - } + + verifyAndMergeCheckpoints(taskGroupsToVerify.values()); } @@ -1438,7 +1437,7 @@ private void verifyAndMergeCheckpoints(final Collection taskGroupsToV * 1. Makes sure the checkpoints information in the taskGroup is consistent with that of the tasks, if not kill * inconsistent tasks. * 2. truncates the checkpoints in the taskGroup corresponding to which segments have been published, so that any newly - * created tasks for the taskGroup start indexing from after the latest published offsets. + * created tasks for the taskGroup start indexing from after the latest published sequences. */ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) { @@ -1497,7 +1496,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) final boolean hasValidOffsetsFromDb = latestDataSourceMetadata != null && latestDataSourceMetadata.getSeekableStreamPartitions() != null && - ioConfig.getId().equals( + ioConfig.getStream().equals( latestDataSourceMetadata.getSeekableStreamPartitions().getStream() ); final Map latestOffsetsFromDb; @@ -1523,7 +1522,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) if (taskCheckpoints.entrySet().stream().anyMatch( sequenceCheckpoint -> sequenceCheckpoint.getValue().entrySet().stream().allMatch( partitionOffset -> { - OrderedSequenceNumber offset = makeSequenceNumber(partitionOffset.getValue()); + OrderedSequenceNumber sequence = makeSequenceNumber(partitionOffset.getValue()); OrderedSequenceNumber latestOffset = makeSequenceNumber( latestOffsetsFromDb == null ? partitionOffset.getValue() : latestOffsetsFromDb.getOrDefault( @@ -1534,7 +1533,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) ) ); - return offset.equals(latestOffset); + return sequence.equals(latestOffset); } ) && earliestConsistentSequenceId.compareAndSet(-1, sequenceCheckpoint.getKey())) || ( pendingCompletionTaskGroups.getOrDefault(groupId, EMPTY_LIST).size() > 0 @@ -1576,17 +1575,17 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) if ((tasksToKill.size() > 0 && tasksToKill.size() == taskGroup.tasks.size()) || (taskGroup.tasks.size() == 0 && pendingCompletionTaskGroups.getOrDefault(groupId, EMPTY_LIST).size() == 0)) { // 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 sequence information is fetched from metadata store log.warn("Clearing task group [%d] information as no valid tasks left the group", groupId); - taskGroups.remove(groupId); - partitionGroups.get(groupId).replaceAll((partition, offset) -> getNotSetMarker()); + activelyReadingTaskGroups.remove(groupId); + partitionGroups.get(groupId).replaceAll((partition, sequence) -> getNotSetMarker()); } taskSequences.stream().filter(taskIdSequences -> tasksToKill.contains(taskIdSequences.lhs)).forEach( sequenceCheckpoint -> { log.warn( "Killing task [%s], as its checkpoints [%s] are not consistent with group checkpoints[%s] or latest " - + "persisted offsets in metadata store [%s]", + + "persisted sequences in metadata store [%s]", sequenceCheckpoint.lhs, sequenceCheckpoint.rhs, taskGroup.checkpointSequences, @@ -1666,9 +1665,9 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) .get(); String taskSequenceName = task.getIOConfig().getBaseSequenceName(); - if (taskGroups.get(taskGroupId) != null) { + if (activelyReadingTaskGroups.get(taskGroupId) != null) { return Preconditions - .checkNotNull(taskGroups.get(taskGroupId), "null taskGroup for taskId[%s]", taskGroupId) + .checkNotNull(activelyReadingTaskGroups.get(taskGroupId), "null taskGroup for taskId[%s]", taskGroupId) .baseSequenceName .equals(taskSequenceName); } else { @@ -1722,20 +1721,20 @@ private void updatePartitionDataFromStream() Set partitionIds; try { synchronized (recordSupplierLock) { - partitionIds = recordSupplier.getPartitionIds(ioConfig.getId()); + partitionIds = recordSupplier.getPartitionIds(ioConfig.getStream()); } } catch (Exception e) { - log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getId()); + log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getStream()); return; } if (partitionIds == null || partitionIds.size() == 0) { - log.warn("No partitions found for stream[%s]", ioConfig.getId()); + log.warn("No partitions found for stream[%s]", ioConfig.getStream()); return; } - log.debug("Found [%d] partitions for stream [%s]", partitionIds.size(), ioConfig.getId()); + log.debug("Found [%d] partitions for stream [%s]", partitionIds.size(), ioConfig.getStream()); Set closedPartitions = getOffsetsFromMetadataStorage() .entrySet() @@ -1765,9 +1764,9 @@ private void updatePartitionDataFromStream() if (partitionMap.putIfAbsent(partitionId, getNotSetMarker()) == null) { log.info( - "New partition [%s] discovered for topic [%s], added to task group [%d]", + "New partition [%s] discovered for stream [%s], added to task group [%d]", partitionId, - ioConfig.getId(), + ioConfig.getStream(), taskGroupId ); } @@ -1779,8 +1778,8 @@ private void updateTaskStatus() throws ExecutionException, InterruptedException, final List> futures = new ArrayList<>(); final List futureTaskIds = new ArrayList<>(); - // update status (and startTime if unknown) of current tasks in taskGroups - for (TaskGroup group : taskGroups.values()) { + // update status (and startTime if unknown) of current tasks in activelyReadingTaskGroups + for (TaskGroup group : activelyReadingTaskGroups.values()) { for (Entry entry : group.tasks.entrySet()) { final String taskId = entry.getKey(); final TaskData taskData = entry.getValue(); @@ -1846,7 +1845,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException final List>> futures = new ArrayList<>(); final List futureGroupIds = new ArrayList<>(); - for (Entry entry : taskGroups.entrySet()) { + for (Entry entry : activelyReadingTaskGroups.entrySet()) { Integer groupId = entry.getKey(); TaskGroup group = entry.getValue(); @@ -1879,7 +1878,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException .get(futureTimeoutInSeconds, TimeUnit.SECONDS); for (int j = 0; j < results.size(); j++) { Integer groupId = futureGroupIds.get(j); - TaskGroup group = taskGroups.get(groupId); + TaskGroup group = activelyReadingTaskGroups.get(groupId); Map endOffsets = results.get(j); if (endOffsets != null) { @@ -1900,13 +1899,13 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException for (String id : group.taskIds()) { killTask(id); } - // clear partitionGroups, so that latest offsets from db is used as start offsets not the stale ones + // clear partitionGroups, so that latest sequences from db is used as start sequences not the stale ones // if tasks did some successful incremental handoffs - partitionGroups.get(groupId).replaceAll((partition, offset) -> getNotSetMarker()); + partitionGroups.get(groupId).replaceAll((partition, sequence) -> getNotSetMarker()); } // remove this task group from the list of current task groups now that it has been handled - taskGroups.remove(groupId); + activelyReadingTaskGroups.remove(groupId); } } @@ -1926,7 +1925,7 @@ private ListenableFuture> checkpointTaskGroup( if (task.status != null) { if (task.status.isSuccess()) { // If any task in this group has already completed, stop the rest of the tasks in the group and return. - // This will cause us to create a new set of tasks next cycle that will start from the offsets in + // This will cause us to create a new set of tasks next cycle that will start from the sequences in // 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) return Futures.transform( @@ -1969,7 +1968,7 @@ public Map apply(@Nullable Object input) @Override public Map apply(List> input) { - // 3) Build a map of the highest offset read by any task in the group for each partition + // 3) Build a map of the highest sequence read by any task in the group for each partition final Map endOffsets = new HashMap<>(); for (int i = 0; i < input.size(); i++) { Map result = input.get(i); @@ -1980,18 +1979,18 @@ public Map apply(List offset : result.entrySet()) { - if (!endOffsets.containsKey(offset.getKey()) - || makeSequenceNumber(endOffsets.get(offset.getKey())).compareTo( - makeSequenceNumber(offset.getValue())) < 0) { - endOffsets.put(offset.getKey(), offset.getValue()); + } else { // otherwise build a map of the highest sequences seen + for (Entry sequence : result.entrySet()) { + if (!endOffsets.containsKey(sequence.getKey()) + || makeSequenceNumber(endOffsets.get(sequence.getKey())).compareTo( + makeSequenceNumber(sequence.getValue())) < 0) { + endOffsets.put(sequence.getKey(), sequence.getValue()); } } } } - // 4) Set the end offsets for each task to the values from step 3 and resume the tasks. All the tasks should + // 4) Set the end sequences for each task to the values from step 3 and resume the tasks. All the tasks should // finish reading and start publishing within a short period, depending on how in sync the tasks were. final List> setEndOffsetFutures = new ArrayList<>(); final List setEndOffsetTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); @@ -2005,7 +2004,7 @@ public Map apply(List apply(List getNotSetMarker()); + // reset partitions sequences for this task group so that they will be re-read from metadata storage + partitionGroups.get(groupId).replaceAll((partition, sequence) -> getNotSetMarker()); // kill all the tasks in this pending completion group killTasksInGroup(group); // set a flag so the other pending completion groups for this set of partitions will also stop stopTasksInTaskGroup = true; // kill all the tasks in the currently reading task group and remove the bad task group - killTasksInGroup(taskGroups.remove(groupId)); + killTasksInGroup(activelyReadingTaskGroups.remove(groupId)); toRemove.add(group); } } @@ -2158,15 +2157,15 @@ private void checkPendingCompletionTasks() private void checkCurrentTaskState() throws ExecutionException, InterruptedException, TimeoutException { List> futures = new ArrayList<>(); - Iterator> iTaskGroups = taskGroups.entrySet().iterator(); + Iterator> iTaskGroups = activelyReadingTaskGroups.entrySet().iterator(); while (iTaskGroups.hasNext()) { Entry taskGroupEntry = iTaskGroups.next(); Integer groupId = taskGroupEntry.getKey(); TaskGroup taskGroup = taskGroupEntry.getValue(); // Iterate the list of known tasks in this group and: - // 1) Kill any tasks which are not "current" (have the partitions, starting offsets, and minimumMessageTime - // & maximumMessageTime (if applicable) in [taskGroups]) + // 1) Kill any tasks which are not "current" (have the partitions, starting sequences, and minimumMessageTime + // & maximumMessageTime (if applicable) in [activelyReadingTaskGroups]) // 2) Remove any tasks that have failed from the list // 3) If any task completed successfully, stop all the tasks in this group and move to the next group @@ -2180,7 +2179,7 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep // stop and remove bad tasks from the task group if (!isTaskCurrent(groupId, taskId)) { - log.info("Stopping task [%s] which does not match the expected offset range and ingestion spec", taskId); + log.info("Stopping task [%s] which does not match the expected sequence range and ingestion spec", taskId); futures.add(stopTask(taskId, false)); iTasks.remove(); continue; @@ -2195,7 +2194,7 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep } // check for successful tasks, and if we find one, stop all tasks in the group and remove the group so it can - // be recreated with the next set of offsets + // be recreated with the next set of sequences if (taskData.status.isSuccess()) { futures.add(stopTasksInGroup(taskGroup)); iTaskGroups.remove(); @@ -2212,18 +2211,16 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep private void createNewTasks() throws JsonProcessingException { // update the checkpoints in the taskGroup to latest ones so that new tasks do not read what is already published - if (isCheckpointSupported) { - verifyAndMergeCheckpoints( - taskGroups.values() - .stream() - .filter(taskGroup -> taskGroup.tasks.size() < ioConfig.getReplicas()) - .collect(Collectors.toList()) - ); - } + verifyAndMergeCheckpoints( + activelyReadingTaskGroups.values() + .stream() + .filter(taskGroup -> taskGroup.tasks.size() < ioConfig.getReplicas()) + .collect(Collectors.toList()) + ); // check that there is a current task group for each group of partitions in [partitionGroups] for (Integer groupId : partitionGroups.keySet()) { - if (!taskGroups.containsKey(groupId)) { + if (!activelyReadingTaskGroups.containsKey(groupId)) { log.info("Creating new task group [%d] for partitions %s", groupId, partitionGroups.get(groupId).keySet()); Optional minimumMessageTime = (ioConfig.getLateMessageRejectionPeriod().isPresent() ? Optional.of( @@ -2234,52 +2231,46 @@ private void createNewTasks() throws JsonProcessingException DateTimes.nowUtc().plus(ioConfig.getTaskDuration()).plus(ioConfig.getEarlyMessageRejectionPeriod().get()) ) : Optional.absent()); - try { - Map> startingOffsets = generateStartingSequencesForPartitionGroup( - groupId); - - ImmutableMap simpleStartingOffsets = startingOffsets - .entrySet() - .stream() - .filter(x -> x.getValue().get() != null) - .collect(Collectors.collectingAndThen( - Collectors.toMap(Entry::getKey, x -> x.getValue().get()), - ImmutableMap::copyOf - )); - - Set exclusiveStartSequenceNumberPartitions = !useExclusiveStartingSequence - ? Collections.emptySet() - : startingOffsets - .entrySet().stream() - .filter(x -> x.getValue().get() != null - && x.getValue().isExclusive()) - .map(Entry::getKey) - .collect(Collectors.toSet()); - - taskGroups.put( - groupId, - new TaskGroup( - groupId, - simpleStartingOffsets, - minimumMessageTime, - maximumMessageTime, - exclusiveStartSequenceNumberPartitions - ) - ); - } - catch (TimeoutException e) { - log.warn( - e, - "Timeout while fetching sequence numbers - if you are reading from the latest sequence number, you need to write events to the stream before the sequence number can be determined" - ); - } + Map> startingOffsets = generateStartingSequencesForPartitionGroup( + groupId); + + ImmutableMap simpleStartingOffsets = startingOffsets + .entrySet() + .stream() + .filter(x -> x.getValue().get() != null) + .collect(Collectors.collectingAndThen( + Collectors.toMap(Entry::getKey, x -> x.getValue().get()), + ImmutableMap::copyOf + )); + + Set exclusiveStartSequenceNumberPartitions = !useExclusiveStartingSequence + ? Collections.emptySet() + : startingOffsets + .entrySet() + .stream() + .filter(x -> x.getValue().get() != null + && x.getValue().isExclusive()) + .map(Entry::getKey) + .collect(Collectors.toSet()); + + activelyReadingTaskGroups.put( + groupId, + new TaskGroup( + groupId, + simpleStartingOffsets, + minimumMessageTime, + maximumMessageTime, + exclusiveStartSequenceNumberPartitions + ) + ); + } } // iterate through all the current task groups and make sure each one has the desired number of replica tasks boolean createdTask = false; - for (Entry entry : taskGroups.entrySet()) { + for (Entry entry : activelyReadingTaskGroups.entrySet()) { TaskGroup taskGroup = entry.getValue(); Integer groupId = entry.getKey(); @@ -2316,13 +2307,13 @@ protected void addNotice(Notice notice) @Nullable protected TaskGroup removeTaskGroup(int taskGroupId) { - return taskGroups.remove(taskGroupId); + return activelyReadingTaskGroups.remove(taskGroupId); } @VisibleForTesting protected void moveTaskGroupToPendingCompletion(int taskGroupId) { - final TaskGroup taskGroup = taskGroups.remove(taskGroupId); + final TaskGroup taskGroup = activelyReadingTaskGroups.remove(taskGroupId); if (taskGroup != null) { pendingCompletionTaskGroups.computeIfAbsent(taskGroupId, k -> new CopyOnWriteArrayList<>()).add(taskGroup); } @@ -2337,21 +2328,20 @@ protected int getNoticesQueueSize() private ImmutableMap> generateStartingSequencesForPartitionGroup( int groupId ) - throws TimeoutException { ImmutableMap.Builder> builder = ImmutableMap.builder(); for (Entry entry : partitionGroups.get(groupId).entrySet()) { PartitionType partition = entry.getKey(); - SequenceType offset = entry.getValue(); + SequenceType sequence = entry.getValue(); - if (!getNotSetMarker().equals(offset)) { + if (!getNotSetMarker().equals(sequence)) { // if we are given a startingOffset (set by a previous task group which is pending completion) then use it - if (!OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(offset)) { - builder.put(partition, makeSequenceNumber(offset, true)); + if (!OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(sequence)) { + builder.put(partition, makeSequenceNumber(sequence, false)); } } else { - // if we don't have a startingOffset (first run or we had some previous failures and reset the offsets) then - // get the offset from metadata storage (if available) or Kafka/Kinesis (otherwise) + // if we don't have a startingOffset (first run or we had some previous failures and reset the sequences) then + // get the sequence from metadata storage (if available) or Kafka/Kinesis (otherwise) OrderedSequenceNumber offsetFromStorage = getOffsetFromStorageForPartition(partition); if (offsetFromStorage != null) { @@ -2363,43 +2353,37 @@ private ImmutableMap> generat } /** - * Queries the dataSource metadata table to see if there is a previous ending offset for this partition. If it doesn't - * find any data, it will retrieve the latest or earliest Kafka/Kinesis offset depending on the useEarliestOffset config. + * Queries the dataSource metadata table to see if there is a previous ending sequence for this partition. If it doesn't + * find any data, it will retrieve the latest or earliest Kafka/Kinesis sequence depending on the useEarliestOffset config. */ private OrderedSequenceNumber getOffsetFromStorageForPartition(PartitionType partition) - throws TimeoutException { final Map metadataOffsets = getOffsetsFromMetadataStorage(); - SequenceType offset = metadataOffsets.get(partition); - if (offset != null) { - log.debug("Getting offset [%s] from metadata storage for partition [%s]", offset, partition); + SequenceType sequence = metadataOffsets.get(partition); + if (sequence != null) { + log.debug("Getting sequence [%s] from metadata storage for partition [%s]", sequence, partition); if (!taskTuningConfig.isSkipSequenceNumberAvailabilityCheck()) { - try { - if (!checkSequenceAvailability(partition, offset)) { - if (taskTuningConfig.isResetOffsetAutomatically()) { - resetInternal( - createDataSourceMetaData(ioConfig.getId(), ImmutableMap.of(partition, offset)) - ); - throw new ISE( - "Previous sequenceNumber [%s] is no longer available for partition [%s] - automatically resetting offset", - offset, - partition - ); + if (!checkSequenceAvailability(partition, sequence)) { + if (taskTuningConfig.isResetOffsetAutomatically()) { + resetInternal( + createDataSourceMetaData(ioConfig.getStream(), ImmutableMap.of(partition, sequence)) + ); + throw new ISE( + "Previous sequenceNumber [%s] is no longer available for partition [%s] - automatically resetting sequence", + sequence, + partition + ); - } else { - throw new ISE( - "Previous sequenceNumber [%s] is no longer available for partition [%s]. You can clear the previous sequenceNumber and start reading from a valid message by using the supervisor's reset API.", - offset, - partition - ); - } + } else { + throw new ISE( + "Previous sequenceNumber [%s] is no longer available for partition [%s]. You can clear the previous sequenceNumber and start reading from a valid message by using the supervisor's reset API.", + sequence, + partition + ); } } - catch (TimeoutException e) { - throw new ISE(e, "Timeout while fetching earliest sequence number for partition [%s]", partition); - } } - return makeSequenceNumber(offset, true); + return makeSequenceNumber(sequence, true); } else { boolean useEarliestSequenceNumber = ioConfig.isUseEarliestSequenceNumber(); if (subsequentlyDiscoveredPartitions.contains(partition)) { @@ -2410,9 +2394,12 @@ private OrderedSequenceNumber getOffsetFromStorageForPartition(Par useEarliestSequenceNumber = true; } - offset = getOffsetFromStreamForPartition(partition, useEarliestSequenceNumber); - log.info("Getting sequence number [%s] for partition [%s]", offset, partition); - return makeSequenceNumber(offset, false); + sequence = getOffsetFromStreamForPartition(partition, useEarliestSequenceNumber); + if (sequence == null) { + throw new ISE("unable to fetch sequence number for partition[%s] from stream", partition); + } + log.info("Getting sequence number [%s] for partition [%s]", sequence, partition); + return makeSequenceNumber(sequence, false); } } @@ -2425,11 +2412,11 @@ && checkSourceMetadataMatch(dataSourceMetadata)) { SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata) .getSeekableStreamPartitions(); if (partitions != null) { - if (!ioConfig.getId().equals(partitions.getStream())) { + if (!ioConfig.getStream().equals(partitions.getStream())) { log.warn( - "Topic/stream in metadata storage [%s] doesn't match spec topic/stream [%s], ignoring stored offsets", + "Topic/stream in metadata storage [%s] doesn't match spec topic/stream [%s], ignoring stored sequences", partitions.getStream(), - ioConfig.getId() + ioConfig.getStream() ); return Collections.emptyMap(); } else if (partitions.getPartitionSequenceNumberMap() != null) { @@ -2441,11 +2428,11 @@ && checkSourceMetadataMatch(dataSourceMetadata)) { return Collections.emptyMap(); } - protected SequenceType getOffsetFromStreamForPartition(PartitionType partition, boolean useEarliestOffset) - throws TimeoutException + @Nullable + private SequenceType getOffsetFromStreamForPartition(PartitionType partition, boolean useEarliestOffset) { synchronized (recordSupplierLock) { - StreamPartition topicPartition = new StreamPartition<>(ioConfig.getId(), partition); + StreamPartition topicPartition = new StreamPartition<>(ioConfig.getStream(), partition); if (!recordSupplier.getAssignment().contains(topicPartition)) { recordSupplier.assign(Collections.singleton(topicPartition)); } @@ -2458,16 +2445,16 @@ protected SequenceType getOffsetFromStreamForPartition(PartitionType partition, private void createTasksForGroup(int groupId, int replicas) throws JsonProcessingException { - TaskGroup group = taskGroups.get(groupId); + TaskGroup group = activelyReadingTaskGroups.get(groupId); Map startPartitions = group.startingSequences; Map endPartitions = new HashMap<>(); for (PartitionType partition : startPartitions.keySet()) { endPartitions.put(partition, getEndOfPartitionMarker()); } - Set exclusiveStartSequenceNumberPartitions = taskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; + Set exclusiveStartSequenceNumberPartitions = activelyReadingTaskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; - DateTime minimumMessageTime = taskGroups.get(groupId).minimumMessageTime.orNull(); - DateTime maximumMessageTime = taskGroups.get(groupId).maximumMessageTime.orNull(); + DateTime minimumMessageTime = activelyReadingTaskGroups.get(groupId).minimumMessageTime.orNull(); + DateTime maximumMessageTime = activelyReadingTaskGroups.get(groupId).maximumMessageTime.orNull(); SeekableStreamIOConfig newIoConfig = createIoConfig( groupId, @@ -2517,7 +2504,7 @@ protected Runnable updateCurrentAndLatestOffsets() sequenceLastUpdated = DateTimes.nowUtc(); } catch (Exception e) { - log.warn(e, "Exception while getting current/latest offsets"); + log.warn(e, "Exception while getting current/latest sequences"); } }; } @@ -2525,7 +2512,7 @@ protected Runnable updateCurrentAndLatestOffsets() private void updateCurrentOffsets() throws InterruptedException, ExecutionException, TimeoutException { final List> futures = Stream.concat( - taskGroups.values().stream().flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()), + activelyReadingTaskGroups.values().stream().flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()), pendingCompletionTaskGroups.values() .stream() .flatMap(List::stream) @@ -2552,16 +2539,16 @@ private void updateLatestOffsetsFromStream() synchronized (recordSupplierLock) { Set partitionIds = null; try { - partitionIds = recordSupplier.getPartitionIds(ioConfig.getId()); + partitionIds = recordSupplier.getPartitionIds(ioConfig.getStream()); } catch (Exception e) { - log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getId()); + log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getStream()); Throwables.propagate(e); } Set> partitions = partitionIds .stream() - .map(e -> new StreamPartition<>(ioConfig.getId(), e)) + .map(e -> new StreamPartition<>(ioConfig.getStream(), e)) .collect(Collectors.toSet()); recordSupplier.assign(partitions); @@ -2570,7 +2557,7 @@ private void updateLatestOffsetsFromStream() latestSequenceFromStream = partitions.stream() .collect(Collectors.toMap( StreamPartition::getPartitionId, - x -> recordSupplier.position(x) + x -> recordSupplier.getPosition(x) )); } @@ -2578,7 +2565,7 @@ private void updateLatestOffsetsFromStream() protected Map getHighestCurrentOffsets() { - return taskGroups + return activelyReadingTaskGroups .values() .stream() .flatMap(taskGroup -> taskGroup.tasks.entrySet().stream()) @@ -2625,12 +2612,35 @@ public SeekableStreamSupervisorIOConfig getIoConfig() } @Override - public abstract void checkpoint( + public void checkpoint( @Nullable Integer taskGroupId, - String baseSequenceName, + @Deprecated String baseSequenceName, DataSourceMetadata previousCheckPoint, DataSourceMetadata currentCheckPoint - ); + ) + { + Preconditions.checkNotNull(previousCheckPoint, "previousCheckpoint"); + Preconditions.checkNotNull(currentCheckPoint, "current checkpoint cannot be null"); + Preconditions.checkArgument( + spec.getIoConfig() + .getStream() + .equals(((SeekableStreamDataSourceMetadata) currentCheckPoint).getSeekableStreamPartitions() + .getStream()), + "Supervisor stream [%s] and stream in checkpoint [%s] does not match", + spec.getIoConfig().getStream(), + ((SeekableStreamDataSourceMetadata) currentCheckPoint).getSeekableStreamPartitions().getStream() + ); + + log.info("Checkpointing [%s] for taskGroup [%s]", currentCheckPoint, taskGroupId); + addNotice( + new CheckpointNotice( + taskGroupId, + baseSequenceName, + (SeekableStreamDataSourceMetadata) previousCheckPoint, + (SeekableStreamDataSourceMetadata) currentCheckPoint + ) + ); + } /** * creates a specific task IOConfig instance for Kafka/Kinesis* @@ -2755,14 +2765,20 @@ protected abstract SeekableStreamSupervisorReportPayload= 0); + } + protected abstract SequenceType getNotSetMarker(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java index 5be635b4dec2..e6c0baa1cbef 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java @@ -27,7 +27,7 @@ public abstract class SeekableStreamSupervisorIOConfig { - private final String id; + private final String stream; private final Integer replicas; private final Integer taskCount; private final Duration taskDuration; @@ -39,7 +39,7 @@ public abstract class SeekableStreamSupervisorIOConfig private final Optional earlyMessageRejectionPeriod; public SeekableStreamSupervisorIOConfig( - String id, + String stream, Integer replicas, Integer taskCount, Period taskDuration, @@ -51,7 +51,7 @@ public SeekableStreamSupervisorIOConfig( Period earlyMessageRejectionPeriod ) { - this.id = Preconditions.checkNotNull(id, "id cannot be null"); + this.stream = Preconditions.checkNotNull(stream, "stream cannot be null"); this.replicas = replicas != null ? replicas : 1; this.taskCount = taskCount != null ? taskCount : 1; this.taskDuration = defaultDuration(taskDuration, "PT1H"); @@ -72,9 +72,9 @@ private static Duration defaultDuration(final Period period, final String theDef return (period == null ? new Period(theDefault) : period).toStandardDuration(); } - public String getId() + public String getStream() { - return id; + return stream; } @JsonProperty From 810fb78f12ebf5c3bcc976284a3e3318bde690e0 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 14 Nov 2018 13:41:50 -0800 Subject: [PATCH 42/87] comments --- .../kafka/supervisor/KafkaSupervisor.java | 2 +- .../kinesis/supervisor/KinesisSupervisor.java | 2 +- .../supervisor/SeekableStreamSupervisor.java | 76 +++++++------------ 3 files changed, 30 insertions(+), 50 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 9779249a0cff..e92ad874ec1e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -163,7 +163,7 @@ protected boolean checkSourceMetadataMatch(DataSourceMetadata metadata) } @Override - protected boolean checkTaskInstance(Task task) + protected boolean doesTaskTypeMatchSupervisor(Task task) { return task instanceof KafkaIndexTask; } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 52334af7c17d..d07e2d4308a3 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -218,7 +218,7 @@ protected boolean checkSourceMetadataMatch(DataSourceMetadata metadata) } @Override - protected boolean checkTaskInstance(Task task) + protected boolean doesTaskTypeMatchSupervisor(Task task) { return task instanceof KinesisIndexTask; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 5ac3d55d63b1..7cffdb26f1e4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -718,7 +718,7 @@ protected void tryInit() { synchronized (stateChangeLock) { if (started) { - log.warn("SUpervisor was already started, skipping init"); + log.warn("Supervisor was already started, skipping init"); return; } @@ -1053,33 +1053,28 @@ private void possiblyRegisterListener() Optional taskRunner = taskMaster.getTaskRunner(); if (taskRunner.isPresent()) { - try { - taskRunner.get().registerListener( - new TaskRunnerListener() + taskRunner.get().registerListener( + new TaskRunnerListener() + { + @Override + public String getListenerId() { - @Override - public String getListenerId() - { - return supervisorId; - } + return supervisorId; + } - @Override - public void locationChanged(final String taskId, final TaskLocation newLocation) - { - // do nothing - } + @Override + public void locationChanged(final String taskId, final TaskLocation newLocation) + { + // do nothing + } - @Override - public void statusChanged(String taskId, TaskStatus status) - { - notices.add(new RunNotice()); - } - }, MoreExecutors.sameThreadExecutor() - ); - } - catch (ISE e) { - log.info("listener already registered with taskrunner."); - } + @Override + public void statusChanged(String taskId, TaskStatus status) + { + notices.add(new RunNotice()); + } + }, MoreExecutors.sameThreadExecutor() + ); listenerRegistered = true; } } @@ -1088,8 +1083,7 @@ public void statusChanged(String taskId, TaskStatus status) protected void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException { for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) { - for (Entry entry : - taskGroup.tasks.entrySet()) { + for (Entry entry : taskGroup.tasks.entrySet()) { if (taskInfoProvider.getTaskLocation(entry.getKey()).equals(TaskLocation.unknown())) { killTask(entry.getKey()); } else { @@ -1250,7 +1244,7 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti final Map taskGroupsToVerify = new HashMap<>(); for (Task task : tasks) { - if (!checkTaskInstance(task) || !dataSource.equals(task.getDataSource())) { + if (!doesTaskTypeMatchSupervisor(task) || !dataSource.equals(task.getDataSource())) { continue; } @@ -1656,7 +1650,7 @@ public Void apply(@Nullable Boolean result) private boolean isTaskCurrent(int taskGroupId, String taskId) { Optional taskOptional = taskStorage.getTask(taskId); - if (!taskOptional.isPresent() || !checkTaskInstance(taskOptional.get())) { + if (!taskOptional.isPresent() || !doesTaskTypeMatchSupervisor(taskOptional.get())) { return false; } @@ -1719,14 +1713,8 @@ protected String generateSequenceName( private void updatePartitionDataFromStream() { Set partitionIds; - try { - synchronized (recordSupplierLock) { - partitionIds = recordSupplier.getPartitionIds(ioConfig.getStream()); - } - } - catch (Exception e) { - log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getStream()); - return; + synchronized (recordSupplierLock) { + partitionIds = recordSupplier.getPartitionIds(ioConfig.getStream()); } if (partitionIds == null || partitionIds.size() == 0) { @@ -2298,18 +2286,11 @@ private void createNewTasks() throws JsonProcessingException } - protected void addNotice(Notice notice) + private void addNotice(Notice notice) { notices.add(notice); } - @VisibleForTesting - @Nullable - protected TaskGroup removeTaskGroup(int taskGroupId) - { - return activelyReadingTaskGroups.remove(taskGroupId); - } - @VisibleForTesting protected void moveTaskGroupToPendingCompletion(int taskGroupId) { @@ -2478,9 +2459,8 @@ private void createTasksForGroup(int groupId, int replicas) throws JsonProcessin rowIngestionMetersFactory ); - for (int i = 0; i < replicas; i++) { + for (SeekableStreamIndexTask indexTask : taskList) { Optional taskQueue = taskMaster.getTaskQueue(); - SeekableStreamIndexTask indexTask = taskList.get(i); if (taskQueue.isPresent()) { try { taskQueue.get().add(indexTask); @@ -2704,7 +2684,7 @@ protected abstract List> cr * * @return true if isInstance else false */ - protected abstract boolean checkTaskInstance(Task task); + protected abstract boolean doesTaskTypeMatchSupervisor(Task task); /** * creates a specific instance of kafka/kinesis datasource metadata From 25c8638b73fd519659eb65ba08862b71a1e49607 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 14 Nov 2018 14:09:27 -0800 Subject: [PATCH 43/87] github comments --- .../druid/indexing/kafka/KafkaIndexTask.java | 2 +- .../indexing/kafka/KafkaIndexTaskTest.java | 38 +- .../druid/indexing/kafka/test/TestBroker.java | 2 +- .../indexing/kinesis/KinesisIOConfig.java | 2 +- .../indexing/kinesis/KinesisIndexTask.java | 2 +- .../kinesis/KinesisIndexTaskTest.java | 4 +- .../SeekableStreamIndexTask.java | 176 +- .../SeekableStreamIndexTaskRunner.java | 1515 +++++++++-------- .../supervisor/SeekableStreamSupervisor.java | 11 +- 9 files changed, 891 insertions(+), 861 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 80e72ede0869..54511b7b76b5 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -95,7 +95,7 @@ long getPollRetryMs() } @Override - protected RecordSupplier getRecordSupplier() + protected RecordSupplier newRecordSupplier() { ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); try { diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index e460be25ea80..f9165694e995 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -69,7 +69,6 @@ import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskTest; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor; import org.apache.druid.indexing.kafka.test.TestBroker; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -79,6 +78,7 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; import org.apache.druid.indexing.test.TestDataSegmentKiller; import org.apache.druid.java.util.common.CompressionUtils; @@ -548,8 +548,9 @@ public void testIncrementalHandOff() throws Exception Thread.sleep(10); } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets) || checkpoint2.getPartitionSequenceNumberMap() - .equals(currentOffsets)); + Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets) + || checkpoint2.getPartitionSequenceNumberMap() + .equals(currentOffsets)); task.getRunner().setEndOffsets(currentOffsets, false); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); @@ -827,7 +828,10 @@ public void testTimeBasedIncrementalHandOff() throws Exception DATA_SCHEMA.getDataSource(), 0, new KafkaDataSourceMetadata(startPartitions), - new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, checkpoint.getPartitionSequenceNumberMap())) + new KafkaDataSourceMetadata(new SeekableStreamPartitions<>( + topic, + checkpoint.getPartitionSequenceNumberMap() + )) ) ) ); @@ -881,9 +885,18 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception Map consumerProps = kafkaServer.consumerProperties(); consumerProps.put("max.poll.records", "1"); - final KafkaPartitions startPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 0L)); - final KafkaPartitions checkpoint1 = new KafkaPartitions(topic, ImmutableMap.of(0, 5L)); - final KafkaPartitions endPartitions = new KafkaPartitions(topic, ImmutableMap.of(0, 7L)); + final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of(0, 0L) + ); + final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of(0, 5L) + ); + final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( + topic, + ImmutableMap.of(0, 7L) + ); final KafkaIndexTask task = createTask( null, @@ -900,11 +913,11 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception ) ); final ListenableFuture future = runTask(task); - while (task.getRunner().getStatus() != KafkaIndexTask.Status.PAUSED) { + while (task.getRunner().getStatus() != Status.PAUSED) { Thread.sleep(10); } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets)); + Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets)); // actual checkpoint offset is 5, but simulating behavior of publishing set end offset call, to ensure this task // will continue reading through the end offset of the checkpointed sequence @@ -2173,7 +2186,10 @@ private KafkaIndexTask createTask( maxSavedParseExceptions ); final Map context = isIncrementalHandoffSupported - ? ImmutableMap.of(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true) + ? ImmutableMap.of( + SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, + true + ) : null; final KafkaIndexTask task = new KafkaIndexTask( taskId, @@ -2219,7 +2235,7 @@ private KafkaIndexTask createTask( maxSavedParseExceptions ); if (isIncrementalHandoffSupported) { - context.put(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true); + context.put(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true); } final KafkaIndexTask task = new KafkaIndexTask( diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java index f59a8909196a..6659f92ebbf0 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java @@ -83,7 +83,7 @@ public void start() server = new KafkaServer( config, - SystemTime.SYSTEM, + Time.SYSTEM, Some.apply(StringUtils.format("TestingBroker[%d]-", id)), List$.MODULE$.empty() ); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java index 10cde9ea52f0..e6a3be9810ac 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java @@ -76,7 +76,7 @@ public KinesisIOConfig( maximumMessageTime, true ); - Preconditions.checkArgument(endPartitions.getPartitionOffsetMap() + Preconditions.checkArgument(endPartitions.getPartitionSequenceNumberMap() .values() .stream() .noneMatch(x -> x.equals(OrderedPartitionableRecord.END_OF_SHARD_MARKER))); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 5e1e171778d8..f744343a8e66 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -69,7 +69,7 @@ public KinesisIndexTask( } @Override - protected RecordSupplier getRecordSupplier() + protected RecordSupplier newRecordSupplier() { int fetchThreads = tuningConfig.getFetchThreads() != null ? tuningConfig.getFetchThreads() diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index c8319c022a6c..0325c6d493a8 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -657,8 +657,8 @@ public void testIncrementalHandOff() throws Exception Thread.sleep(10); } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertTrue(checkpoint1.getPartitionOffsetMap().equals(currentOffsets) || checkpoint2.getPartitionOffsetMap() - .equals(currentOffsets)); + Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets) || checkpoint2.getPartitionSequenceNumberMap() + .equals(currentOffsets)); task.getRunner().setEndOffsets(currentOffsets, false); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index fa92c0b4b827..e3183f397bf0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -74,12 +74,13 @@ public abstract class SeekableStreamIndexTask exten protected final SeekableStreamTuningConfig tuningConfig; protected final SeekableStreamIOConfig ioConfig; protected final Optional chatHandlerProvider; + private final SeekableStreamIndexTaskRunner runner; protected final String type; protected final Map context; protected final AuthorizerMapper authorizerMapper; protected final RowIngestionMetersFactory rowIngestionMetersFactory; protected CircularBuffer savedParseExceptions; - private final SeekableStreamIndexTaskRunner runner; + @JsonCreator public SeekableStreamIndexTask( @@ -119,8 +120,6 @@ public SeekableStreamIndexTask( this.runner = createTaskRunner(); } - protected abstract SeekableStreamIndexTaskRunner createTaskRunner(); - private static String makeTaskId(String dataSource, int randomBits, String type) { @@ -131,89 +130,6 @@ private static String makeTaskId(String dataSource, int randomBits, String type) return Joiner.on("_").join(type, dataSource, suffix); } - public StreamAppenderatorDriver newDriver( - final Appenderator appenderator, - final TaskToolbox toolbox, - final FireDepartmentMetrics metrics - ) - { - return new StreamAppenderatorDriver( - appenderator, - new ActionBasedSegmentAllocator( - toolbox.getTaskActionClient(), - dataSchema, - (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( - schema.getDataSource(), - row.getTimestamp(), - schema.getGranularitySpec().getQueryGranularity(), - schema.getGranularitySpec().getSegmentGranularity(), - sequenceName, - previousSegmentId, - skipSegmentLineageCheck - ) - ), - toolbox.getSegmentHandoffNotifierFactory(), - new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), - toolbox.getDataSegmentKiller(), - toolbox.getObjectMapper(), - metrics - ); - } - - public Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) - { - return Appenderators.createRealtime( - dataSchema, - tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), - metrics, - toolbox.getSegmentPusher(), - toolbox.getObjectMapper(), - toolbox.getIndexIO(), - toolbox.getIndexMergerV9(), - toolbox.getQueryRunnerFactoryConglomerate(), - toolbox.getSegmentAnnouncer(), - toolbox.getEmitter(), - toolbox.getQueryExecutorService(), - toolbox.getCache(), - toolbox.getCacheConfig(), - toolbox.getCachePopulatorStats() - ); - } - - public boolean withinMinMaxRecordTime(final InputRow row) - { - final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent() - && ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp()); - - final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent() - && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp()); - - if (!Intervals.ETERNITY.contains(row.getTimestamp())) { - final String errorMsg = StringUtils.format( - "Encountered row with timestamp that cannot be represented as a long: [%s]", - row - ); - throw new ParseException(errorMsg); - } - - if (log.isDebugEnabled()) { - if (beforeMinimumMessageTime) { - log.debug( - "CurrentTimeStamp[%s] is before MinimumMessageTime[%s]", - row.getTimestamp(), - ioConfig.getMinimumMessageTime().get() - ); - } else if (afterMaximumMessageTime) { - log.debug( - "CurrentTimeStamp[%s] is after MaximumMessageTime[%s]", - row.getTimestamp(), - ioConfig.getMaximumMessageTime().get() - ); - } - } - - return !beforeMinimumMessageTime && !afterMaximumMessageTime; - } @Override public int getPriority() @@ -280,7 +196,93 @@ public QueryRunner getQueryRunner(Query query) return (queryPlus, responseContext) -> queryPlus.run(runner.getAppenderator(), responseContext); } - protected abstract RecordSupplier getRecordSupplier(); + public Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) + { + return Appenderators.createRealtime( + dataSchema, + tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), + metrics, + toolbox.getSegmentPusher(), + toolbox.getObjectMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMergerV9(), + toolbox.getQueryRunnerFactoryConglomerate(), + toolbox.getSegmentAnnouncer(), + toolbox.getEmitter(), + toolbox.getQueryExecutorService(), + toolbox.getCache(), + toolbox.getCacheConfig(), + toolbox.getCachePopulatorStats() + ); + } + + public StreamAppenderatorDriver newDriver( + final Appenderator appenderator, + final TaskToolbox toolbox, + final FireDepartmentMetrics metrics + ) + { + return new StreamAppenderatorDriver( + appenderator, + new ActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + dataSchema, + (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck + ) + ), + toolbox.getSegmentHandoffNotifierFactory(), + new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), + toolbox.getDataSegmentKiller(), + toolbox.getObjectMapper(), + metrics + ); + } + + protected abstract RecordSupplier newRecordSupplier(); + + public boolean withinMinMaxRecordTime(final InputRow row) + { + final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent() + && ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp()); + + final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent() + && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp()); + + if (!Intervals.ETERNITY.contains(row.getTimestamp())) { + final String errorMsg = StringUtils.format( + "Encountered row with timestamp that cannot be represented as a long: [%s]", + row + ); + throw new ParseException(errorMsg); + } + + if (log.isDebugEnabled()) { + if (beforeMinimumMessageTime) { + log.debug( + "CurrentTimeStamp[%s] is before MinimumMessageTime[%s]", + row.getTimestamp(), + ioConfig.getMinimumMessageTime().get() + ); + } else if (afterMaximumMessageTime) { + log.debug( + "CurrentTimeStamp[%s] is after MaximumMessageTime[%s]", + row.getTimestamp(), + ioConfig.getMaximumMessageTime().get() + ); + } + } + + return !beforeMinimumMessageTime && !afterMaximumMessageTime; + } + + protected abstract SeekableStreamIndexTaskRunner createTaskRunner(); @VisibleForTesting public Appenderator getAppenderator() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index fdb6b72e4333..197decc3a4c8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -132,6 +132,15 @@ */ public abstract class SeekableStreamIndexTaskRunner implements ChatHandler { + public enum Status + { + NOT_STARTED, + STARTING, + READING, + PAUSED, + PUBLISHING + } + private final EmittingLogger log = new EmittingLogger(this.getClass()); private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; private static final String METADATA_PUBLISH_PARTITIONS = "publishPartitions"; @@ -140,12 +149,6 @@ public abstract class SeekableStreamIndexTaskRunner private final Map currOffsets = new ConcurrentHashMap<>(); private final Map lastPersistedOffsets = new ConcurrentHashMap<>(); - private final Set publishingSequences = Sets.newConcurrentHashSet(); - private final List> publishWaitList = new ArrayList<>(); - private final List> handOffWaitList = new ArrayList<>(); - private final Map initialOffsetsSnapshot = new HashMap<>(); - - // The pause lock and associated conditions are to support coordination between the Jetty threads and the main // ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully // the ingestion loop has been stopped at the returned sequences and will not ingest any more data until resumed. The @@ -165,9 +168,6 @@ public abstract class SeekableStreamIndexTaskRunner private final Condition hasPaused = pauseLock.newCondition(); private final Condition shouldResume = pauseLock.newCondition(); - protected final Lock pollRetryLock = new ReentrantLock(); - protected final Condition isAwaitingRetry = pollRetryLock.newCondition(); - protected final AtomicBoolean stopRequested = new AtomicBoolean(false); private final AtomicBoolean publishOnStop = new AtomicBoolean(false); @@ -180,416 +180,154 @@ public abstract class SeekableStreamIndexTaskRunner // transitioning to publishing state. private final Object statusLock = new Object(); + protected final Lock pollRetryLock = new ReentrantLock(); + protected final Condition isAwaitingRetry = pollRetryLock.newCondition(); + private final SeekableStreamIndexTask task; private final SeekableStreamIOConfig ioConfig; private final SeekableStreamTuningConfig tuningConfig; - private final SeekableStreamIndexTask task; private final InputRowParser parser; private final AuthorizerMapper authorizerMapper; private final Optional chatHandlerProvider; private final CircularBuffer savedParseExceptions; - private final RowIngestionMeters rowIngestionMeters; private final String stream; + private final RowIngestionMeters rowIngestionMeters; + + private final Set publishingSequences = Sets.newConcurrentHashSet(); + private final List> publishWaitList = new ArrayList<>(); + private final List> handOffWaitList = new ArrayList<>(); + private final Map initialOffsetsSnapshot = new HashMap<>(); + + // true for kafka, falsse for kinesis private final boolean isSkipSegmentLineageCheck; - private volatile CopyOnWriteArrayList sequences; - private volatile IngestionState ingestionState; + private volatile DateTime startTime; private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread) private volatile TaskToolbox toolbox; private volatile Thread runThread; - protected volatile boolean pauseRequested = false; - private volatile long nextCheckpointTime; - private volatile Appenderator appenderator; private volatile StreamAppenderatorDriver driver; + private volatile IngestionState ingestionState; + + private volatile boolean pauseRequested = false; + private volatile long nextCheckpointTime; + + private volatile CopyOnWriteArrayList sequences; private volatile Throwable backgroundThreadException; - private volatile DateTime startTime; - protected class SequenceMetadata + public SeekableStreamIndexTaskRunner( + final SeekableStreamIndexTask task, + final InputRowParser parser, + final AuthorizerMapper authorizerMapper, + final Optional chatHandlerProvider, + final CircularBuffer savedParseExceptions, + final RowIngestionMetersFactory rowIngestionMetersFactory, + final boolean isSkipSegmentLineageCheck + ) { - /** - * Lock for accessing {@link #endOffsets} and {@link #checkpointed}. This lock is required because - * {@link #setEndOffsets)} can be called by both the main thread and the HTTP thread. - */ - private final ReentrantLock lock = new ReentrantLock(); + this.task = task; + this.ioConfig = task.getIOConfig(); + this.tuningConfig = task.getTuningConfig(); + this.parser = parser; + this.authorizerMapper = authorizerMapper; + this.chatHandlerProvider = chatHandlerProvider; + this.savedParseExceptions = savedParseExceptions; + this.stream = ioConfig.getStartPartitions().getStream(); + this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters(); + this.isSkipSegmentLineageCheck = isSkipSegmentLineageCheck; - private final int sequenceId; - private final String sequenceName; - private final Map startOffsets; - private final Map endOffsets; - private final Set assignments; - private final boolean sentinel; - private boolean checkpointed; + this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndPartitions().getPartitionSequenceNumberMap()); + this.sequences = new CopyOnWriteArrayList<>(); + this.ingestionState = IngestionState.NOT_STARTED; - @JsonCreator - public SequenceMetadata( - @JsonProperty("sequenceId") int sequenceId, - @JsonProperty("sequenceName") String sequenceName, - @JsonProperty("startOffsets") Map startOffsets, - @JsonProperty("endOffsets") Map endOffsets, - @JsonProperty("checkpointed") boolean checkpointed - ) - { - Preconditions.checkNotNull(sequenceName); - Preconditions.checkNotNull(startOffsets); - Preconditions.checkNotNull(endOffsets); - this.sequenceId = sequenceId; - this.sequenceName = sequenceName; - this.startOffsets = ImmutableMap.copyOf(startOffsets); - this.endOffsets = new HashMap<>(endOffsets); - this.assignments = new HashSet<>(startOffsets.keySet()); - this.checkpointed = checkpointed; - this.sentinel = false; - } + resetNextCheckpointTime(); + } - @JsonProperty - public int getSequenceId() - { - return sequenceId; - } - @JsonProperty - public boolean isCheckpointed() - { - lock.lock(); - try { - return checkpointed; - } - finally { - lock.unlock(); - } + public TaskStatus run(TaskToolbox toolbox) + { + try { + return runInternal(toolbox); } - - @JsonProperty - public String getSequenceName() - { - return sequenceName; + catch (Exception e) { + log.error(e, "Encountered exception while running task."); + final String errorMsg = Throwables.getStackTraceAsString(e); + toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(errorMsg)); + return TaskStatus.failure( + task.getId(), + errorMsg + ); } + } - @JsonProperty - public Map getStartOffsets() - { - return startOffsets; - } + private TaskStatus runInternal(TaskToolbox toolbox) throws Exception + { + log.info("SeekableStream indexing task starting up!"); + startTime = DateTimes.nowUtc(); + status = Status.STARTING; + this.toolbox = toolbox; - @JsonProperty - public Map getEndOffsets() - { - lock.lock(); - try { - return endOffsets; - } - finally { - lock.unlock(); + + if (!restoreSequences()) { + final TreeMap> checkpoints = getCheckPointsFromContext(toolbox, task); + if (checkpoints != null) { + Iterator>> sequenceOffsets = checkpoints.entrySet() + .iterator(); + Map.Entry> previous = sequenceOffsets.next(); + while (sequenceOffsets.hasNext()) { + Map.Entry> current = sequenceOffsets.next(); + sequences.add(new SequenceMetadata( + previous.getKey(), + StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), + previous.getValue(), + current.getValue(), + true + )); + previous = current; + } + sequences.add(new SequenceMetadata( + previous.getKey(), + StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), + previous.getValue(), + endOffsets, + false + )); + } else { + sequences.add(new SequenceMetadata( + 0, + StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0), + ioConfig.getStartPartitions().getPartitionSequenceNumberMap(), + endOffsets, + false + )); } } - @JsonProperty - public boolean isSentinel() - { - return sentinel; - } + log.info("Starting with sequences: %s", sequences); - void setEndOffsets(Map newEndOffsets) - { - lock.lock(); - try { - endOffsets.putAll(newEndOffsets); - checkpointed = true; - } - finally { - lock.unlock(); - } + if (chatHandlerProvider.isPresent()) { + log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); + chatHandlerProvider.get().register(task.getId(), this, false); + } else { + log.warn("No chat handler detected"); } - void updateAssignments(Map nextPartitionOffset) - { - lock.lock(); - try { - assignments.clear(); - nextPartitionOffset.forEach((key, value) -> { - if (endOffsets.get(key).equals(SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER) - || createSequenceNumber(endOffsets.get(key)).compareTo(createSequenceNumber(nextPartitionOffset.get(key))) - > 0) { - assignments.add(key); - } - }); - } - finally { - lock.unlock(); - } - } + runThread = Thread.currentThread(); - boolean isOpen() - { - return !assignments.isEmpty(); - } + // Set up FireDepartmentMetrics + final FireDepartment fireDepartmentForMetrics = new FireDepartment( + task.getDataSchema(), + new RealtimeIOConfig(null, null, null), + null + ); + FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); + toolbox.getMonitorScheduler() + .addMonitor(TaskRealtimeMetricsMonitorBuilder.build(task, fireDepartmentForMetrics, rowIngestionMeters)); - boolean canHandle(OrderedPartitionableRecord record) - { - lock.lock(); - try { - final OrderedSequenceNumber partitionEndOffset = createSequencenNumber(endOffsets.get(record.getPartitionId())); - final OrderedSequenceNumber partitionStartOffset = createSequencenNumber(startOffsets.get(record.getPartitionId())); - final OrderedSequenceNumber recordOffset = createSequencenNumber(record.getSequenceNumber()); - return isOpen() - && recordOffset != null - && partitionEndOffset != null - && partitionStartOffset != null - && recordOffset.compareTo(partitionStartOffset) >= 0 - && recordOffset.compareTo(partitionEndOffset) < 0; - } - finally { - lock.unlock(); - } - } - - @Override - public String toString() - { - lock.lock(); - try { - return "SequenceMetadata{" + - "sequenceName='" + sequenceName + '\'' + - ", sequenceId=" + sequenceId + - ", startOffsets=" + startOffsets + - ", endOffsets=" + endOffsets + - ", assignments=" + assignments + - ", sentinel=" + sentinel + - ", checkpointed=" + checkpointed + - '}'; - } - finally { - lock.unlock(); - } - } - - Supplier getCommitterSupplier(String stream, Map lastPersistedOffsets) - { - // Set up committer. - return () -> - new Committer() - { - @Override - public Object getMetadata() - { - lock.lock(); - - try { - Preconditions.checkState( - assignments.isEmpty(), - "This committer can be used only once all the records till sequences [%s] have been consumed, also make" - + " sure to call updateAssignments before using this committer", - endOffsets - ); - - - // merge endOffsets for this sequence with globally lastPersistedOffsets - // This is done because this committer would be persisting only sub set of segments - // corresponding to the current sequence. Generally, lastPersistedOffsets should already - // cover endOffsets but just to be sure take max of sequences and persist that - for (Map.Entry partitionOffset : endOffsets.entrySet()) { - SequenceType newOffsets = partitionOffset.getValue(); - if (lastPersistedOffsets.containsKey(partitionOffset.getKey()) && - createSequencenNumber(lastPersistedOffsets.get(partitionOffset.getKey())).compareTo( - createSequencenNumber(newOffsets)) > 0) { - newOffsets = lastPersistedOffsets.get(partitionOffset.getKey()); - } - lastPersistedOffsets.put( - partitionOffset.getKey(), - newOffsets - ); - } - - // Publish metadata can be different from persist metadata as we are going to publish only - // subset of segments - return ImmutableMap.of( - METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>(stream, lastPersistedOffsets), - METADATA_PUBLISH_PARTITIONS, new SeekableStreamPartitions<>(stream, endOffsets) - ); - } - finally { - lock.unlock(); - } - } - - @Override - public void run() - { - // Do nothing. - } - }; - - } - - TransactionalSegmentPublisher createPublisher(TaskToolbox toolbox, boolean useTransaction) - { - return (segments, commitMetadata) -> { - final SeekableStreamPartitions finalPartitions = createSeekableStreamPartitions( - toolbox.getObjectMapper(), - ((Map) Preconditions - .checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_PUBLISH_PARTITIONS) - ); - - // Sanity check, we should only be publishing things that match our desired end state. - if (!getEndOffsets().equals(finalPartitions.getPartitionOffsetMap())) { - throw new ISE( - "WTF?! Driver for sequence [%s], attempted to publish invalid metadata[%s].", - toString(), - commitMetadata - ); - } - - final SegmentTransactionalInsertAction action; - - if (useTransaction) { - action = new SegmentTransactionalInsertAction( - segments, - createDataSourceMetadata(new SeekableStreamPartitions<>( - finalPartitions.getTopic(), - getStartOffsets() - )), - createDataSourceMetadata(finalPartitions) - ); - } else { - action = new SegmentTransactionalInsertAction(segments, null, null); - } - - log.info("Publishing with isTransaction[%s].", useTransaction); - - return toolbox.getTaskActionClient().submit(action); - }; - } - - } - - public enum Status - { - NOT_STARTED, - STARTING, - READING, - PAUSED, - PUBLISHING - } - - public SeekableStreamIndexTaskRunner( - final SeekableStreamIndexTask task, - final InputRowParser parser, - final AuthorizerMapper authorizerMapper, - final Optional chatHandlerProvider, - final CircularBuffer savedParseExceptions, - final RowIngestionMetersFactory rowIngestionMetersFactory, - final boolean isSkipSegmentLineageCheck - ) - { - this.task = task; - this.ioConfig = task.getIOConfig(); - this.tuningConfig = task.getTuningConfig(); - this.parser = parser; - this.authorizerMapper = authorizerMapper; - this.chatHandlerProvider = chatHandlerProvider; - this.savedParseExceptions = savedParseExceptions; - this.stream = ioConfig.getStartPartitions().getStream(); - this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters(); - this.isSkipSegmentLineageCheck = isSkipSegmentLineageCheck; - - this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndPartitions().getPartitionOffsetMap()); - this.sequences = new CopyOnWriteArrayList<>(); - this.ingestionState = IngestionState.NOT_STARTED; - - - resetNextCheckpointTime(); - } - - public Appenderator getAppenderator() - { - return appenderator; - } - - public TaskStatus run(TaskToolbox toolbox) - { - try { - return runInternal(toolbox); - } - catch (Exception e) { - log.error(e, "Encountered exception while running task."); - final String errorMsg = Throwables.getStackTraceAsString(e); - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(errorMsg)); - return TaskStatus.failure( - task.getId(), - errorMsg - ); - } - } - - private TaskStatus runInternal(TaskToolbox toolbox) throws Exception - { - log.info("SeekableStream indexing task starting up!"); - startTime = DateTimes.nowUtc(); - status = Status.STARTING; - this.toolbox = toolbox; - - - if (!restoreSequences()) { - final TreeMap> checkpoints = getCheckPointsFromContext(toolbox, task); - if (checkpoints != null) { - Iterator>> sequenceOffsets = checkpoints.entrySet() - .iterator(); - Map.Entry> previous = sequenceOffsets.next(); - while (sequenceOffsets.hasNext()) { - Map.Entry> current = sequenceOffsets.next(); - sequences.add(new SequenceMetadata( - previous.getKey(), - StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), - previous.getValue(), - current.getValue(), - true - )); - previous = current; - } - sequences.add(new SequenceMetadata( - previous.getKey(), - StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), - previous.getValue(), - endOffsets, - false - )); - } else { - sequences.add(new SequenceMetadata( - 0, - StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0), - ioConfig.getStartPartitions().getPartitionOffsetMap(), - endOffsets, - false - )); - } - } - - log.info("Starting with sequences: %s", sequences); - - if (chatHandlerProvider.isPresent()) { - log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); - chatHandlerProvider.get().register(task.getId(), this, false); - } else { - log.warn("No chat handler detected"); - } - - runThread = Thread.currentThread(); - - // Set up FireDepartmentMetrics - final FireDepartment fireDepartmentForMetrics = new FireDepartment( - task.getDataSchema(), - new RealtimeIOConfig(null, null, null), - null - ); - FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); - toolbox.getMonitorScheduler() - .addMonitor(TaskRealtimeMetricsMonitorBuilder.build(task, fireDepartmentForMetrics, rowIngestionMeters)); - - final String lookupTier = task.getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER); - final LookupNodeService lookupNodeService = lookupTier == null ? - toolbox.getLookupNodeService() : - new LookupNodeService(lookupTier); + final String lookupTier = task.getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER); + final LookupNodeService lookupNodeService = lookupTier == null ? + toolbox.getLookupNodeService() : + new LookupNodeService(lookupTier); final DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( toolbox.getDruidNode(), @@ -601,14 +339,14 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception ); Throwable caughtExceptionOuter = null; - try (final RecordSupplier recordSupplier = task.getRecordSupplier()) { + try (final RecordSupplier recordSupplier = task.newRecordSupplier()) { toolbox.getDataSegmentServerAnnouncer().announce(); toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); appenderator = task.newAppenderator(fireDepartmentMetrics, toolbox); driver = task.newDriver(appenderator, toolbox, fireDepartmentMetrics); - final String stream = ioConfig.getStartPartitions().getTopic(); + final String stream = ioConfig.getStartPartitions().getStream(); // Start up, set up initial sequences. final Object restoredMetadata = driver.startJob(); @@ -619,7 +357,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception partitionOffsetEntry -> createSequencenNumber(partitionOffsetEntry.getValue()).compareTo( createSequencenNumber(ioConfig.getStartPartitions() - .getPartitionOffsetMap() + .getPartitionSequenceNumberMap() .get(partitionOffsetEntry.getKey()) )) >= 0 ), "Sequence sequences are not compatible with start sequences of task"); @@ -632,22 +370,22 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception restoredMetadataMap.get(METADATA_NEXT_PARTITIONS) ); - currOffsets.putAll(restoredNextPartitions.getPartitionOffsetMap()); + currOffsets.putAll(restoredNextPartitions.getPartitionSequenceNumberMap()); // Sanity checks. - if (!restoredNextPartitions.getTopic().equals(ioConfig.getStartPartitions().getTopic())) { + if (!restoredNextPartitions.getStream().equals(ioConfig.getStartPartitions().getStream())) { throw new ISE( "WTF?! Restored stream[%s] but expected stream[%s]", - restoredNextPartitions.getTopic(), - ioConfig.getStartPartitions().getTopic() + restoredNextPartitions.getStream(), + ioConfig.getStartPartitions().getStream() ); } - if (!currOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionOffsetMap().keySet())) { + if (!currOffsets.keySet().equals(ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet())) { throw new ISE( "WTF?! Restored partitions[%s] but expected partitions[%s]", currOffsets.keySet(), - ioConfig.getStartPartitions().getPartitionOffsetMap().keySet() + ioConfig.getStartPartitions().getPartitionSequenceNumberMap().keySet() ); } // sequences size can be 0 only when all sequences got published and task stopped before it could finish @@ -683,7 +421,7 @@ public Object getMetadata() { return ImmutableMap.of( METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>( - ioConfig.getStartPartitions().getTopic(), + ioConfig.getStartPartitions().getStream(), snapshot ) ); @@ -1082,19 +820,6 @@ public void onFailure(@ParametersAreNonnullByDefault Throwable t) return TaskStatus.success(task.getId()); } - protected abstract SequenceType getNextSequenceNumber( - RecordSupplier recordSupplier, - StreamPartition partition, - SequenceType sequenceNumber - ); - - - @NotNull - protected abstract List> getRecords( - RecordSupplier recordSupplier, - TaskToolbox toolbox - ) throws Exception; - private void checkPublishAndHandoffFailure() throws ExecutionException, InterruptedException { // Check if any publishFuture failed. @@ -1124,7 +849,6 @@ private void checkPublishAndHandoffFailure() throws ExecutionException, Interrup handOffWaitList.removeAll(handoffFinished); } - private void publishAndRegisterHandoff(SequenceMetadata sequenceMetadata) { log.info("Publishing segments for sequence [%s]", sequenceMetadata); @@ -1213,69 +937,99 @@ public void onFailure(@ParametersAreNonnullByDefault Throwable t) ); } - private void handleParseException(ParseException pe, OrderedPartitionableRecord record) + private static File getSequencesPersistFile(TaskToolbox toolbox) { - if (pe.isFromPartiallyValidRow()) { - rowIngestionMeters.incrementProcessedWithError(); - } else { - rowIngestionMeters.incrementUnparseable(); - } + return new File(toolbox.getPersistDir(), "sequences.json"); + } - if (tuningConfig.isLogParseExceptions()) { - log.error( - pe, - "Encountered parse exception on row from partition[%s] sequence[%s]", - record.getPartitionId(), - record.getSequenceNumber() + private boolean restoreSequences() throws IOException + { + final File sequencesPersistFile = getSequencesPersistFile(toolbox); + if (sequencesPersistFile.exists()) { + sequences = new CopyOnWriteArrayList<>( + toolbox.getObjectMapper().>readValue( + sequencesPersistFile, + new TypeReference>() + { + } + ) ); + return true; + } else { + return false; } + } - if (savedParseExceptions != null) { - savedParseExceptions.add(pe); - } - - if (rowIngestionMeters.getUnparseable() + rowIngestionMeters.getProcessedWithError() - > tuningConfig.getMaxParseExceptions()) { - log.error("Max parse exceptions exceeded, terminating task..."); - throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); - } + private synchronized void persistSequences() throws IOException + { + log.info("Persisting Sequences Metadata [%s]", sequences); + toolbox.getObjectMapper().writerWithType( + new TypeReference>() + { + } + ).writeValue(getSequencesPersistFile(toolbox), sequences); } - private boolean verifyInitialRecordAndSkipExclusiveParition( - final OrderedPartitionableRecord record, - final Map intialSequenceSnapshot - ) + private Map getTaskCompletionReports(@Nullable String errorMsg) { - if (intialSequenceSnapshot.containsKey(record.getPartitionId())) { - if (!intialSequenceSnapshot.get(record.getPartitionId()).equals(record.getSequenceNumber())) { - throw new ISE( - "Starting sequenceNumber [%s] does not match expected [%s] for partition [%s]", - record.getSequenceNumber(), - intialSequenceSnapshot.get(record.getPartitionId()), - record.getPartitionId() - ); - } + return TaskReport.buildTaskReports( + new IngestionStatsAndErrorsTaskReport( + task.getId(), + new IngestionStatsAndErrorsTaskReportData( + ingestionState, + getTaskCompletionUnparseableEvents(), + getTaskCompletionRowStats(), + errorMsg + ) + ) + ); + } - log.info( - "Verified starting sequenceNumber [%s] for partition [%s]", - record.getSequenceNumber(), record.getPartitionId() - ); + private Map getTaskCompletionUnparseableEvents() + { + Map unparseableEventsMap = new HashMap<>(); + List buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions( + savedParseExceptions + ); + if (buildSegmentsParseExceptionMessages != null) { + unparseableEventsMap.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegmentsParseExceptionMessages); + } + return unparseableEventsMap; + } - intialSequenceSnapshot.remove(record.getPartitionId()); - if (intialSequenceSnapshot.isEmpty()) { - log.info("Verified starting sequences for all partitions"); - } + private Map getTaskCompletionRowStats() + { + Map metrics = new HashMap<>(); + metrics.put( + RowIngestionMeters.BUILD_SEGMENTS, + rowIngestionMeters.getTotals() + ); + return metrics; + } - // check exclusive starting sequence - if (ioConfig.getExclusiveStartSequenceNumberPartitions() != null - && ioConfig.getExclusiveStartSequenceNumberPartitions().contains(record.getPartitionId())) { - log.info("Skipping starting sequenceNumber for partition [%s] marked exclusive", record.getPartitionId()); - return false; + private void maybePersistAndPublishSequences(Supplier committerSupplier) + throws InterruptedException + { + for (SequenceMetadata sequenceMetadata : sequences) { + sequenceMetadata.updateAssignments(currOffsets); + if (!sequenceMetadata.isOpen() && !publishingSequences.contains(sequenceMetadata.getSequenceName())) { + publishingSequences.add(sequenceMetadata.getSequenceName()); + try { + Object result = driver.persist(committerSupplier.get()); + log.info( + "Persist completed with results: [%s], adding sequence [%s] to publish queue", + result, + sequenceMetadata + ); + publishAndRegisterHandoff(sequenceMetadata); + } + catch (InterruptedException e) { + log.warn("Interrupted while persisting sequence [%s]", sequenceMetadata); + throw e; + } } } - - return true; } private Set> assignPartitions( @@ -1317,6 +1071,48 @@ private void seekToStartingSequence( } } + /** + * Checks if the pauseRequested flag was set and if so blocks: + * a) if pauseMillis == PAUSE_FOREVER, until pauseRequested is cleared + * b) if pauseMillis != PAUSE_FOREVER, until pauseMillis elapses -or- pauseRequested is cleared + *

+ * If pauseMillis is changed while paused, the new pause timeout will be applied. This allows adjustment of the + * pause timeout (making a timed pause into an indefinite pause and vice versa is valid) without having to resume + * and ensures that the loop continues to stay paused without ingesting any new events. You will need to signal + * shouldResume after adjusting pauseMillis for the new value to take effect. + *

+ * Sets paused = true and signals paused so callers can be notified when the pause command has been accepted. + *

+ * Additionally, pauses if all partitions assignments have been read and pauseAfterRead flag is set. + * + * @return true if a pause request was handled, false otherwise + */ + private boolean possiblyPause() throws InterruptedException + { + pauseLock.lockInterruptibly(); + try { + if (pauseRequested) { + status = Status.PAUSED; + hasPaused.signalAll(); + + while (pauseRequested) { + log.info("Pausing ingestion until resumed"); + shouldResume.await(); + } + + status = Status.READING; + shouldResume.signalAll(); + log.info("Ingestion loop resumed"); + return true; + } + } + finally { + pauseLock.unlock(); + } + + return false; + } + private void possiblyResetDataSourceMetadata( RecordSupplier recordSupplier, Set> assignment @@ -1353,48 +1149,46 @@ private void possiblyResetDataSourceMetadata( } } - /** - * Checks if the pauseRequested flag was set and if so blocks: - * a) if pauseMillis == PAUSE_FOREVER, until pauseRequested is cleared - * b) if pauseMillis != PAUSE_FOREVER, until pauseMillis elapses -or- pauseRequested is cleared - *

- * If pauseMillis is changed while paused, the new pause timeout will be applied. This allows adjustment of the - * pause timeout (making a timed pause into an indefinite pause and vice versa is valid) without having to resume - * and ensures that the loop continues to stay paused without ingesting any new events. You will need to signal - * shouldResume after adjusting pauseMillis for the new value to take effect. - *

- * Sets paused = true and signals paused so callers can be notified when the pause command has been accepted. - *

- * Additionally, pauses if all partitions assignments have been read and pauseAfterRead flag is set. - * - * @return true if a pause request was handled, false otherwise - */ - private boolean possiblyPause() throws InterruptedException + + private void handleParseException(ParseException pe, OrderedPartitionableRecord record) { - pauseLock.lockInterruptibly(); - try { - if (pauseRequested) { - status = Status.PAUSED; - hasPaused.signalAll(); + if (pe.isFromPartiallyValidRow()) { + rowIngestionMeters.incrementProcessedWithError(); + } else { + rowIngestionMeters.incrementUnparseable(); + } - while (pauseRequested) { - log.info("Pausing ingestion until resumed"); - shouldResume.await(); - } + if (tuningConfig.isLogParseExceptions()) { + log.error( + pe, + "Encountered parse exception on row from partition[%s] sequence[%s]", + record.getPartitionId(), + record.getSequenceNumber() + ); + } - status = Status.READING; - shouldResume.signalAll(); - log.info("Ingestion loop resumed"); - return true; - } + if (savedParseExceptions != null) { + savedParseExceptions.add(pe); } - finally { - pauseLock.unlock(); + + if (rowIngestionMeters.getUnparseable() + rowIngestionMeters.getProcessedWithError() + > tuningConfig.getMaxParseExceptions()) { + log.error("Max parse exceptions exceeded, terminating task..."); + throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); } + } - return false; + private boolean isPaused() + { + return status == Status.PAUSED; + } + + private void requestPause() + { + pauseRequested = true; } + protected void sendResetRequestAndWait( Map, SequenceType> outOfRangePartitions, TaskToolbox taskToolbox @@ -1429,92 +1223,27 @@ protected void sendResetRequestAndWait( } } + /** + * Authorizes action to be performed on this task's datasource + * + * @return authorization result + */ + private Access authorizationCheck(final HttpServletRequest req, Action action) + { + return IndexTaskUtils.datasourceAuthorizationCheck(req, action, task.getDataSource(), authorizerMapper); + } - private void maybePersistAndPublishSequences(Supplier committerSupplier) - throws InterruptedException + public Appenderator getAppenderator() { - for (SequenceMetadata sequenceMetadata : sequences) { - sequenceMetadata.updateAssignments(currOffsets); - if (!sequenceMetadata.isOpen() && !publishingSequences.contains(sequenceMetadata.getSequenceName())) { - publishingSequences.add(sequenceMetadata.getSequenceName()); - try { - Object result = driver.persist(committerSupplier.get()); - log.info( - "Persist completed with results: [%s], adding sequence [%s] to publish queue", - result, - sequenceMetadata - ); - publishAndRegisterHandoff(sequenceMetadata); - } - catch (InterruptedException e) { - log.warn("Interrupted while persisting sequence [%s]", sequenceMetadata); - throw e; - } - } - } - } - - protected abstract SeekableStreamPartitions createSeekableStreamPartitions( - ObjectMapper mapper, - Object obeject - ); - - protected abstract OrderedSequenceNumber createSequencenNumber(SequenceType sequenceNumber); - - private boolean restoreSequences() throws IOException - { - final File sequencesPersistFile = getSequencesPersistFile(toolbox); - if (sequencesPersistFile.exists()) { - sequences = new CopyOnWriteArrayList<>( - toolbox.getObjectMapper().>readValue( - sequencesPersistFile, - new TypeReference>() - { - } - ) - ); - return true; - } else { - return false; - } - } - - private Map getTaskCompletionReports(@Nullable String errorMsg) - { - return TaskReport.buildTaskReports( - new IngestionStatsAndErrorsTaskReport( - task.getId(), - new IngestionStatsAndErrorsTaskReportData( - ingestionState, - getTaskCompletionUnparseableEvents(), - getTaskCompletionRowStats(), - errorMsg - ) - ) - ); + return appenderator; } - private Map getTaskCompletionUnparseableEvents() + @VisibleForTesting + public RowIngestionMeters getRowIngestionMeters() { - Map unparseableEventsMap = new HashMap<>(); - List buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions( - savedParseExceptions - ); - if (buildSegmentsParseExceptionMessages != null) { - unparseableEventsMap.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegmentsParseExceptionMessages); - } - return unparseableEventsMap; + return rowIngestionMeters; } - private Map getTaskCompletionRowStats() - { - Map metrics = new HashMap<>(); - metrics.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getTotals() - ); - return metrics; - } public void stopGracefully() { @@ -1562,6 +1291,111 @@ public void stopGracefully() } } + @POST + @Path("/stop") + public Response stop(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.WRITE); + stopGracefully(); + return Response.status(Response.Status.OK).build(); + } + + @GET + @Path("/status") + @Produces(MediaType.APPLICATION_JSON) + public Status getStatusHTTP(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return status; + } + + @VisibleForTesting + public Status getStatus() + { + return status; + } + + @GET + @Path("/offsets/current") + @Produces(MediaType.APPLICATION_JSON) + public Map getCurrentOffsets(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return getCurrentOffsets(); + } + + public Map getCurrentOffsets() + { + return currOffsets; + } + + @GET + @Path("/offsets/end") + @Produces(MediaType.APPLICATION_JSON) + public Map getEndOffsetsHTTP(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return getEndOffsets(); + } + + public Map getEndOffsets() + { + return endOffsets; + } + + @POST + @Path("/offsets/end") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response setEndOffsetsHTTP( + Map sequences, + @QueryParam("finish") @DefaultValue("true") final boolean finish, + // this field is only for internal purposes, shouldn't be usually set by users + @Context final HttpServletRequest req + ) throws InterruptedException + { + authorizationCheck(req, Action.WRITE); + return setEndOffsets(sequences, finish); + } + + @GET + @Path("/rowStats") + @Produces(MediaType.APPLICATION_JSON) + public Response getRowStats( + @Context final HttpServletRequest req + ) + { + authorizationCheck(req, Action.READ); + Map returnMap = new HashMap<>(); + Map totalsMap = new HashMap<>(); + Map averagesMap = new HashMap<>(); + + totalsMap.put( + RowIngestionMeters.BUILD_SEGMENTS, + rowIngestionMeters.getTotals() + ); + averagesMap.put( + RowIngestionMeters.BUILD_SEGMENTS, + rowIngestionMeters.getMovingAverages() + ); + + returnMap.put("movingAverages", averagesMap); + returnMap.put("totals", totalsMap); + return Response.ok(returnMap).build(); + } + + @GET + @Path("/unparseableEvents") + @Produces(MediaType.APPLICATION_JSON) + public Response getUnparseableEvents( + @Context final HttpServletRequest req + ) + { + authorizationCheck(req, Action.READ); + List events = IndexTaskUtils.getMessagesFromSavedParseExceptions(savedParseExceptions); + return Response.ok(events).build(); + } + @VisibleForTesting public Response setEndOffsets( Map sequenceNumbers, @@ -1661,6 +1495,47 @@ public Response setEndOffsets( return Response.ok(sequenceNumbers).build(); } + private void resetNextCheckpointTime() + { + nextCheckpointTime = DateTimes.nowUtc().plus(tuningConfig.getIntermediateHandoffPeriod()).getMillis(); + } + + @GET + @Path("/checkpoints") + @Produces(MediaType.APPLICATION_JSON) + public Map> getCheckpointsHTTP(@Context final HttpServletRequest req) + { + authorizationCheck(req, Action.READ); + return getCheckpoints(); + } + + private Map> getCheckpoints() + { + return new TreeMap<>(sequences.stream() + .collect(Collectors.toMap( + SequenceMetadata::getSequenceId, + SequenceMetadata::getStartOffsets + ))); + } + + /** + * Signals the ingestion loop to pause. + * + * @return one of the following Responses: 400 Bad Request if the task has started publishing; 202 Accepted if the + * method has timed out and returned before the task has paused; 200 OK with a map of the current partition sequences + * in the response body if the task successfully paused + */ + @POST + @Path("/pause") + @Produces(MediaType.APPLICATION_JSON) + public Response pauseHTTP( + @Context final HttpServletRequest req + ) throws InterruptedException + { + authorizationCheck(req, Action.WRITE); + return pause(); + } + @VisibleForTesting public Response pause() throws InterruptedException { @@ -1708,6 +1583,16 @@ public Response pause() throws InterruptedException } } + @POST + @Path("/resume") + public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException + { + authorizationCheck(req, Action.WRITE); + resume(); + return Response.status(Response.Status.OK).build(); + } + + @VisibleForTesting public void resume() throws InterruptedException { @@ -1729,220 +1614,340 @@ public void resume() throws InterruptedException } } - protected abstract SeekableStreamDataSourceMetadata createDataSourceMetadata( - SeekableStreamPartitions partitions - ); - - protected abstract OrderedSequenceNumber createSequenceNumber(SequenceType sequenceNumber); - private void resetNextCheckpointTime() + @GET + @Path("/time/start") + @Produces(MediaType.APPLICATION_JSON) + public DateTime getStartTime(@Context final HttpServletRequest req) { - nextCheckpointTime = DateTimes.nowUtc().plus(tuningConfig.getIntermediateHandoffPeriod()).getMillis(); + authorizationCheck(req, Action.WRITE); + return startTime; } - private boolean isPaused() + protected class SequenceMetadata { - return status == Status.PAUSED; - } + /** + * Lock for accessing {@link #endOffsets} and {@link #checkpointed}. This lock is required because + * {@link #setEndOffsets)} can be called by both the main thread and the HTTP thread. + */ + private final ReentrantLock lock = new ReentrantLock(); - private void requestPause() - { - pauseRequested = true; - } + private final int sequenceId; + private final String sequenceName; + private final Map startOffsets; + private final Map endOffsets; + private final Set assignments; + private final boolean sentinel; + private boolean checkpointed; - private synchronized void persistSequences() throws IOException - { - log.info("Persisting Sequences Metadata [%s]", sequences); - toolbox.getObjectMapper().writerWithType( - new TypeReference>() - { - } - ).writeValue(getSequencesPersistFile(toolbox), sequences); - } + @JsonCreator + public SequenceMetadata( + @JsonProperty("sequenceId") int sequenceId, + @JsonProperty("sequenceName") String sequenceName, + @JsonProperty("startOffsets") Map startOffsets, + @JsonProperty("endOffsets") Map endOffsets, + @JsonProperty("checkpointed") boolean checkpointed + ) + { + Preconditions.checkNotNull(sequenceName); + Preconditions.checkNotNull(startOffsets); + Preconditions.checkNotNull(endOffsets); + this.sequenceId = sequenceId; + this.sequenceName = sequenceName; + this.startOffsets = ImmutableMap.copyOf(startOffsets); + this.endOffsets = new HashMap<>(endOffsets); + this.assignments = new HashSet<>(startOffsets.keySet()); + this.checkpointed = checkpointed; + this.sentinel = false; + } - private static File getSequencesPersistFile(TaskToolbox toolbox) - { - return new File(toolbox.getPersistDir(), "sequences.json"); - } + @JsonProperty + public int getSequenceId() + { + return sequenceId; + } - @Nullable - protected abstract TreeMap> getCheckPointsFromContext( - TaskToolbox toolbox, - SeekableStreamIndexTask task - ) throws IOException; + @JsonProperty + public boolean isCheckpointed() + { + lock.lock(); + try { + return checkpointed; + } + finally { + lock.unlock(); + } + } - /** - * Authorizes action to be performed on this task's datasource - * - * @return authorization result - */ - private Access authorizationCheck(final HttpServletRequest req, Action action) - { - return IndexTaskUtils.datasourceAuthorizationCheck(req, action, task.getDataSource(), authorizerMapper); - } + @JsonProperty + public String getSequenceName() + { + return sequenceName; + } - @VisibleForTesting - public RowIngestionMeters getRowIngestionMeters() - { - return rowIngestionMeters; - } + @JsonProperty + public Map getStartOffsets() + { + return startOffsets; + } - @POST - @Path("/stop") - public Response stop(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.WRITE); - stopGracefully(); - return Response.status(Response.Status.OK).build(); - } + @JsonProperty + public Map getEndOffsets() + { + lock.lock(); + try { + return endOffsets; + } + finally { + lock.unlock(); + } + } - @GET - @Path("/status") - @Produces(MediaType.APPLICATION_JSON) - public Status getStatusHTTP(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.READ); - return status; - } + @JsonProperty + public boolean isSentinel() + { + return sentinel; + } - @VisibleForTesting - public Status getStatus() - { - return status; - } + void setEndOffsets(Map newEndOffsets) + { + lock.lock(); + try { + endOffsets.putAll(newEndOffsets); + checkpointed = true; + } + finally { + lock.unlock(); + } + } - @GET - @Path("/offsets/current") - @Produces(MediaType.APPLICATION_JSON) - public Map getCurrentOffsets(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.READ); - return getCurrentOffsets(); - } + void updateAssignments(Map nextPartitionOffset) + { + lock.lock(); + try { + assignments.clear(); + nextPartitionOffset.forEach((key, value) -> { + if (endOffsets.get(key).equals(SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER) + || createSequenceNumber(endOffsets.get(key)).compareTo(createSequenceNumber(nextPartitionOffset.get(key))) + > 0) { + assignments.add(key); + } + }); + } + finally { + lock.unlock(); + } + } - public Map getCurrentOffsets() - { - return currOffsets; - } + boolean isOpen() + { + return !assignments.isEmpty(); + } - @GET - @Path("/offsets/end") - @Produces(MediaType.APPLICATION_JSON) - public Map getEndOffsetsHTTP(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.READ); - return getEndOffsets(); - } + boolean canHandle(OrderedPartitionableRecord record) + { + lock.lock(); + try { + final OrderedSequenceNumber partitionEndOffset = createSequencenNumber(endOffsets.get(record.getPartitionId())); + final OrderedSequenceNumber partitionStartOffset = createSequencenNumber(startOffsets.get(record.getPartitionId())); + final OrderedSequenceNumber recordOffset = createSequencenNumber(record.getSequenceNumber()); + return isOpen() + && recordOffset != null + && partitionEndOffset != null + && partitionStartOffset != null + && recordOffset.compareTo(partitionStartOffset) >= 0 + && recordOffset.compareTo(partitionEndOffset) < 0; + } + finally { + lock.unlock(); + } + } - public Map getEndOffsets() - { - return endOffsets; - } + @Override + public String toString() + { + lock.lock(); + try { + return "SequenceMetadata{" + + "sequenceName='" + sequenceName + '\'' + + ", sequenceId=" + sequenceId + + ", startOffsets=" + startOffsets + + ", endOffsets=" + endOffsets + + ", assignments=" + assignments + + ", sentinel=" + sentinel + + ", checkpointed=" + checkpointed + + '}'; + } + finally { + lock.unlock(); + } + } - @POST - @Path("/offsets/end") - @Consumes(MediaType.APPLICATION_JSON) - @Produces(MediaType.APPLICATION_JSON) - public Response setEndOffsetsHTTP( - Map sequences, - @QueryParam("finish") @DefaultValue("true") final boolean finish, - // this field is only for internal purposes, shouldn't be usually set by users - @Context final HttpServletRequest req - ) throws InterruptedException - { - authorizationCheck(req, Action.WRITE); - return setEndOffsets(sequences, finish); - } + Supplier getCommitterSupplier(String stream, Map lastPersistedOffsets) + { + // Set up committer. + return () -> + new Committer() + { + @Override + public Object getMetadata() + { + lock.lock(); - @GET - @Path("/rowStats") - @Produces(MediaType.APPLICATION_JSON) - public Response getRowStats( - @Context final HttpServletRequest req - ) - { - authorizationCheck(req, Action.READ); - Map returnMap = new HashMap<>(); - Map totalsMap = new HashMap<>(); - Map averagesMap = new HashMap<>(); + try { + Preconditions.checkState( + assignments.isEmpty(), + "This committer can be used only once all the records till sequences [%s] have been consumed, also make" + + " sure to call updateAssignments before using this committer", + endOffsets + ); - totalsMap.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getTotals() - ); - averagesMap.put( - RowIngestionMeters.BUILD_SEGMENTS, - rowIngestionMeters.getMovingAverages() - ); - returnMap.put("movingAverages", averagesMap); - returnMap.put("totals", totalsMap); - return Response.ok(returnMap).build(); + // merge endOffsets for this sequence with globally lastPersistedOffsets + // This is done because this committer would be persisting only sub set of segments + // corresponding to the current sequence. Generally, lastPersistedOffsets should already + // cover endOffsets but just to be sure take max of sequences and persist that + for (Map.Entry partitionOffset : endOffsets.entrySet()) { + SequenceType newOffsets = partitionOffset.getValue(); + if (lastPersistedOffsets.containsKey(partitionOffset.getKey()) && + createSequencenNumber(lastPersistedOffsets.get(partitionOffset.getKey())).compareTo( + createSequencenNumber(newOffsets)) > 0) { + newOffsets = lastPersistedOffsets.get(partitionOffset.getKey()); + } + lastPersistedOffsets.put( + partitionOffset.getKey(), + newOffsets + ); + } + + // Publish metadata can be different from persist metadata as we are going to publish only + // subset of segments + return ImmutableMap.of( + METADATA_NEXT_PARTITIONS, new SeekableStreamPartitions<>(stream, lastPersistedOffsets), + METADATA_PUBLISH_PARTITIONS, new SeekableStreamPartitions<>(stream, endOffsets) + ); + } + finally { + lock.unlock(); + } + } + + @Override + public void run() + { + // Do nothing. + } + }; + + } + + TransactionalSegmentPublisher createPublisher(TaskToolbox toolbox, boolean useTransaction) + { + return (segments, commitMetadata) -> { + final SeekableStreamPartitions finalPartitions = createSeekableStreamPartitions( + toolbox.getObjectMapper(), + ((Map) Preconditions + .checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_PUBLISH_PARTITIONS) + ); + + // Sanity check, we should only be publishing things that match our desired end state. + if (!getEndOffsets().equals(finalPartitions.getPartitionSequenceNumberMap())) { + throw new ISE( + "WTF?! Driver for sequence [%s], attempted to publish invalid metadata[%s].", + toString(), + commitMetadata + ); + } + + final SegmentTransactionalInsertAction action; + + if (useTransaction) { + action = new SegmentTransactionalInsertAction( + segments, + createDataSourceMetadata(new SeekableStreamPartitions<>( + finalPartitions.getStream(), + getStartOffsets() + )), + createDataSourceMetadata(finalPartitions) + ); + } else { + action = new SegmentTransactionalInsertAction(segments, null, null); + } + + log.info("Publishing with isTransaction[%s].", useTransaction); + + return toolbox.getTaskActionClient().submit(action); + }; + } + } - @GET - @Path("/unparseableEvents") - @Produces(MediaType.APPLICATION_JSON) - public Response getUnparseableEvents( - @Context final HttpServletRequest req + private boolean verifyInitialRecordAndSkipExclusiveParition( + final OrderedPartitionableRecord record, + final Map intialSequenceSnapshot ) { - authorizationCheck(req, Action.READ); - List events = IndexTaskUtils.getMessagesFromSavedParseExceptions(savedParseExceptions); - return Response.ok(events).build(); - } + if (intialSequenceSnapshot.containsKey(record.getPartitionId())) { + if (!intialSequenceSnapshot.get(record.getPartitionId()).equals(record.getSequenceNumber())) { + throw new ISE( + "Starting sequenceNumber [%s] does not match expected [%s] for partition [%s]", + record.getSequenceNumber(), + intialSequenceSnapshot.get(record.getPartitionId()), + record.getPartitionId() + ); + } - @GET - @Path("/checkpoints") - @Produces(MediaType.APPLICATION_JSON) - public Map> getCheckpointsHTTP(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.READ); - return getCheckpoints(); - } + log.info( + "Verified starting sequenceNumber [%s] for partition [%s]", + record.getSequenceNumber(), record.getPartitionId() + ); - private Map> getCheckpoints() - { - return new TreeMap<>(sequences.stream() - .collect(Collectors.toMap( - SequenceMetadata::getSequenceId, - SequenceMetadata::getStartOffsets - ))); - } + intialSequenceSnapshot.remove(record.getPartitionId()); + if (intialSequenceSnapshot.isEmpty()) { + log.info("Verified starting sequences for all partitions"); + } - /** - * Signals the ingestion loop to pause. - * - * @return one of the following Responses: 400 Bad Request if the task has started publishing; 202 Accepted if the - * method has timed out and returned before the task has paused; 200 OK with a map of the current partition sequences - * in the response body if the task successfully paused - */ - @POST - @Path("/pause") - @Produces(MediaType.APPLICATION_JSON) - public Response pauseHTTP( - @Context final HttpServletRequest req - ) throws InterruptedException - { - authorizationCheck(req, Action.WRITE); - return pause(); - } + // check exclusive starting sequence + if (ioConfig.getExclusiveStartSequenceNumberPartitions() != null + && ioConfig.getExclusiveStartSequenceNumberPartitions().contains(record.getPartitionId())) { + log.info("Skipping starting sequenceNumber for partition [%s] marked exclusive", record.getPartitionId()); - @POST - @Path("/resume") - public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException - { - authorizationCheck(req, Action.WRITE); - resume(); - return Response.status(Response.Status.OK).build(); - } + return false; + } + } - @GET - @Path("/time/start") - @Produces(MediaType.APPLICATION_JSON) - public DateTime getStartTime(@Context final HttpServletRequest req) - { - authorizationCheck(req, Action.WRITE); - return startTime; + return true; } + + @Nullable + protected abstract TreeMap> getCheckPointsFromContext( + TaskToolbox toolbox, + SeekableStreamIndexTask task + ) throws IOException; + + protected abstract SequenceType getNextSequenceNumber( + RecordSupplier recordSupplier, + StreamPartition partition, + SequenceType sequenceNumber + ); + + protected abstract SeekableStreamPartitions createSeekableStreamPartitions( + ObjectMapper mapper, + Object obeject + ); + + protected abstract OrderedSequenceNumber createSequencenNumber(SequenceType sequenceNumber); + + + @NotNull + protected abstract List> getRecords( + RecordSupplier recordSupplier, + TaskToolbox toolbox + ) throws Exception; + + protected abstract SeekableStreamDataSourceMetadata createDataSourceMetadata( + SeekableStreamPartitions partitions + ); + + protected abstract OrderedSequenceNumber createSequenceNumber(SequenceType sequenceNumber); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 7cffdb26f1e4..43b0169a647e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1713,8 +1713,15 @@ protected String generateSequenceName( private void updatePartitionDataFromStream() { Set partitionIds; - synchronized (recordSupplierLock) { - partitionIds = recordSupplier.getPartitionIds(ioConfig.getStream()); + try { + synchronized (recordSupplierLock) { + partitionIds = recordSupplier.getPartitionIds(ioConfig.getStream()); + } + } + catch (Exception e) { + log.warn("Could not fetch partitions for topic/stream [%s]", ioConfig.getStream()); + log.debug(e, "full stack trace"); + return; } if (partitionIds == null || partitionIds.size() == 0) { From f40feb103a614c0e62313aa91641c7c31aded77e Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 14 Nov 2018 15:42:36 -0800 Subject: [PATCH 44/87] minor fix --- .../indexing/seekablestream/SeekableStreamIndexTaskRunner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 197decc3a4c8..48332c290b35 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -209,7 +209,7 @@ public enum Status private volatile StreamAppenderatorDriver driver; private volatile IngestionState ingestionState; - private volatile boolean pauseRequested = false; + protected volatile boolean pauseRequested = false; private volatile long nextCheckpointTime; private volatile CopyOnWriteArrayList sequences; From 0252b20ec99be5b54799cb479520103e55c843f9 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 14 Nov 2018 16:58:48 -0800 Subject: [PATCH 45/87] removed unneeded readme --- .../kinesis-indexing-service/README.md | 163 ------------------ 1 file changed, 163 deletions(-) delete mode 100644 extensions-core/kinesis-indexing-service/README.md diff --git a/extensions-core/kinesis-indexing-service/README.md b/extensions-core/kinesis-indexing-service/README.md deleted file mode 100644 index 32db6263a098..000000000000 --- a/extensions-core/kinesis-indexing-service/README.md +++ /dev/null @@ -1,163 +0,0 @@ -# Kinesis Indexing Service - -Similar to the [Kafka indexing service](http://druid.io/docs/0.10.0/development/extensions-core/kafka-ingestion.html), -the Kinesis indexing service uses supervisors which run on the overlord and manage the creation and lifetime of Kinesis -indexing tasks. This indexing service can handle non-recent events and provides exactly-once ingestion semantics. - -The Kinesis indexing service is provided as the `druid-kinesis-indexing-service` core extension (see -[Including Extensions](http://druid.io/docs/0.10.0/operations/including-extensions.html)). Please note that this is -currently designated as an *experimental feature* and is subject to the usual -[experimental caveats](http://druid.io/docs/0.10.0/development/experimental.html). - -## Submitting a Supervisor Spec - -The Kinesis indexing service requires that the `druid-kinesis-indexing-service` extension be loaded on both the overlord -and the middle managers. A supervisor for a dataSource is started by submitting a supervisor spec via HTTP POST to -`http://:/druid/indexer/v1/supervisor`, for example: - -``` -curl -X POST -H 'Content-Type: application/json' -d @supervisor-spec.json http://localhost:8090/druid/indexer/v1/supervisor -``` - -A sample supervisor spec is shown below: - -```json -{ - "type": "kinesis", - "dataSchema": { - "dataSource": "metrics-kinesis", - "parser": { - "type": "string", - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [], - "dimensionExclusions": [ - "timestamp", - "value" - ] - } - } - }, - "metricsSpec": [ - { - "name": "count", - "type": "count" - }, - { - "name": "value_sum", - "fieldName": "value", - "type": "doubleSum" - }, - { - "name": "value_min", - "fieldName": "value", - "type": "doubleMin" - }, - { - "name": "value_max", - "fieldName": "value", - "type": "doubleMax" - } - ], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "HOUR", - "queryGranularity": "NONE" - } - }, - "tuningConfig": { - "type": "kinesis", - "maxRowsPerSegment": 5000000 - }, - "ioConfig": { - "stream": "metrics", - "endpoint": "kinesis.us-east-1.amazonaws.com", - "taskCount": 1, - "replicas": 1, - "taskDuration": "PT1H", - "recordsPerFetch": 2000, - "fetchDelayMillis": 1000 - } -} -``` - -## Supervisor Configuration - -|Field|Description|Required| -|--------|-----------|---------| -|`type`|The supervisor type, this should always be `kinesis`.|yes| -|`dataSchema`|The schema that will be used by the Kinesis indexing task during ingestion, see [Ingestion Spec](http://druid.io/docs/0.10.0/ingestion/index.html).|yes| -|`tuningConfig`|A KinesisSupervisorTuningConfig to configure the supervisor and indexing tasks, see below.|no| -|`ioConfig`|A KinesisSupervisorIOConfig to configure the supervisor and indexing tasks, see below.|yes| - -### KinesisSupervisorTuningConfig - -The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. - -|Field|Type|Description|Required| -|-----|----|-----------|--------| -|`type`|String|The indexing task type, this should always be `kinesis`.|yes| -|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 75000)| -|`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number is post-aggregation rows.|no (default == 5000000)| -|`intermediatePersistPeriod`|ISO8601 Period|The period that determines the rate at which intermediate persists occur.|no (default == PT10M)| -|`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)| -|`indexSpec`|Object|Tune how data is indexed, see 'IndexSpec' [here](http://druid.io/docs/0.10.0/development/extensions-core/kafka-ingestion.html#indexspec).|no| -|`buildV9Directly`|Boolean|Whether to build a v9 index directly instead of first building a v8 index and then converting it to v9 format.|no (default == true)| -|`reportParseExceptions`|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|no (default == false)| -|`workerThreads`|Integer|The number of threads that will be used by the supervisor for asynchronous operations.|no (default == min(10, taskCount))| -|`chatThreads`|Integer|The number of threads that will be used for communicating with indexing tasks.|no (default == min(10, taskCount * replicas))| -|`chatRetries`|Integer|The number of times HTTP requests to indexing tasks will be retried before considering tasks unresponsive.|no (default == 8)| -|`httpTimeout`|ISO8601 Period|How long to wait for a HTTP response from an indexing task.|no (default == PT10S)| -|`shutdownTimeout`|ISO8601 Period|How long to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|no (default == PT80S)| -|`recordBufferSize`|Integer|Size of the buffer (number of events) used between the Kinesis fetch threads and the main ingestion thread.|no (default == 10000)| -|`recordBufferOfferTimeout`|Integer|Length of time in milliseconds to wait for space to become available in the buffer before timing out.|no (default == 10000)| -|`recordBufferFullWait`|Integer|Length of time in milliseconds to wait for the buffer to drain before attempting to fetch records from Kinesis again.|no (default == 10000)| -|`fetchSequenceNumberTimeout`|Integer|Length of time in milliseconds to wait for Kinesis to return the earliest or latest sequence number for a partition. Kinesis will not return the latest sequence number if no data is actively being written to that partition. In this case, this fetch call will repeatedly timeout and retry until fresh data is written to the stream.|no (default == 60000)| -|`fetchThreads`|Integer|Size of the pool of threads fetching data from Kinesis. There is no benefit in having more threads than Kinesis partitions.|no (default == max(1, {numProcessors} - 1))| - -### KinesisSupervisorIOConfig - -|Field|Type|Description|Required| -|-----|----|-----------|--------| -|`stream`|String|The Kinesis stream to read.|yes| -|`endpoint`|String|The AWS Kinesis stream endpoint for a region. You can find a list of endpoints [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region).|no (default == kinesis.us-east-1.amazonaws.com)| -|`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against node failure.|no (default == 1)| -|`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. See 'Capacity Planning' below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKinesisPartitions}`.|no (default == 1)| -|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment. Note that segments are only pushed to deep storage and loadable by historical nodes when the indexing task completes.|no (default == PT1H)| -|`startDelay`|ISO8601 Period|The period to wait before the supervisor starts managing tasks.|no (default == PT5S)| -|`period`|ISO8601 Period|How often the supervisor will execute its management logic. Note that the supervisor will also run in response to certain events (such as tasks succeeding, failing, and reaching their taskDuration) so this value specifies the maximum time between iterations.|no (default == PT30S)| -|`useEarliestSequenceNumber`|Boolean|If a supervisor is managing a dataSource for the first time, it will obtain a set of starting sequence numbers from Kinesis. This flag determines whether it retrieves the earliest or latest offsets in Kinesis. Under normal circumstances, subsequent tasks will start from where the previous segments ended so this flag will only be used on first run.|no (default == false)| -|`completionTimeout`|ISO8601 Period|The length of time to wait before declaring a publishing task as failed and terminating it. If this is set too low, your tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|no (default == PT6H)| -|`lateMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps earlier than this period before the task was created; for example if this is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps earlier than *2016-01-01T11:00Z* will be dropped. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments (e.g. a realtime and a nightly batch ingestion pipeline).|no (default == none)| -|`recordsPerFetch`|Integer|The number of records to request per GetRecords call to Kinesis. See 'Determining Fetch Settings' below.|no (default == 2000)| -|`fetchDelayMillis`|Integer|Time in milliseconds to wait between subsequent GetRecords calls to Kinesis. See 'Determining Fetch Settings' below.|no (default == 1000)| -|`awsAccessKeyId`|String|The AWS access key ID to use for Kinesis API requests. If this is not provided, the service will look for credentials set in the environment variables, system properties, in the default profile configuration file, and from the EC2 instance profile provider (in this order).|no| -|`awsSecretAccessKey`|String|The AWS secret access key to use for Kinesis API requests. Only used if `awsAccessKeyId` is also provided.|no| - -## Determining Fetch Settings - -Kinesis places the following restrictions on calls to fetch records: - -- Each data record can be up to 1 MB in size. -- Each shard can support up to 5 transactions per second for reads. -- Each shard can read up to 2 MB per second. -- The maximum size of data that GetRecords can return is 10 MB. - -Values for `recordsPerFetch` and `fetchDelayMillis` should be chosen to maximize throughput under the above constraints. -The values that you choose will depend on the average size of a record and the number of consumers you have reading from -a given shard (which will be `replicas` unless you have other consumers also reading from this Kinesis stream). - -If the above limits are violated, AWS will throw ProvisionedThroughputExceededException errors on subsequent calls to -read data. When this happens, the Kinesis indexing service will pause by `fetchDelayMillis` and then attempt the call -again. - -## Supervisor API, Capacity Planning, Persistence, and Schema Changes - -The Kinesis indexing service uses the same supervisor API and has the same considerations for capacity planning, -persistence, and schema changes as the Kafka indexing service. For documentation on these topics, see the relevant -sections of the Kafka indexing service [documentation](http://druid.io/docs/0.10.0/development/extensions-core/kafka-ingestion.html#supervisor-api). From 1b027e0f2c8b991bdeeebc2e7430fadde03947ea Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 15 Nov 2018 13:38:40 -0800 Subject: [PATCH 46/87] fix deserialization bug --- .../supervisor/SeekableStreamSupervisorIOConfig.java | 1 + 1 file changed, 1 insertion(+) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java index e6c0baa1cbef..88d343546532 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java @@ -72,6 +72,7 @@ private static Duration defaultDuration(final Period period, final String theDef return (period == null ? new Period(theDefault) : period).toStandardDuration(); } + @JsonProperty public String getStream() { return stream; From 308268e45d282fc421ea5d2e26a3d8f002d6a6a7 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Fri, 16 Nov 2018 14:43:32 -0800 Subject: [PATCH 47/87] fix various bugs --- .../indexing/kinesis/KinesisIOConfig.java | 4 ++- .../kinesis/KinesisRecordSupplier.java | 3 +- .../indexing/kinesis/KinesisTuningConfig.java | 2 +- .../kinesis/supervisor/KinesisSupervisor.java | 20 ++++++++++-- .../indexing/kinesis/KinesisIOConfigTest.java | 3 ++ .../kinesis/KinesisIndexTaskTest.java | 32 +++++++++++++++++-- .../supervisor/KinesisSupervisorTest.java | 1 + .../SeekableStreamIndexTaskRunner.java | 8 +++-- .../supervisor/SeekableStreamSupervisor.java | 27 +++++++++------- .../SeekableStreamSupervisorIOConfig.java | 1 + 10 files changed, 79 insertions(+), 22 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java index e6a3be9810ac..326f30b64bde 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java @@ -27,6 +27,7 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.util.Set; public class KinesisIOConfig extends SeekableStreamIOConfig @@ -48,6 +49,7 @@ public class KinesisIOConfig extends SeekableStreamIOConfig @JsonCreator public KinesisIOConfig( + @JsonProperty("taskGroupId") @Nullable Integer taskGroupId, @JsonProperty("baseSequenceName") String baseSequenceName, @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, @@ -67,7 +69,7 @@ public KinesisIOConfig( ) { super( - null, + taskGroupId, baseSequenceName, startPartitions, endPartitions, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index c9f7a23f4770..25780353c88e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -620,10 +620,9 @@ private String getSequenceNumberInternal(StreamPartition partition, Stri GetRecordsResult recordsResult; try { - recordsResult = kinesis.getRecords(new GetRecordsRequest().withShardIterator(shardIterator).withLimit(1)); + recordsResult = kinesis.getRecords(new GetRecordsRequest().withShardIterator(shardIterator).withLimit(10000)); } catch (ProvisionedThroughputExceededException e) { - log.warn("Exceeded provisioned throughput, retrying in [%,dms]", PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS); try { Thread.sleep(PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS); continue; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java index 767aec18ebad..7b0ceaa0ccdc 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java @@ -37,7 +37,7 @@ public class KinesisTuningConfig extends SeekableStreamTuningConfig private static final int DEFAULT_RECORD_BUFFER_SIZE = 10000; private static final int DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT = 5000; private static final int DEFAULT_RECORD_BUFFER_FULL_WAIT = 5000; - private static final int DEFAULT_FETCH_SEQUENCE_NUMBER_TIMEOUT = 10000; + private static final int DEFAULT_FETCH_SEQUENCE_NUMBER_TIMEOUT = 20000; private static final int DEFAULT_MAX_RECORDS_PER_POLL = 5; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index d07e2d4308a3..f40fc60a267e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.kinesis.supervisor; import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; @@ -115,6 +116,7 @@ protected SeekableStreamIOConfig createIoConfig( { KinesisSupervisorIOConfig ioConfig = (KinesisSupervisorIOConfig) ioConfigg; return new KinesisIOConfig( + groupId, baseSequenceName, new SeekableStreamPartitions<>(ioConfig.getStream(), startPartitions), new SeekableStreamPartitions<>(ioConfig.getStream(), endPartitions), @@ -143,8 +145,22 @@ protected List> createIndexTasks( SeekableStreamIOConfig taskIoConfig, SeekableStreamTuningConfig taskTuningConfig, RowIngestionMetersFactory rowIngestionMetersFactory - ) + ) throws JsonProcessingException { + final String checkpoints = sortingMapper.writerWithType(new TypeReference>>() + { + }).writeValueAsString(sequenceOffsets); + final Map context = spec.getContext() == null + ? ImmutableMap.of( + "checkpoints", + checkpoints, + IS_INCREMENTAL_HANDOFF_SUPPORTED, + true + ) : ImmutableMap.builder() + .put("checkpoints", checkpoints) + .put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true) + .putAll(spec.getContext()) + .build(); List> taskList = new ArrayList<>(); for (int i = 0; i < replicas; i++) { String taskId = Joiner.on("_").join(baseSequenceName, getRandomId()); @@ -154,7 +170,7 @@ protected List> createIndexTasks( spec.getDataSchema(), (KinesisTuningConfig) taskTuningConfig, (KinesisIOConfig) taskIoConfig, - spec.getContext(), + context, null, null, rowIngestionMetersFactory diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index e20d2944074d..7557f639bed2 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -66,6 +66,7 @@ public void testSerdeWithDefaults() throws Exception ), IOConfig.class ); + Assert.assertNull(config.getTaskGroupId()); Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); Assert.assertEquals("mystream", config.getStartPartitions().getStream()); Assert.assertEquals( @@ -96,6 +97,7 @@ public void testSerdeWithDefaults() throws Exception public void testSerdeWithNonDefaults() throws Exception { String jsonStr = "{\n" + + " \"taskGroupId\": 0,\n" + " \"type\": \"kinesis\",\n" + " \"baseSequenceName\": \"my-sequence-name\",\n" + " \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n" @@ -124,6 +126,7 @@ public void testSerdeWithNonDefaults() throws Exception ), IOConfig.class ); + Assert.assertEquals((Integer) 0, config.getTaskGroupId()); Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); Assert.assertEquals("mystream", config.getStartPartitions().getStream()); Assert.assertEquals( diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 0325c6d493a8..f5da32c71ed7 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -427,6 +427,7 @@ public void testRunAfterDataInserted() throws Exception final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -505,6 +506,7 @@ public void testRunBeforeDataInserted() throws Exception final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, @@ -634,6 +636,7 @@ public void testIncrementalHandOff() throws Exception final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, baseSequenceName, startPartitions, endPartitions, @@ -657,8 +660,9 @@ public void testIncrementalHandOff() throws Exception Thread.sleep(10); } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets) || checkpoint2.getPartitionSequenceNumberMap() - .equals(currentOffsets)); + Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets) + || checkpoint2.getPartitionSequenceNumberMap() + .equals(currentOffsets)); task.getRunner().setEndOffsets(currentOffsets, false); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); @@ -756,6 +760,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, baseSequenceName, startPartitions, endPartitions, @@ -888,6 +893,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, baseSequenceName, startPartitions, endPartitions, @@ -965,6 +971,7 @@ public void testRunWithMinimumMessageTime() throws Exception final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1035,6 +1042,7 @@ public void testRunWithMaximumMessageTime() throws Exception final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1115,6 +1123,7 @@ public void testRunWithTransformSpec() throws Exception ) ), new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1184,6 +1193,7 @@ public void testRunOnNothing() throws Exception final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1235,6 +1245,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1301,6 +1312,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1369,6 +1381,7 @@ public void testReportParseExceptions() throws Exception final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1423,6 +1436,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1524,6 +1538,7 @@ public void testMultipleParseExceptionsFailure() throws Exception final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1601,6 +1616,7 @@ public void testRunReplicas() throws Exception final KinesisIndexTask task1 = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1628,6 +1644,7 @@ public void testRunReplicas() throws Exception final KinesisIndexTask task2 = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1699,6 +1716,7 @@ public void testRunConflicting() throws Exception final KinesisIndexTask task1 = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1726,6 +1744,7 @@ public void testRunConflicting() throws Exception final KinesisIndexTask task2 = createTask( null, new KinesisIOConfig( + null, "sequence1", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1798,6 +1817,7 @@ public void testRunConflictingWithoutTransactions() throws Exception final KinesisIndexTask task1 = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1825,6 +1845,7 @@ public void testRunConflictingWithoutTransactions() throws Exception final KinesisIndexTask task2 = createTask( null, new KinesisIOConfig( + null, "sequence1", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1895,6 +1916,7 @@ public void testRunOneTaskTwoPartitions() throws Exception final KinesisIndexTask task = createTask( null, new KinesisIOConfig( + null, "sequence1", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -1976,6 +1998,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception final KinesisIndexTask task1 = createTask( null, new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -2003,6 +2026,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception final KinesisIndexTask task2 = createTask( null, new KinesisIOConfig( + null, "sequence1", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, @@ -2087,6 +2111,7 @@ public void testRestore() throws Exception final KinesisIndexTask task1 = createTask( "task1", new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -2132,6 +2157,7 @@ public void testRestore() throws Exception final KinesisIndexTask task2 = createTask( task1.getId(), new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -2206,6 +2232,7 @@ public void testRunWithPauseAndResume() throws Exception final KinesisIndexTask task = createTask( "task1", new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, @@ -2314,6 +2341,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception final KinesisIndexTask task = createTask( "task1", new KinesisIOConfig( + null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 2bfee014be13..4088433dad1c 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -3450,6 +3450,7 @@ private KinesisIndexTask createKinesisIndexTask( getDataSchema(dataSource), tuningConfig, new KinesisIOConfig( + null, "sequenceName-" + taskGroupId, startPartitions, endPartitions, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 48332c290b35..85ab1301aa7e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -1122,8 +1122,12 @@ private void possiblyResetDataSourceMetadata( SequenceType sequence = currOffsets.get(streamPartition.getPartitionId()); if (!tuningConfig.isSkipSequenceNumberAvailabilityCheck()) { SequenceType earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); - if (earliestSequenceNumber == null - || createSequencenNumber(earliestSequenceNumber).compareTo(createSequencenNumber(sequence)) > 0) { + if (earliestSequenceNumber == null) { + log.warn( + "unable to verify sequence number[%s] availability, unable to fetch earliest sequence number", + sequence + ); + } else if (createSequencenNumber(earliestSequenceNumber).compareTo(createSequencenNumber(sequence)) > 0) { if (tuningConfig.isResetOffsetAutomatically()) { log.info("Attempting to reset sequences automatically for all partitions"); try { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 43b0169a647e..eb1a59f3c7ac 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -356,18 +356,21 @@ public void handle() throws ExecutionException, InterruptedException }) .findAny() .map(Entry::getKey); - taskGroupId = maybeGroupId.orElse( - pendingCompletionTaskGroups - .entrySet() - .stream() - .filter(entry -> { - final List taskGroups = entry.getValue(); - return taskGroups.stream().anyMatch(group -> group.baseSequenceName.equals(baseSequenceName)); - }) - .findAny() - .orElseThrow(() -> new ISE("Cannot find taskGroup for baseSequenceName[%s]", baseSequenceName)) - .getKey() - ); + + if (maybeGroupId.isPresent()) { + taskGroupId = maybeGroupId.get(); + } else { + taskGroupId = pendingCompletionTaskGroups + .entrySet() + .stream() + .filter(entry -> { + final List taskGroups = entry.getValue(); + return taskGroups.stream().anyMatch(group -> group.baseSequenceName.equals(baseSequenceName)); + }) + .findAny() + .orElseThrow(() -> new ISE("Cannot find taskGroup for baseSequenceName[%s]", baseSequenceName)) + .getKey(); + } } else { taskGroupId = nullableTaskGroupId; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java index 88d343546532..f758f68b00de 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java @@ -108,6 +108,7 @@ public Duration getPeriod() return period; } + @JsonProperty public boolean isUseEarliestSequenceNumber() { return useEarliestSequenceNumber; From 5437df3d86e18d7d220f01a88f65e47c3f176548 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Fri, 16 Nov 2018 17:26:41 -0800 Subject: [PATCH 48/87] KinesisRecordSupplier unable to catch up to earliest position in stream bug fix --- ...ementalPublishingKafkaIndexTaskRunner.java | 6 ---- .../kafka/LegacyKafkaIndexTaskRunner.java | 10 ++---- .../kinesis/KinesisIndexTaskRunner.java | 6 ---- .../kinesis/KinesisRecordSupplier.java | 1 - .../SeekableStreamIndexTaskRunner.java | 33 ++++++++++--------- 5 files changed, 19 insertions(+), 37 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index fd55d653eefb..f440d1befc81 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -188,12 +188,6 @@ private void possiblyResetOffsetsOrWait( } } - @Override - protected OrderedSequenceNumber createSequencenNumber(Long sequenceNumber) - { - return KafkaSequenceNumber.of(sequenceNumber); - } - @Override protected SeekableStreamDataSourceMetadata createDataSourceMetadata( SeekableStreamPartitions partitions diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 7486b48cda6a..e587917c36ae 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -714,12 +714,6 @@ protected SeekableStreamPartitions createSeekableStreamPartitions return null; } - @Override - protected OrderedSequenceNumber createSequencenNumber(Long sequenceNumber) - { - return null; - } - private void possiblyResetOffsetsOrWait( Map outOfRangePartitions, KafkaConsumer consumer, @@ -886,7 +880,7 @@ public void stopGracefully() } try { - if (pauseLock.tryLock(KafkaIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { + if (pauseLock.tryLock(SeekableStreamIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { try { if (pauseRequested) { pauseRequested = false; @@ -902,7 +896,7 @@ public void stopGracefully() return; } - if (pollRetryLock.tryLock(KafkaIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { + if (pollRetryLock.tryLock(SeekableStreamIndexTask.LOCK_ACQUIRE_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { try { isAwaitingRetry.signalAll(); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index f9ba0b3998e7..eb0beac9755e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -109,12 +109,6 @@ protected SeekableStreamPartitions createSeekableStreamPartition )); } - @Override - protected OrderedSequenceNumber createSequencenNumber(String sequenceNumber) - { - return KinesisSequenceNumber.of(sequenceNumber); - } - @Override protected SeekableStreamDataSourceMetadata createDataSourceMetadata( SeekableStreamPartitions partitions diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 25780353c88e..1cca9fc36671 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -215,7 +215,6 @@ private Runnable getRecordRunnable() } catch (ProvisionedThroughputExceededException e) { long retryMs = Math.max(PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS, fetchDelayMillis); - log.warn("Exceeded provisioned throughput, retrying in [%,dms]", retryMs); rescheduleRunnable(retryMs); } catch (Throwable e) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 85ab1301aa7e..ddb22f507038 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -355,10 +355,10 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception // so either this is a brand new task or replacement of a failed task Preconditions.checkState(sequences.get(0).startOffsets.entrySet().stream().allMatch( partitionOffsetEntry -> - createSequencenNumber(partitionOffsetEntry.getValue()).compareTo( - createSequencenNumber(ioConfig.getStartPartitions() - .getPartitionSequenceNumberMap() - .get(partitionOffsetEntry.getKey()) + createSequenceNumber(partitionOffsetEntry.getValue()).compareTo( + createSequenceNumber(ioConfig.getStartPartitions() + .getPartitionSequenceNumberMap() + .get(partitionOffsetEntry.getKey()) )) >= 0 ), "Sequence sequences are not compatible with start sequences of task"); currOffsets.putAll(sequences.get(0).startOffsets); @@ -513,8 +513,8 @@ public void run() if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(record.getSequenceNumber())) { // shard is closed currOffsets.put(record.getPartitionId(), record.getSequenceNumber()); - } else if (createSequencenNumber(record.getSequenceNumber()).compareTo( - createSequencenNumber(endOffsets.get(record.getPartitionId()))) < 0) { + } else if (createSequenceNumber(record.getSequenceNumber()).compareTo( + createSequenceNumber(endOffsets.get(record.getPartitionId()))) < 0) { if (!record.getSequenceNumber().equals(currOffsets.get(record.getPartitionId()))) { @@ -1041,7 +1041,7 @@ private Set> assignPartitions( final SequenceType endOffset = endOffsets.get(entry.getKey()); if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(endOffset) || SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) - || createSequencenNumber(entry.getValue()).compareTo(createSequencenNumber(endOffset)) < 0) { + || createSequenceNumber(entry.getValue()).compareTo(createSequenceNumber(endOffset)) < 0) { assignment.add(StreamPartition.of(stream, entry.getKey())); } else if (entry.getValue().equals(endOffset)) { log.info("Finished reading partition[%s].", entry.getKey()); @@ -1122,12 +1122,16 @@ private void possiblyResetDataSourceMetadata( SequenceType sequence = currOffsets.get(streamPartition.getPartitionId()); if (!tuningConfig.isSkipSequenceNumberAvailabilityCheck()) { SequenceType earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); + /* if (earliestSequenceNumber == null) { log.warn( "unable to verify sequence number[%s] availability, unable to fetch earliest sequence number", sequence ); - } else if (createSequencenNumber(earliestSequenceNumber).compareTo(createSequencenNumber(sequence)) > 0) { + } + */ + if (earliestSequenceNumber == null + || createSequenceNumber(earliestSequenceNumber).compareTo(createSequenceNumber(sequence)) > 0) { if (tuningConfig.isResetOffsetAutomatically()) { log.info("Attempting to reset sequences automatically for all partitions"); try { @@ -1752,9 +1756,9 @@ boolean canHandle(OrderedPartitionableRecord record { lock.lock(); try { - final OrderedSequenceNumber partitionEndOffset = createSequencenNumber(endOffsets.get(record.getPartitionId())); - final OrderedSequenceNumber partitionStartOffset = createSequencenNumber(startOffsets.get(record.getPartitionId())); - final OrderedSequenceNumber recordOffset = createSequencenNumber(record.getSequenceNumber()); + final OrderedSequenceNumber partitionEndOffset = createSequenceNumber(endOffsets.get(record.getPartitionId())); + final OrderedSequenceNumber partitionStartOffset = createSequenceNumber(startOffsets.get(record.getPartitionId())); + final OrderedSequenceNumber recordOffset = createSequenceNumber(record.getSequenceNumber()); return isOpen() && recordOffset != null && partitionEndOffset != null @@ -1814,8 +1818,8 @@ public Object getMetadata() for (Map.Entry partitionOffset : endOffsets.entrySet()) { SequenceType newOffsets = partitionOffset.getValue(); if (lastPersistedOffsets.containsKey(partitionOffset.getKey()) && - createSequencenNumber(lastPersistedOffsets.get(partitionOffset.getKey())).compareTo( - createSequencenNumber(newOffsets)) > 0) { + createSequenceNumber(lastPersistedOffsets.get(partitionOffset.getKey())).compareTo( + createSequenceNumber(newOffsets)) > 0) { newOffsets = lastPersistedOffsets.get(partitionOffset.getKey()); } lastPersistedOffsets.put( @@ -1940,9 +1944,6 @@ protected abstract SeekableStreamPartitions createS Object obeject ); - protected abstract OrderedSequenceNumber createSequencenNumber(SequenceType sequenceNumber); - - @NotNull protected abstract List> getRecords( RecordSupplier recordSupplier, From 2b87b99062a7f41fd8b4bff9720f10b68701e291 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 19 Nov 2018 15:29:25 -0800 Subject: [PATCH 49/87] minor changes to kinesis --- .../apache/druid/indexing/kinesis/KinesisRecordSupplier.java | 2 +- .../apache/druid/indexing/kinesis/KinesisTuningConfig.java | 2 +- .../druid/indexing/kinesis/KinesisTuningConfigTest.java | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 1cca9fc36671..bd613ea2ef40 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -619,7 +619,7 @@ private String getSequenceNumberInternal(StreamPartition partition, Stri GetRecordsResult recordsResult; try { - recordsResult = kinesis.getRecords(new GetRecordsRequest().withShardIterator(shardIterator).withLimit(10000)); + recordsResult = kinesis.getRecords(new GetRecordsRequest().withShardIterator(shardIterator).withLimit(1000)); } catch (ProvisionedThroughputExceededException e) { try { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java index 7b0ceaa0ccdc..1f70a3460c22 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java @@ -34,7 +34,7 @@ public class KinesisTuningConfig extends SeekableStreamTuningConfig { - private static final int DEFAULT_RECORD_BUFFER_SIZE = 10000; + private static final int DEFAULT_RECORD_BUFFER_SIZE = 100000; private static final int DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT = 5000; private static final int DEFAULT_RECORD_BUFFER_FULL_WAIT = 5000; private static final int DEFAULT_FETCH_SEQUENCE_NUMBER_TIMEOUT = 20000; diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java index 2251f22f6773..4d5cc92176ed 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java @@ -71,10 +71,10 @@ public void testSerdeWithDefaults() throws Exception Assert.assertTrue(config.getBuildV9Directly()); Assert.assertFalse(config.isReportParseExceptions()); Assert.assertEquals(0, config.getHandoffConditionTimeout()); - Assert.assertEquals(10000, config.getRecordBufferSize()); + Assert.assertEquals(100000, config.getRecordBufferSize()); Assert.assertEquals(5000, config.getRecordBufferOfferTimeout()); Assert.assertEquals(5000, config.getRecordBufferFullWait()); - Assert.assertEquals(10000, config.getFetchSequenceNumberTimeout()); + Assert.assertEquals(20000, config.getFetchSequenceNumberTimeout()); Assert.assertNull(config.getFetchThreads()); Assert.assertFalse(config.isSkipSequenceNumberAvailabilityCheck()); Assert.assertFalse(config.isResetOffsetAutomatically()); From 212b59ce25bcb12013b44b8c61c9b7de7b8d6b44 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 20 Nov 2018 15:02:44 -0800 Subject: [PATCH 50/87] implement deaggregate for kinesis --- .../kinesis-indexing-service/pom.xml | 7 ++ .../indexing/kinesis/KinesisIndexTask.java | 1 + .../kinesis/KinesisRecordSupplier.java | 49 ++++++++++- .../kinesis/supervisor/KinesisSupervisor.java | 3 + .../kinesis/KinesisRecordSupplierTest.java | 83 +++++++++++++++++-- .../SeekableStreamIndexTask.java | 3 +- .../supervisor/SeekableStreamSupervisor.java | 3 +- pom.xml | 2 + 8 files changed, 138 insertions(+), 13 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index 2f7d7d0ddf96..ed6f90ca77a5 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -78,6 +78,13 @@ ${aws.sdk.version} + + com.amazonaws + amazon-kinesis-client + 1.9.2 + provided + + org.easymock diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index f744343a8e66..3c2d7b559bde 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -70,6 +70,7 @@ public KinesisIndexTask( @Override protected RecordSupplier newRecordSupplier() + throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException { int fetchThreads = tuningConfig.getFetchThreads() != null ? tuningConfig.getFetchThreads() diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index bd613ea2ef40..e6651b4911a0 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -50,6 +50,9 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.lang.invoke.MethodHandle; +import java.lang.invoke.MethodHandles; +import java.lang.reflect.Method; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; @@ -165,10 +168,26 @@ private Runnable getRecordRunnable() // list will come back empty if there are no records for (Record kinesisRecord : recordsResult.getRecords()) { + final List data; + if (deaggregate) { + if (deaggregateHandle == null || getDataHandle == null) { + throw new ISE("deaggregateHandle or getDataHandle is null!"); + } + + data = new ArrayList<>(); - data = Collections.singletonList(toByteArray(kinesisRecord.getData())); + final List userRecords = (List) deaggregateHandle.invokeExact( + Collections.singletonList(kinesisRecord) + ); + + for (Object userRecord : userRecords) { + data.add(toByteArray((ByteBuffer) getDataHandle.invoke(userRecord))); + } + } else { + data = Collections.singletonList(toByteArray(kinesisRecord.getData())); + } final OrderedPartitionableRecord record = new OrderedPartitionableRecord<>( streamPartition.getStream(), @@ -244,6 +263,10 @@ private void rescheduleRunnable(long delayMillis) } } + // used for deaggregate + private final MethodHandle deaggregateHandle; + private final MethodHandle getDataHandle; + private final int recordsPerFetch; private final int fetchDelayMillis; private final boolean deaggregate; @@ -279,7 +302,7 @@ public KinesisRecordSupplier( int recordBufferFullWait, int fetchSequenceNumberTimeout, int maxRecordsPerPoll - ) + ) throws ClassNotFoundException, IllegalAccessException, NoSuchMethodException { this.recordsPerFetch = recordsPerFetch; this.fetchDelayMillis = fetchDelayMillis; @@ -291,6 +314,28 @@ public KinesisRecordSupplier( this.fetchThreads = fetchThreads; this.recordBufferSize = recordBufferSize; + if (deaggregate) { + try { + Class KCLUserRecordclass = Class.forName("com.amazonaws.services.kinesis.clientlibrary.types.UserRecord"); + MethodHandles.Lookup lookup = MethodHandles.publicLookup(); + + Method deaggregateMethod = KCLUserRecordclass.getMethod("deaggregate", List.class); + Method getDataMethod = KCLUserRecordclass.getMethod("getData"); + + deaggregateHandle = lookup.unreflect(deaggregateMethod); + getDataHandle = lookup.unreflect(getDataMethod); + } + catch (ClassNotFoundException e) { + log.error( + "cannot find class[com.amazonaws.services.kinesis.clientlibrary.types.UserRecord], " + + "note that when using deaggregate=true, you must provide the Kinesis Client Library jar in the classpath"); + throw e; + } + } else { + deaggregateHandle = null; + getDataHandle = null; + } + AWSCredentialsProvider awsCredentialsProvider = AWSCredentialsUtils.defaultAWSCredentialsProviderChain( new ConstructibleAWSCredentialsConfig(awsAccessKeyId, awsSecretAccessKey) ); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 193d83fd17bb..e85975e42afc 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -181,9 +181,11 @@ protected List> createIndexTasks( @Override protected RecordSupplier setupRecordSupplier() + throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException { KinesisSupervisorIOConfig ioConfig = spec.getIoConfig(); KinesisTuningConfig taskTuningConfig = spec.getTuningConfig(); + return new KinesisRecordSupplier( ioConfig.getEndpoint(), ioConfig.getAwsAccessKeyId(), @@ -200,6 +202,7 @@ protected RecordSupplier setupRecordSupplier() taskTuningConfig.getFetchSequenceNumberTimeout(), taskTuningConfig.getMaxRecordsPerPoll() ); + } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index 657c3523dd9f..63a04f8b056f 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -191,7 +191,8 @@ public void tearDownTest() } @Test - public void testSupplierSetup() throws InterruptedException + public void testSupplierSetup() + throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { getKinesisClientInstance(); @@ -228,7 +229,8 @@ public void testSupplierSetup() throws InterruptedException } @Test - public void testPoll() throws InterruptedException + public void testPoll() + throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { AmazonKinesis kinesis = getKinesisClientInstance(); List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); @@ -262,6 +264,7 @@ public void testPoll() throws InterruptedException 60000, 100 ); + recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); @@ -277,7 +280,8 @@ public void testPoll() throws InterruptedException } @Test - public void testPollAfterMoreDataAdded() throws InterruptedException + public void testPollAfterMoreDataAdded() + throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { AmazonKinesis kinesis = getKinesisClientInstance(); List insertDataResults1 = insertData(kinesis, generateRecordsRequests(stream, 0, 5)); @@ -339,7 +343,8 @@ public void testPollAfterMoreDataAdded() throws InterruptedException } @Test - public void testSeek() throws InterruptedException + public void testSeek() + throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { AmazonKinesis kinesis = getKinesisClientInstance(); List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); @@ -407,7 +412,8 @@ public void testSeek() throws InterruptedException } @Test - public void testSeekToLatest() throws InterruptedException + public void testSeekToLatest() + throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { AmazonKinesis kinesis = getKinesisClientInstance(); List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); @@ -436,6 +442,7 @@ public void testSeekToLatest() throws InterruptedException 5 ); + recordSupplier.assign(partitions); Assert.assertEquals(insertDataResults.get(0).getSequenceNumber(), recordSupplier.getEarliestSequenceNumber(shard1)); @@ -446,7 +453,8 @@ public void testSeekToLatest() throws InterruptedException } @Test(expected = ISE.class) - public void testSeekUnassigned() throws InterruptedException + public void testSeekUnassigned() + throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { AmazonKinesis kinesis = getKinesisClientInstance(); List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); @@ -482,7 +490,8 @@ public void testSeekUnassigned() throws InterruptedException } @Test - public void testPollAfterSeek() throws InterruptedException + public void testPollAfterSeek() + throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { AmazonKinesis kinesis = getKinesisClientInstance(); List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); @@ -507,6 +516,7 @@ public void testPollAfterSeek() throws InterruptedException 60000, 1 ); + recordSupplier.assign(partitions); recordSupplier.seek(StreamPartition.of(stream, shardId1), getSequenceNumber(insertDataResults, shardId1, 5)); @@ -545,7 +555,8 @@ public void testPollAfterSeek() throws InterruptedException } @Test - public void testPosition() throws InterruptedException + public void testPosition() + throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { AmazonKinesis kinesis = getKinesisClientInstance(); List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); @@ -571,6 +582,7 @@ public void testPosition() throws InterruptedException 60000, 1 ); + recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); @@ -607,7 +619,8 @@ public void testPosition() throws InterruptedException } @Test - public void testPositionAfterPollBatch() throws InterruptedException + public void testPositionAfterPollBatch() + throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { AmazonKinesis kinesis = getKinesisClientInstance(); List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); @@ -633,6 +646,7 @@ public void testPositionAfterPollBatch() throws InterruptedException 60000, 3 ); + recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); @@ -648,6 +662,57 @@ public void testPositionAfterPollBatch() throws InterruptedException Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 3), recordSupplier.getPosition(partition1)); } + @Test + public void testDeaggregate() + throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException + { + AmazonKinesis kinesis = getKinesisClientInstance(); + List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); + Set> initialRecords = insertDataResults.stream() + .map(r -> new OrderedPartitionableRecord<>( + stream, + r.getShardId(), + r.getSequenceNumber(), + null + )) + .collect(Collectors.toSet()); + + Set> partitions = ImmutableSet.of( + StreamPartition.of(stream, shardId0), + StreamPartition.of(stream, shardId1) + ); + + recordSupplier = new KinesisRecordSupplier( + LocalstackTestRunner.getEndpointKinesis(), + TestUtils.TEST_ACCESS_KEY, + TestUtils.TEST_SECRET_KEY, + 100, + 0, + 2, + null, + null, + true, + 100, + 5000, + 5000, + 60000, + 100 + ); + + recordSupplier.assign(partitions); + recordSupplier.seekToEarliest(partitions); + + List> polledRecords = recordSupplier.poll(poll_timeout_millis); + for (int i = 0; polledRecords.size() != initialRecords.size() && i < pollRetry; i++) { + polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); + Thread.sleep(200); + } + + Assert.assertEquals(partitions, recordSupplier.getAssignment()); + Assert.assertEquals(initialRecords.size(), polledRecords.size()); + Assert.assertTrue(polledRecords.containsAll(initialRecords)); + } + private static String getSequenceNumber(List entries, String shardId, int sequence) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index e3183f397bf0..ffea28a32862 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -245,7 +245,8 @@ public StreamAppenderatorDriver newDriver( ); } - protected abstract RecordSupplier newRecordSupplier(); + protected abstract RecordSupplier newRecordSupplier() + throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException; public boolean withinMinMaxRecordTime(final InputRow row) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index c7aadf3b49fb..8267ae440486 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -2756,7 +2756,8 @@ protected abstract OrderedSequenceNumber makeSequenceNumber( * * @return specific instance of Kafka/Kinesis RecordSupplier */ - protected abstract RecordSupplier setupRecordSupplier(); + protected abstract RecordSupplier setupRecordSupplier() + throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException; /** * creates a specific instance of Kafka/Kinesis Supervisor Report Payload diff --git a/pom.xml b/pom.xml index b8f5d9400d86..5f55b3a70f39 100644 --- a/pom.xml +++ b/pom.xml @@ -1000,6 +1000,8 @@ sun.nio.ch.DirectBuffer sun.misc.Cleaner sun.misc.Unsafe + + java.lang.invoke.MethodHandle From 624c35e3b29800d4536d1e041c0527dd6f8cd37d Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 20 Nov 2018 15:07:15 -0800 Subject: [PATCH 51/87] Merge remote-tracking branch 'upstream/master' into seekablestream --- .../druid/common/aws/AWSClientConfig.java | 1 + .../benchmark/FilterPartitionBenchmark.java | 7 ++--- .../druid/benchmark/TimeParseBenchmark.java | 1 + .../timecompare/TimeCompareBenchmark.java | 1 + codestyle/checkstyle-suppressions.xml | 10 +++++-- codestyle/checkstyle.xml | 28 ++++++++++++++----- .../collections/DummyNonBlockingPool.java | 1 + .../data/SearchableVersionedDataFinder.java | 4 +-- .../druid/data/input/impl/SqlFirehose.java | 1 + .../input/impl/prefetch/JsonIterator.java | 1 + .../input/impl/prefetch/PrefetchConfig.java | 1 + .../apache/druid/indexer/RunnerTaskState.java | 5 +++- .../org/apache/druid/indexer/TaskInfo.java | 1 + .../druid/java/util/common/FileUtils.java | 1 + .../guava/FilteringYieldingAccumulator.java | 1 + .../java/util/common/guava/MergeIterator.java | 1 + .../druid/java/util/common/io/NativeIO.java | 6 ++-- .../emitter/core/BaseHttpEmittingConfig.java | 1 + .../util/emitter/core/HttpPostEmitter.java | 1 - .../druid/java/util/metrics/SysMonitor.java | 3 +- .../segment/loading/DataSegmentPusher.java | 1 + .../CloseableDefaultBlockingPool.java | 1 + .../collections/CloseableStupidPool.java | 1 + .../druid/common/utils/SocketUtilTest.java | 1 + .../input/impl/prefetch/JsonIteratorTest.java | 1 + .../java/util/common/StreamUtilsTest.java | 1 + .../timeline/partition/NoneShardSpecTest.java | 1 + docs/content/configuration/index.md | 2 +- eclipse.importorder | 12 +++----- .../metrics/AmbariMetricsEmitterModule.java | 3 +- .../rocketmq/RocketMQDruidModule.java | 1 + .../rocketmq/RocketMQFirehoseFactory.java | 1 + .../graphite/GraphiteEmitterModule.java | 1 + .../input/influx/InfluxExtensionsModule.java | 1 + .../data/input/influx/InfluxParseSpec.java | 1 + .../druid/data/input/influx/InfluxParser.java | 1 + ...fkaEightSimpleConsumerFirehoseFactory.java | 1 - .../DatasourceOptimizerTest.java | 1 + .../data/input/orc/OrcExtensionsModule.java | 1 + .../input/orc/OrcHadoopInputRowParser.java | 1 + .../input/orc/DruidOrcInputFormatTest.java | 1 + .../input/orc/OrcIndexGeneratorJobTest.java | 1 + .../rabbitmq/RabbitMQDruidModule.java | 1 + .../storage/sqlserver/SQLServerConnector.java | 1 + .../SQLServerMetadataStorageModule.java | 1 + .../CustomStatementRewriterTest.java | 1 + .../sqlserver/SQLServerConnectorTest.java | 1 + .../emitter/statsd/DimensionConverter.java | 2 ++ .../druid/emitter/statsd/StatsDEmitter.java | 6 ++-- .../input/thrift/ThriftDeserialization.java | 1 + .../aggregation/TimestampMinMaxModule.java | 1 + ...TypeMapVirtualColumnDimensionSelector.java | 1 + .../MapVirtualColumnDimensionSelector.java | 1 + .../MapVirtualColumnValueSelector.java | 1 + ...TypeMapVirtualColumnDimensionSelector.java | 1 + .../segment/MapVirtualColumnGroupByTest.java | 1 + .../segment/MapVirtualColumnTestBase.java | 1 + .../segment/MapVirtualColumnTopNTest.java | 1 + .../data/input/AvroHadoopInputRowParser.java | 1 + .../data/input/AvroStreamInputRowParser.java | 1 + .../data/input/avro/AvroBytesDecoder.java | 1 + .../data/input/avro/AvroExtensionsModule.java | 1 + .../data/input/avro/AvroValueInputFormat.java | 1 + .../input/avro/AvroValueRecordReader.java | 1 + .../SchemaRegistryBasedAvroBytesDecoder.java | 1 + .../avro/SchemaRepoBasedAvroBytesDecoder.java | 1 + .../Avro1124RESTRepositoryClientWrapper.java | 1 + .../Avro1124SubjectAndIdConverter.java | 1 + .../schemarepo/SubjectAndIdConverter.java | 1 + .../input/AvroHadoopInputRowParserTest.java | 1 + .../input/AvroStreamInputRowParserTest.java | 1 + .../hll/HllSketchBuildAggregator.java | 1 - .../hll/HllSketchBuildAggregatorFactory.java | 5 ++-- .../hll/HllSketchBuildBufferAggregator.java | 17 ++++++----- .../hll/HllSketchJsonSerializer.java | 4 +-- .../hll/HllSketchMergeAggregator.java | 3 +- .../hll/HllSketchMergeAggregatorFactory.java | 5 ++-- .../hll/HllSketchMergeBufferAggregator.java | 11 ++++---- .../datasketches/hll/HllSketchModule.java | 7 ++--- .../hll/HllSketchObjectStrategy.java | 7 ++--- ...tchToEstimateWithBoundsPostAggregator.java | 4 +-- .../hll/HllSketchToStringPostAggregator.java | 1 - .../DoublesSketchObjectStrategy.java | 1 - .../theta/SketchEstimatePostAggregator.java | 1 + .../theta/SketchEstimateWithErrorBounds.java | 1 + .../datasketches/hll/GenerateTestData.java | 11 ++++---- .../SketchEstimateWithErrorBoundsTest.java | 1 + .../guice/BloomFilterSerializersModule.java | 1 + .../security/kerberos/KerberosEscalator.java | 2 +- .../hdfs/HdfsFileTimestampVersionFinder.java | 4 --- .../storage/hdfs/tasklog/HdfsTaskLogs.java | 1 + .../hdfs/tasklog/HdfsTaskLogsConfig.java | 1 + .../org/apache/hadoop/fs/HadoopFsWrapper.java | 4 +-- ...ementalPublishingKafkaIndexTaskRunner.java | 1 + .../kafka/LegacyKafkaIndexTaskRunner.java | 1 + .../kafka/supervisor/KafkaSupervisor.java | 6 ++++ .../kinesis/KinesisIndexTaskRunner.java | 2 -- .../kinesis/supervisor/KinesisSupervisor.java | 6 ++++ .../lookup/namespace/UriCacheGenerator.java | 2 -- .../server/lookup/PollingLookupTest.java | 1 + .../sql/MySQLFirehoseDatabaseConnector.java | 1 + .../avro/DruidParquetAvroInputFormat.java | 1 + .../avro/ParquetAvroHadoopInputRowParser.java | 1 + .../avro/DruidParquetAvroReadSupport.java | 6 ++-- .../s3/CustomServerSideEncryption.java | 1 + .../storage/s3/KmsServerSideEncryption.java | 1 + .../storage/s3/NoopServerSideEncryption.java | 1 + .../druid/storage/s3/S3SSECustomConfig.java | 1 + .../druid/storage/s3/S3SSEKmsConfig.java | 1 + .../storage/s3/S3ServerSideEncryption.java | 1 + .../druid/storage/s3/S3StorageConfig.java | 1 + .../s3/S3TimestampVersionedDataFinder.java | 4 --- .../s3/ServerSideEncryptingAmazonS3.java | 1 + .../storage/s3/ServerSideEncryption.java | 1 + .../hll/HyperLogLogSerdeBenchmarkTest.java | 2 ++ .../druid/indexer/HadoopTuningConfig.java | 1 - .../indexer/HadoopDruidIndexerConfigTest.java | 2 +- .../guice/IndexingServiceModuleHelper.java | 1 + .../common/actions/TaskAuditLogConfig.java | 1 + .../RealtimeAppenderatorIngestionSpec.java | 1 + .../RealtimeAppenderatorTuningConfig.java | 1 + .../parallel/ParallelIndexingProgress.java | 1 + .../SinglePhaseParallelIndexingProgress.java | 1 + .../task/batch/parallel/TaskHistory.java | 1 + .../overlord/HeapMemoryTaskStorage.java | 17 +++++------ .../overlord/MetadataTaskStorage.java | 6 ++-- .../druid/indexing/overlord/TaskStorage.java | 6 ++-- .../overlord/TaskStorageQueryAdapter.java | 4 +-- .../indexing/overlord/WorkerTaskRunner.java | 1 + .../overlord/http/OverlordResource.java | 12 ++++---- .../SeekableStreamIndexTaskRunner.java | 1 + .../supervisor/SeekableStreamSupervisor.java | 6 ++-- .../indexing/overlord/TaskLockboxTest.java | 2 +- .../overlord/http/OverlordResourceTest.java | 14 +++++----- .../supervisor/SupervisorManagerTest.java | 1 + .../guice/ITTLSCertificateCheckerModule.java | 1 - .../org/testng/DruidTestRunnerFactory.java | 5 +--- .../AbstractITRealtimeIndexTaskTest.java | 1 + ...penderatorDriverRealtimeIndexTaskTest.java | 1 + .../indexer/ITNestedQueryPushDownTest.java | 1 + pom.xml | 6 ++-- .../bitmap/RoaringBitmapFactory.java | 1 + .../apache/druid/query/LegacyDataSource.java | 1 + .../druid/query/SubqueryQueryRunner.java | 2 +- .../apache/druid/query/TableDataSource.java | 1 + .../PrefixFilteredDimensionSpec.java | 7 +++-- .../druid/query/filter/SpatialDimFilter.java | 1 + ...ngStringGroupByColumnSelectorStrategy.java | 1 + .../SpecializationService.java | 1 + .../apache/druid/query/scan/ScanQuery.java | 1 + .../druid/query/scan/ScanQueryEngine.java | 1 + .../query/scan/ScanQueryLimitRowIterator.java | 1 + .../query/scan/ScanQueryRunnerFactory.java | 1 + .../druid/query/scan/ScanResultValue.java | 1 + .../apache/druid/segment/AbstractSegment.java | 1 + .../apache/druid/segment/BitmapOffset.java | 2 +- .../QueryableIndexIndexableAdapter.java | 1 - .../druid/segment/filter/SpatialFilter.java | 1 + .../segment/serde/ComplexColumnPartSerde.java | 1 + .../serde/SpatialIndexColumnPartSupplier.java | 1 + .../apache/druid/query/DoubleStorageTest.java | 1 + .../JavaScriptAggregatorBenchmark.java | 2 +- .../CardinalityAggregatorTest.java | 19 ++++--------- .../PrefixFilteredDimensionSpecTest.java | 3 +- .../extraction/LowerExtractionFnTest.java | 1 + .../extraction/UpperExtractionFnTest.java | 1 + .../druid/query/filter/AndDimFilterTest.java | 1 + .../query/filter/InDimFilterSerDesrTest.java | 4 ++- .../groupby/GroupByQueryMergeBufferTest.java | 1 + .../groupby/NestedQueryPushDownTest.java | 1 + .../query/select/SelectQueryRunnerTest.java | 4 ++- .../query/select/SelectQuerySpecTest.java | 6 ++-- .../IndexMergerV9CompatibilityTest.java | 1 - .../segment/data/BitmapCreationBenchmark.java | 1 + .../segment/filter/FilterPartitionTest.java | 7 ++--- .../IncrementalIndexRowSizeTest.java | 1 + .../SQLFirehoseDatabaseConnector.java | 1 + .../lookup/MapLookupExtractorFactory.java | 1 + .../druid/segment/indexing/TuningConfigs.java | 1 + .../LocalFileTimestampVersionFinder.java | 8 ++---- .../realtime/firehose/SqlFirehoseFactory.java | 1 + .../server/audit/SQLAuditManagerProvider.java | 1 + .../coordination/ChangeRequestsSnapshot.java | 1 - .../coordinator/BalancerStrategyFactory.java | 1 + .../BytesAccumulatingResponseHandler.java | 1 + .../coordinator/CoordinatorDynamicConfig.java | 1 + .../CostBalancerStrategyFactory.java | 1 + ...NormalizedCostBalancerStrategyFactory.java | 1 + .../RandomBalancerStrategyFactory.java | 1 + .../server/http/OverlordProxyServlet.java | 2 +- .../initialization/TLSServerConfig.java | 1 + .../metrics/QueryCountStatsMonitor.java | 1 + .../metrics/QueryCountStatsProvider.java | 1 + .../druid/server/router/ConsistentHasher.java | 6 ++-- .../DefaultTLSCertificateCheckerModule.java | 1 - .../druid/server/security/TLSUtils.java | 1 - ...chingClusteredClientFunctionalityTest.java | 1 + .../CachingClusteredClientTestUtils.java | 1 + .../metadata/SQLMetadataConnectorTest.java | 1 + .../appenderator/AppenderatorPlumberTest.java | 1 + .../appenderator/SegmentWithStateTest.java | 1 + .../firehose/IngestSegmentFirehoseTest.java | 8 +++--- .../druid/server/QueryResourceTest.java | 2 +- .../ExceptionCapturingServiceEmitter.java | 1 + .../java/org/apache/druid/cli/MainTest.java | 3 ++ .../druid/sql/calcite/rel/SortProject.java | 1 + .../calcite/schema/SegmentMetadataHolder.java | 1 + .../sql/calcite/schema/SystemSchema.java | 1 + .../sql/calcite/schema/SystemSchemaTest.java | 1 + .../calcite/util/TestServerInventoryView.java | 5 +++- 210 files changed, 327 insertions(+), 193 deletions(-) diff --git a/aws-common/src/main/java/org/apache/druid/common/aws/AWSClientConfig.java b/aws-common/src/main/java/org/apache/druid/common/aws/AWSClientConfig.java index 7c8eb8aa130b..198bae4c086e 100644 --- a/aws-common/src/main/java/org/apache/druid/common/aws/AWSClientConfig.java +++ b/aws-common/src/main/java/org/apache/druid/common/aws/AWSClientConfig.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.common.aws; import com.amazonaws.services.s3.S3ClientOptions; diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java index 3a6df5b7b8ce..810d90ff5532 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java @@ -539,14 +539,11 @@ public boolean supportsBitmapIndex(BitmapIndexSelector selector) private static class NoBitmapSelectorDimFilter extends SelectorDimFilter { - public NoBitmapSelectorDimFilter( - String dimension, - String value, - ExtractionFn extractionFn - ) + NoBitmapSelectorDimFilter(String dimension, String value, ExtractionFn extractionFn) { super(dimension, value, extractionFn); } + @Override public Filter toFilter() { diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/TimeParseBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/TimeParseBenchmark.java index 280461a756cc..474cc79df913 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/TimeParseBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/TimeParseBenchmark.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.benchmark; import com.google.common.base.Function; diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java index d719fe25ff8f..b27e74b8abf1 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java @@ -116,6 +116,7 @@ public class TimeCompareBenchmark private int threshold; protected static final Map scriptDoubleSum = new HashMap<>(); + static { scriptDoubleSum.put("fnAggregate", "function aggregate(current, a) { return current + a }"); scriptDoubleSum.put("fnReset", "function reset() { return 0 }"); diff --git a/codestyle/checkstyle-suppressions.xml b/codestyle/checkstyle-suppressions.xml index c1d9d3deca6f..bf5087c9b00d 100644 --- a/codestyle/checkstyle-suppressions.xml +++ b/codestyle/checkstyle-suppressions.xml @@ -20,8 +20,8 @@ --> + "-//Checkstyle//DTD SuppressionFilter Configuration 1.2//EN" + "https://checkstyle.org/dtds/suppressions_1_2.dtd"> @@ -56,4 +56,10 @@ + + + + + + diff --git a/codestyle/checkstyle.xml b/codestyle/checkstyle.xml index 1acafcd65ea1..16e8c14c8c70 100644 --- a/codestyle/checkstyle.xml +++ b/codestyle/checkstyle.xml @@ -29,17 +29,15 @@ - - - - - - - + + + + + @@ -96,6 +94,12 @@ + + + + + + @@ -199,6 +203,16 @@ + + + + + + + + + diff --git a/core/src/main/java/org/apache/druid/collections/DummyNonBlockingPool.java b/core/src/main/java/org/apache/druid/collections/DummyNonBlockingPool.java index cb4af45aaf69..adbdfdfc4058 100644 --- a/core/src/main/java/org/apache/druid/collections/DummyNonBlockingPool.java +++ b/core/src/main/java/org/apache/druid/collections/DummyNonBlockingPool.java @@ -35,6 +35,7 @@ public static NonBlockingPool instance() private DummyNonBlockingPool() { } + @Override public ResourceHolder take() { diff --git a/core/src/main/java/org/apache/druid/data/SearchableVersionedDataFinder.java b/core/src/main/java/org/apache/druid/data/SearchableVersionedDataFinder.java index 589708cf480d..9033762595b6 100644 --- a/core/src/main/java/org/apache/druid/data/SearchableVersionedDataFinder.java +++ b/core/src/main/java/org/apache/druid/data/SearchableVersionedDataFinder.java @@ -27,8 +27,8 @@ * date version of data given a base descriptor and a matching pattern. "Version" is completely dependent on the * implementation but is commonly equal to the "last modified" timestamp. * - * This is implemented explicitly for URIExtractionNamespaceFunctionFactory - * If you have a use case for this interface beyond URIExtractionNamespaceFunctionFactory please bring it up in the dev list. + * This is implemented explicitly for org.apache.druid.query.lookup.namespace.CacheGenerator + * If you have a use case for this interface beyond CacheGenerator please bring it up in the dev list. * * @param The containing type for the data. A simple example would be URI */ diff --git a/core/src/main/java/org/apache/druid/data/input/impl/SqlFirehose.java b/core/src/main/java/org/apache/druid/data/input/impl/SqlFirehose.java index bda4a84cc77f..3a11a95ca641 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/SqlFirehose.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/SqlFirehose.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.impl; import com.google.common.collect.Iterators; diff --git a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/JsonIterator.java b/core/src/main/java/org/apache/druid/data/input/impl/prefetch/JsonIterator.java index 142938b37e37..7c67556ed852 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/JsonIterator.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/prefetch/JsonIterator.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.impl.prefetch; import com.fasterxml.jackson.core.JsonParser; diff --git a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/PrefetchConfig.java b/core/src/main/java/org/apache/druid/data/input/impl/prefetch/PrefetchConfig.java index 354aeb9cf55f..4ee66459c22b 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/PrefetchConfig.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/prefetch/PrefetchConfig.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.impl.prefetch; import java.util.concurrent.TimeUnit; diff --git a/core/src/main/java/org/apache/druid/indexer/RunnerTaskState.java b/core/src/main/java/org/apache/druid/indexer/RunnerTaskState.java index 221c49fe84b1..b7a28cb28256 100644 --- a/core/src/main/java/org/apache/druid/indexer/RunnerTaskState.java +++ b/core/src/main/java/org/apache/druid/indexer/RunnerTaskState.java @@ -16,9 +16,12 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.indexer; + /** - * This includes the state of a task in the task runner not covered by {@link TaskState}, this state is not stored in database + * This includes the state of a task in the task runner not covered by {@link TaskState}, this state is not stored + * in database */ public enum RunnerTaskState { diff --git a/core/src/main/java/org/apache/druid/indexer/TaskInfo.java b/core/src/main/java/org/apache/druid/indexer/TaskInfo.java index 5be8099179ac..411d87af6a90 100644 --- a/core/src/main/java/org/apache/druid/indexer/TaskInfo.java +++ b/core/src/main/java/org/apache/druid/indexer/TaskInfo.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.indexer; import com.google.common.base.Preconditions; diff --git a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java index 27af26976397..97ac6cb51ad9 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.java.util.common; import com.google.common.base.Predicate; diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/FilteringYieldingAccumulator.java b/core/src/main/java/org/apache/druid/java/util/common/guava/FilteringYieldingAccumulator.java index 164af6528236..5faee68f7cd0 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/FilteringYieldingAccumulator.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/FilteringYieldingAccumulator.java @@ -20,6 +20,7 @@ package org.apache.druid.java.util.common.guava; import com.google.common.base.Predicate; + /** */ public class FilteringYieldingAccumulator extends YieldingAccumulator diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/MergeIterator.java b/core/src/main/java/org/apache/druid/java/util/common/guava/MergeIterator.java index c12ada1fc4b4..cce334c10f73 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/MergeIterator.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/MergeIterator.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.java.util.common.guava; import com.google.common.collect.Iterators; diff --git a/core/src/main/java/org/apache/druid/java/util/common/io/NativeIO.java b/core/src/main/java/org/apache/druid/java/util/common/io/NativeIO.java index 9eb5d7d68f26..b087e7793fee 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/io/NativeIO.java +++ b/core/src/main/java/org/apache/druid/java/util/common/io/NativeIO.java @@ -82,6 +82,7 @@ public class NativeIO } private static native int posix_fadvise(int fd, long offset, long len, int flag) throws LastErrorException; + private static native int sync_file_range(int fd, long offset, long len, int flags); private NativeIO() {} @@ -100,6 +101,7 @@ private static Field getFieldByReflection(Class cls, String fieldName) return field; } + /** * Get system file descriptor (int) from FileDescriptor object. * @param descriptor - FileDescriptor object to get fd from @@ -207,9 +209,7 @@ public static void chunkedCopy(InputStream src, File dest) throws IOException long offset = 0; long lastOffset = 0; - try ( - final RandomAccessFile raf = new RandomAccessFile(dest, "rw") - ) { + try (final RandomAccessFile raf = new RandomAccessFile(dest, "rw")) { final int fd = getfd(raf.getFD()); for (int numBytes = 0, bytesRead = 0, lastBytes = 0; bytesRead > -1;) { diff --git a/core/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java b/core/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java index 76e3a90e3ff0..560cd63cf5b3 100644 --- a/core/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java +++ b/core/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java @@ -32,6 +32,7 @@ public class BaseHttpEmittingConfig /** ensure the event buffers don't use more than 10% of memory by default */ public static final int DEFAULT_MAX_BATCH_SIZE; public static final int DEFAULT_BATCH_QUEUE_SIZE_LIMIT; + static { Pair batchConfigPair = getDefaultBatchSizeAndLimit(Runtime.getRuntime().maxMemory()); DEFAULT_MAX_BATCH_SIZE = batchConfigPair.lhs; diff --git a/core/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java b/core/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java index 61dc66bf4fdc..8b486cb7503b 100644 --- a/core/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java +++ b/core/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java @@ -739,7 +739,6 @@ private void send(byte[] buffer, int length) throws Exception payloadLength = length; } - request.setHeader(HttpHeaders.Names.CONTENT_TYPE, "application/json"); request.setHeader(HttpHeaders.Names.CONTENT_LENGTH, String.valueOf(payloadLength)); request.setBody(ByteBuffer.wrap(payload, 0, payloadLength)); diff --git a/core/src/main/java/org/apache/druid/java/util/metrics/SysMonitor.java b/core/src/main/java/org/apache/druid/java/util/metrics/SysMonitor.java index 27bc9d2cbf5d..d161bc1ad055 100644 --- a/core/src/main/java/org/apache/druid/java/util/metrics/SysMonitor.java +++ b/core/src/main/java/org/apache/druid/java/util/metrics/SysMonitor.java @@ -148,7 +148,8 @@ public void emit(ServiceEmitter emitter) */ private class SwapStats implements Stats { - private long prevPageIn = 0, prevPageOut = 0; + private long prevPageIn = 0; + private long prevPageOut = 0; private SwapStats() { diff --git a/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java b/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java index e45538a5475c..a1da4f89ec39 100644 --- a/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java +++ b/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java @@ -39,6 +39,7 @@ public interface DataSegmentPusher @Deprecated String getPathForHadoop(String dataSource); + String getPathForHadoop(); /** diff --git a/core/src/test/java/org/apache/druid/collections/CloseableDefaultBlockingPool.java b/core/src/test/java/org/apache/druid/collections/CloseableDefaultBlockingPool.java index bf343834a5dd..5b2c10fa6e15 100644 --- a/core/src/test/java/org/apache/druid/collections/CloseableDefaultBlockingPool.java +++ b/core/src/test/java/org/apache/druid/collections/CloseableDefaultBlockingPool.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.collections; import com.google.common.base.Supplier; diff --git a/core/src/test/java/org/apache/druid/collections/CloseableStupidPool.java b/core/src/test/java/org/apache/druid/collections/CloseableStupidPool.java index 2a38a2c5cab3..c18c5eb3cda6 100644 --- a/core/src/test/java/org/apache/druid/collections/CloseableStupidPool.java +++ b/core/src/test/java/org/apache/druid/collections/CloseableStupidPool.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.collections; import com.google.common.base.Supplier; diff --git a/core/src/test/java/org/apache/druid/common/utils/SocketUtilTest.java b/core/src/test/java/org/apache/druid/common/utils/SocketUtilTest.java index 928d7c87996a..cfb81ad671bf 100644 --- a/core/src/test/java/org/apache/druid/common/utils/SocketUtilTest.java +++ b/core/src/test/java/org/apache/druid/common/utils/SocketUtilTest.java @@ -27,6 +27,7 @@ public class SocketUtilTest { private final int MAX_PORT = 0xffff; + @Test public void testSocketUtil() { diff --git a/core/src/test/java/org/apache/druid/data/input/impl/prefetch/JsonIteratorTest.java b/core/src/test/java/org/apache/druid/data/input/impl/prefetch/JsonIteratorTest.java index 73221f194a20..bff12cc46973 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/prefetch/JsonIteratorTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/prefetch/JsonIteratorTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.impl.prefetch; diff --git a/core/src/test/java/org/apache/druid/java/util/common/StreamUtilsTest.java b/core/src/test/java/org/apache/druid/java/util/common/StreamUtilsTest.java index 86c9235ae8e3..094ca41d7987 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/StreamUtilsTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/StreamUtilsTest.java @@ -39,6 +39,7 @@ public class StreamUtilsTest { @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Test public void testRetryExceptionOnFlush() { diff --git a/core/src/test/java/org/apache/druid/timeline/partition/NoneShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/NoneShardSpecTest.java index 39e3a08688ba..04a519dc5470 100644 --- a/core/src/test/java/org/apache/druid/timeline/partition/NoneShardSpecTest.java +++ b/core/src/test/java/org/apache/druid/timeline/partition/NoneShardSpecTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.timeline.partition; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 0e243c7a2abb..e4dc34614145 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -64,7 +64,7 @@ This page documents all of the configuration properties for each Druid service t * [Autoscaler](#autoscaler) * [MiddleManager & Peons](#middlemanager-and-peons) * [Node Config](#middlemanager-node-config) - * [MiddleManger Configuration](#middlemanager-configuration) + * [MiddleManager Configuration](#middlemanager-configuration) * [Peon Processing](#peon-processing) * [Peon Query Configuration](#peon-query-configuration) * [Caching](#peon-caching) diff --git a/eclipse.importorder b/eclipse.importorder index 46db54ea9fdd..cec7ef04fa6a 100644 --- a/eclipse.importorder +++ b/eclipse.importorder @@ -12,11 +12,7 @@ # 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. - -# Organize Import Order -# Sun Apr 24 10:15:51 PDT 2016 -4=javax -3=java -2=org -1=io -0=com +3=\# +2=java +1=javax +0= diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterModule.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterModule.java index d85812ed3dd0..4468f63d14ca 100644 --- a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterModule.java +++ b/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterModule.java @@ -39,10 +39,11 @@ public class AmbariMetricsEmitterModule implements DruidModule { private static final String EMITTER_TYPE = "ambari-metrics"; + @Override public List getJacksonModules() { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } @Override diff --git a/extensions-contrib/druid-rocketmq/src/main/java/org/apache/druid/firehose/rocketmq/RocketMQDruidModule.java b/extensions-contrib/druid-rocketmq/src/main/java/org/apache/druid/firehose/rocketmq/RocketMQDruidModule.java index 159a3770bf44..0724a890ac40 100644 --- a/extensions-contrib/druid-rocketmq/src/main/java/org/apache/druid/firehose/rocketmq/RocketMQDruidModule.java +++ b/extensions-contrib/druid-rocketmq/src/main/java/org/apache/druid/firehose/rocketmq/RocketMQDruidModule.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.firehose.rocketmq; import com.fasterxml.jackson.databind.Module; diff --git a/extensions-contrib/druid-rocketmq/src/main/java/org/apache/druid/firehose/rocketmq/RocketMQFirehoseFactory.java b/extensions-contrib/druid-rocketmq/src/main/java/org/apache/druid/firehose/rocketmq/RocketMQFirehoseFactory.java index 945461998f78..ce78180145ba 100644 --- a/extensions-contrib/druid-rocketmq/src/main/java/org/apache/druid/firehose/rocketmq/RocketMQFirehoseFactory.java +++ b/extensions-contrib/druid-rocketmq/src/main/java/org/apache/druid/firehose/rocketmq/RocketMQFirehoseFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.firehose.rocketmq; import com.alibaba.rocketmq.client.Validators; diff --git a/extensions-contrib/graphite-emitter/src/main/java/org/apache/druid/emitter/graphite/GraphiteEmitterModule.java b/extensions-contrib/graphite-emitter/src/main/java/org/apache/druid/emitter/graphite/GraphiteEmitterModule.java index 1978b0a58bea..bca977211f5b 100644 --- a/extensions-contrib/graphite-emitter/src/main/java/org/apache/druid/emitter/graphite/GraphiteEmitterModule.java +++ b/extensions-contrib/graphite-emitter/src/main/java/org/apache/druid/emitter/graphite/GraphiteEmitterModule.java @@ -40,6 +40,7 @@ public class GraphiteEmitterModule implements DruidModule { private static final String EMITTER_TYPE = "graphite"; + @Override public List getJacksonModules() { diff --git a/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxExtensionsModule.java b/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxExtensionsModule.java index 8b9ae8f1d097..da0eba6a4b60 100644 --- a/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxExtensionsModule.java +++ b/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxExtensionsModule.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.influx; import com.fasterxml.jackson.databind.Module; diff --git a/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxParseSpec.java b/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxParseSpec.java index db68fb8e4fed..025c3ad08e54 100644 --- a/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxParseSpec.java +++ b/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxParseSpec.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.influx; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxParser.java b/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxParser.java index a090068b1fbd..a14e5e1e1cf8 100644 --- a/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxParser.java +++ b/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxParser.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.influx; import com.google.common.collect.ImmutableList; diff --git a/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/org/apache/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java b/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/org/apache/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java index a68150a0273b..65c501a8e542 100644 --- a/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/org/apache/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java +++ b/extensions-contrib/kafka-eight-simpleConsumer/src/main/java/org/apache/druid/firehose/kafka/KafkaEightSimpleConsumerFirehoseFactory.java @@ -97,7 +97,6 @@ public KafkaEightSimpleConsumerFirehoseFactory( "partitionIdList is null/empty" ); - this.clientId = clientId; Preconditions.checkArgument( clientId != null && !clientId.isEmpty(), diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java index 73b82cd8e606..87e3d0d93cc4 100644 --- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java +++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.materializedview; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcExtensionsModule.java b/extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcExtensionsModule.java index 9efdcd365b10..595bb3b856e7 100644 --- a/extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcExtensionsModule.java +++ b/extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcExtensionsModule.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.orc; import com.fasterxml.jackson.databind.Module; diff --git a/extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java b/extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java index e31bcd88dba2..6d3340ca81a8 100644 --- a/extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java +++ b/extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.orc; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/DruidOrcInputFormatTest.java b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/DruidOrcInputFormatTest.java index 615efdbbd443..57462d2ef204 100644 --- a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/DruidOrcInputFormatTest.java +++ b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/DruidOrcInputFormatTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.orc; import org.apache.druid.data.input.MapBasedInputRow; diff --git a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java index 6aa4715920a6..fd9b9b6ee467 100644 --- a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java +++ b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.orc; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/extensions-contrib/rabbitmq/src/main/java/org/apache/druid/firehose/rabbitmq/RabbitMQDruidModule.java b/extensions-contrib/rabbitmq/src/main/java/org/apache/druid/firehose/rabbitmq/RabbitMQDruidModule.java index 99565b986c2b..0038c5373876 100644 --- a/extensions-contrib/rabbitmq/src/main/java/org/apache/druid/firehose/rabbitmq/RabbitMQDruidModule.java +++ b/extensions-contrib/rabbitmq/src/main/java/org/apache/druid/firehose/rabbitmq/RabbitMQDruidModule.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.firehose.rabbitmq; import com.fasterxml.jackson.databind.Module; diff --git a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java index 783daed4c8c2..152501dc3c4c 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.metadata.storage.sqlserver; import com.google.common.base.Supplier; diff --git a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerMetadataStorageModule.java b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerMetadataStorageModule.java index 5ba63d4b8efb..c4ac67217826 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerMetadataStorageModule.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerMetadataStorageModule.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.metadata.storage.sqlserver; import com.fasterxml.jackson.databind.Module; diff --git a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/CustomStatementRewriterTest.java b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/CustomStatementRewriterTest.java index 3b71ceb93081..7dc3ae300827 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/CustomStatementRewriterTest.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/CustomStatementRewriterTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.metadata.storage.sqlserver; import junit.framework.Assert; diff --git a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java index db7bb0757fb7..d061b99701de 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.metadata.storage.sqlserver; import com.google.common.base.Suppliers; diff --git a/extensions-contrib/statsd-emitter/src/main/java/org/apache/druid/emitter/statsd/DimensionConverter.java b/extensions-contrib/statsd-emitter/src/main/java/org/apache/druid/emitter/statsd/DimensionConverter.java index ae1586a34438..0386472ded59 100644 --- a/extensions-contrib/statsd-emitter/src/main/java/org/apache/druid/emitter/statsd/DimensionConverter.java +++ b/extensions-contrib/statsd-emitter/src/main/java/org/apache/druid/emitter/statsd/DimensionConverter.java @@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.logger.Logger; +import javax.annotation.Nullable; import java.io.File; import java.io.FileInputStream; import java.io.IOException; @@ -45,6 +46,7 @@ public DimensionConverter(ObjectMapper mapper, String dimensionMapPath) metricMap = readMap(mapper, dimensionMapPath); } + @Nullable public StatsDMetric addFilteredUserDims( String service, String metric, diff --git a/extensions-contrib/statsd-emitter/src/main/java/org/apache/druid/emitter/statsd/StatsDEmitter.java b/extensions-contrib/statsd-emitter/src/main/java/org/apache/druid/emitter/statsd/StatsDEmitter.java index 7794ffee2717..a36f5a09b290 100644 --- a/extensions-contrib/statsd-emitter/src/main/java/org/apache/druid/emitter/statsd/StatsDEmitter.java +++ b/extensions-contrib/statsd-emitter/src/main/java/org/apache/druid/emitter/statsd/StatsDEmitter.java @@ -32,9 +32,9 @@ import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import java.util.List; import java.util.Map; import java.util.regex.Pattern; -import java.util.List; /** */ @@ -47,7 +47,7 @@ public class StatsDEmitter implements Emitter private static final Pattern BLANK = Pattern.compile("\\s+"); private static final String[] EMPTY_ARRAY = new String[0]; - static final StatsDEmitter of(StatsDEmitterConfig config, ObjectMapper mapper) + static StatsDEmitter of(StatsDEmitterConfig config, ObjectMapper mapper) { NonBlockingStatsDClient client = new NonBlockingStatsDClient( config.getPrefix(), @@ -163,7 +163,7 @@ public void emit(Event event) } } } else { - log.debug("Metric=[%s] has no StatsD type mapping", statsDMetric); + log.debug("Service=[%s], Metric=[%s] has no StatsD type mapping", service, metric); } } } diff --git a/extensions-contrib/thrift-extensions/src/main/java/org/apache/druid/data/input/thrift/ThriftDeserialization.java b/extensions-contrib/thrift-extensions/src/main/java/org/apache/druid/data/input/thrift/ThriftDeserialization.java index a0c505ce4602..ae5f1f0806b8 100644 --- a/extensions-contrib/thrift-extensions/src/main/java/org/apache/druid/data/input/thrift/ThriftDeserialization.java +++ b/extensions-contrib/thrift-extensions/src/main/java/org/apache/druid/data/input/thrift/ThriftDeserialization.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.thrift; import com.google.common.base.Preconditions; diff --git a/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampMinMaxModule.java b/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampMinMaxModule.java index 9d19299ab4b4..edb6f06a0503 100644 --- a/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampMinMaxModule.java +++ b/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampMinMaxModule.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.aggregation; import com.fasterxml.jackson.databind.Module; diff --git a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapTypeMapVirtualColumnDimensionSelector.java b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapTypeMapVirtualColumnDimensionSelector.java index 45d20cb8d749..4fc361f33253 100644 --- a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapTypeMapVirtualColumnDimensionSelector.java +++ b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapTypeMapVirtualColumnDimensionSelector.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment; import com.google.common.base.Predicate; diff --git a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapVirtualColumnDimensionSelector.java b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapVirtualColumnDimensionSelector.java index 2c56df438eba..3117cf849143 100644 --- a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapVirtualColumnDimensionSelector.java +++ b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapVirtualColumnDimensionSelector.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment; import org.apache.druid.common.config.NullHandling; diff --git a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapVirtualColumnValueSelector.java b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapVirtualColumnValueSelector.java index ba25bb072ea9..5c154d310d0d 100644 --- a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapVirtualColumnValueSelector.java +++ b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapVirtualColumnValueSelector.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment; import org.apache.druid.common.config.NullHandling; diff --git a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/StringTypeMapVirtualColumnDimensionSelector.java b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/StringTypeMapVirtualColumnDimensionSelector.java index 9b15a29718e8..37a005203692 100644 --- a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/StringTypeMapVirtualColumnDimensionSelector.java +++ b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/StringTypeMapVirtualColumnDimensionSelector.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment; import com.google.common.base.Preconditions; diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java index 43d6809aa291..48ee2ef1f69b 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment; import com.google.common.collect.ImmutableList; diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java index 7736ee246b91..ffc3706f95e5 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment; import com.google.common.io.CharSource; diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java index e22368159212..794589447c67 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTopNTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment; import com.google.common.collect.ImmutableList; diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/AvroHadoopInputRowParser.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/AvroHadoopInputRowParser.java index 83b27349b18b..39ce48ec2b5e 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/AvroHadoopInputRowParser.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/AvroHadoopInputRowParser.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/AvroStreamInputRowParser.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/AvroStreamInputRowParser.java index 749970f6505c..f375c63b8b23 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/AvroStreamInputRowParser.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/AvroStreamInputRowParser.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroBytesDecoder.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroBytesDecoder.java index cbb6ecdca20e..62448a1069c8 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroBytesDecoder.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroBytesDecoder.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.avro; import com.fasterxml.jackson.annotation.JsonSubTypes; diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroExtensionsModule.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroExtensionsModule.java index ba14cf89059d..4ffe8db99b2f 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroExtensionsModule.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroExtensionsModule.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.avro; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroValueInputFormat.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroValueInputFormat.java index e74c7f2e6601..97dcd876d2eb 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroValueInputFormat.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroValueInputFormat.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.avro; import org.apache.avro.Schema; diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroValueRecordReader.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroValueRecordReader.java index 2b5090370a1a..68a955fe66b2 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroValueRecordReader.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroValueRecordReader.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.avro; import org.apache.avro.Schema; diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoder.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoder.java index 0f8e4ce272d5..5b9d38695561 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoder.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/SchemaRegistryBasedAvroBytesDecoder.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.avro; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/SchemaRepoBasedAvroBytesDecoder.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/SchemaRepoBasedAvroBytesDecoder.java index 7f461a0ab319..e7de115c7b44 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/SchemaRepoBasedAvroBytesDecoder.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/SchemaRepoBasedAvroBytesDecoder.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.avro; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/schemarepo/Avro1124RESTRepositoryClientWrapper.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/schemarepo/Avro1124RESTRepositoryClientWrapper.java index d3f32f277a5c..0ba557932d61 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/schemarepo/Avro1124RESTRepositoryClientWrapper.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/schemarepo/Avro1124RESTRepositoryClientWrapper.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.schemarepo; import com.fasterxml.jackson.annotation.JsonIgnore; diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/schemarepo/Avro1124SubjectAndIdConverter.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/schemarepo/Avro1124SubjectAndIdConverter.java index a18e6690fce1..82f89f6a1664 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/schemarepo/Avro1124SubjectAndIdConverter.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/schemarepo/Avro1124SubjectAndIdConverter.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.schemarepo; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/schemarepo/SubjectAndIdConverter.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/schemarepo/SubjectAndIdConverter.java index 9e0751733c8f..ff17010126e7 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/schemarepo/SubjectAndIdConverter.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/schemarepo/SubjectAndIdConverter.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.schemarepo; import com.fasterxml.jackson.annotation.JsonSubTypes; diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroHadoopInputRowParserTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroHadoopInputRowParserTest.java index 22f0bd78da0b..dc25e8a2930b 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroHadoopInputRowParserTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroHadoopInputRowParserTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input; import com.fasterxml.jackson.databind.Module; diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java index 4da8906c9eeb..184c8b247023 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input; import com.fasterxml.jackson.databind.DeserializationFeature; diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregator.java index 5ce5be4cb2eb..f4a4e1ead05a 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregator.java @@ -21,7 +21,6 @@ import com.yahoo.sketches.hll.HllSketch; import com.yahoo.sketches.hll.TgtHllType; - import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.segment.ColumnValueSelector; diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java index 587a61759651..edf019fe2786 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java @@ -19,19 +19,18 @@ package org.apache.druid.query.aggregation.datasketches.hll; -import javax.annotation.Nullable; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.yahoo.sketches.hll.HllSketch; import com.yahoo.sketches.hll.TgtHllType; - import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import javax.annotation.Nullable; + /** * This aggregator factory is for building sketches from raw data. * The input column can contain identifiers of type string, char[], byte[] or any numeric type. diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildBufferAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildBufferAggregator.java index bd2d047cbe90..a170502eabf6 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildBufferAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildBufferAggregator.java @@ -19,21 +19,20 @@ package org.apache.druid.query.aggregation.datasketches.hll; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.IdentityHashMap; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReadWriteLock; - import com.google.common.util.concurrent.Striped; import com.yahoo.memory.WritableMemory; import com.yahoo.sketches.hll.HllSketch; import com.yahoo.sketches.hll.TgtHllType; - -import org.apache.druid.query.aggregation.BufferAggregator; -import org.apache.druid.segment.ColumnValueSelector; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.segment.ColumnValueSelector; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.IdentityHashMap; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; /** * This aggregator builds sketches from raw data. diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchJsonSerializer.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchJsonSerializer.java index a1d753157b23..369898de4ec5 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchJsonSerializer.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchJsonSerializer.java @@ -19,13 +19,13 @@ package org.apache.druid.query.aggregation.datasketches.hll; -import java.io.IOException; - import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.JsonSerializer; import com.fasterxml.jackson.databind.SerializerProvider; import com.yahoo.sketches.hll.HllSketch; +import java.io.IOException; + public class HllSketchJsonSerializer extends JsonSerializer { diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregator.java index bdb5a545cec8..44f1157120ab 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregator.java @@ -19,10 +19,9 @@ package org.apache.druid.query.aggregation.datasketches.hll; -import com.yahoo.sketches.hll.Union; import com.yahoo.sketches.hll.HllSketch; import com.yahoo.sketches.hll.TgtHllType; - +import com.yahoo.sketches.hll.Union; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.segment.ColumnValueSelector; diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java index 05da8d611f41..f1a0b1a89621 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java @@ -19,20 +19,19 @@ package org.apache.druid.query.aggregation.datasketches.hll; -import javax.annotation.Nullable; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.yahoo.sketches.hll.HllSketch; import com.yahoo.sketches.hll.TgtHllType; import com.yahoo.sketches.hll.Union; - import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import javax.annotation.Nullable; + /** * This aggregator factory is for merging existing sketches. * The input column must contain {@link HllSketch} diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeBufferAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeBufferAggregator.java index 8e72d2369117..65b273b7cadc 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeBufferAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeBufferAggregator.java @@ -19,20 +19,19 @@ package org.apache.druid.query.aggregation.datasketches.hll; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReadWriteLock; - import com.google.common.util.concurrent.Striped; import com.yahoo.memory.WritableMemory; import com.yahoo.sketches.hll.HllSketch; import com.yahoo.sketches.hll.TgtHllType; import com.yahoo.sketches.hll.Union; - import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.segment.ColumnValueSelector; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; + /** * This aggregator merges existing sketches. * The input column must contain {@link HllSketch} diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchModule.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchModule.java index 548c4d5ee5db..983b6e150f93 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchModule.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchModule.java @@ -19,18 +19,17 @@ package org.apache.druid.query.aggregation.datasketches.hll; -import java.util.Collections; -import java.util.List; - import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; import com.yahoo.sketches.hll.HllSketch; - import org.apache.druid.initialization.DruidModule; import org.apache.druid.segment.serde.ComplexMetrics; +import java.util.Collections; +import java.util.List; + /** * This module is to support count-distinct operations using {@link HllSketch}. * See HyperLogLog Sketch documentation diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategy.java index 6ae54f9e4196..a3a140c01901 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategy.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchObjectStrategy.java @@ -19,14 +19,13 @@ package org.apache.druid.query.aggregation.datasketches.hll; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; - import com.yahoo.memory.Memory; import com.yahoo.sketches.hll.HllSketch; - import org.apache.druid.segment.data.ObjectStrategy; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + public class HllSketchObjectStrategy implements ObjectStrategy { diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimateWithBoundsPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimateWithBoundsPostAggregator.java index c57341ab90c0..f139456bd13f 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimateWithBoundsPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimateWithBoundsPostAggregator.java @@ -22,20 +22,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.yahoo.sketches.hll.HllSketch; - import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; +import javax.annotation.Nullable; import java.util.Comparator; import java.util.Map; import java.util.Objects; import java.util.Set; -import javax.annotation.Nullable; - /** * Returns a distinct count estimate and error bounds from a given {@link HllSketch}. * The result will be three double values: estimate, lower bound and upper bound. diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToStringPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToStringPostAggregator.java index 1da6780ef0c7..19a16149ec39 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToStringPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToStringPostAggregator.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.yahoo.sketches.hll.HllSketch; - import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchObjectStrategy.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchObjectStrategy.java index 618799d57965..38d962686c5e 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchObjectStrategy.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchObjectStrategy.java @@ -22,7 +22,6 @@ import com.yahoo.memory.Memory; import com.yahoo.sketches.quantiles.DoublesSketch; import it.unimi.dsi.fastutil.bytes.ByteArrays; - import org.apache.druid.segment.data.ObjectStrategy; import java.nio.ByteBuffer; diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java index 5bf050dbd039..b52c74df4ba2 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java @@ -33,6 +33,7 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; + public class SketchEstimatePostAggregator implements PostAggregator { diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBounds.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBounds.java index 67f8302ec501..2eda00d68e5c 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBounds.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBounds.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.aggregation.datasketches.theta; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/GenerateTestData.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/GenerateTestData.java index 489ce0ed6e24..75882ff08002 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/GenerateTestData.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/GenerateTestData.java @@ -19,6 +19,9 @@ package org.apache.druid.query.aggregation.datasketches.hll; +import com.yahoo.sketches.hll.HllSketch; +import org.apache.commons.codec.binary.Base64; + import java.io.BufferedWriter; import java.nio.charset.StandardCharsets; import java.nio.file.FileSystems; @@ -27,11 +30,9 @@ import java.util.Random; import java.util.concurrent.ThreadLocalRandom; -import org.apache.commons.codec.binary.Base64; - -import com.yahoo.sketches.hll.HllSketch; - -// This is used for generating test data for HllSketchAggregatorTest +/** + * This is used for generating test data for {@link HllSketchAggregatorTest} + */ class GenerateTestData { diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBoundsTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBoundsTest.java index ad34fbf7c6f4..7be06c4ebe37 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBoundsTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBoundsTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.aggregation.datasketches.theta; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/guice/BloomFilterSerializersModule.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/guice/BloomFilterSerializersModule.java index af4520250e39..5162abb1d059 100644 --- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/guice/BloomFilterSerializersModule.java +++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/guice/BloomFilterSerializersModule.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.guice; import com.fasterxml.jackson.core.JsonGenerator; diff --git a/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/KerberosEscalator.java b/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/KerberosEscalator.java index 87b63aed4c82..b4797e674380 100644 --- a/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/KerberosEscalator.java +++ b/extensions-core/druid-kerberos/src/main/java/org/apache/druid/security/kerberos/KerberosEscalator.java @@ -46,8 +46,8 @@ public KerberosEscalator( this.authorizerName = authorizerName; this.internalClientPrincipal = internalClientPrincipal; this.internalClientKeytab = internalClientKeytab; - } + @Override public HttpClient createEscalatedClient(HttpClient baseClient) { diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsFileTimestampVersionFinder.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsFileTimestampVersionFinder.java index d89213a409a1..c8560c65e682 100644 --- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsFileTimestampVersionFinder.java +++ b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsFileTimestampVersionFinder.java @@ -34,10 +34,6 @@ import java.net.URI; import java.util.regex.Pattern; -/** - * This is implemented explicitly for URIExtractionNamespaceFunctionFactory - * If you have a use case for this interface beyond URIExtractionNamespaceFunctionFactory please bring it up in the dev list. - */ public class HdfsFileTimestampVersionFinder extends HdfsDataSegmentPuller implements SearchableVersionedDataFinder { @Inject diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/tasklog/HdfsTaskLogs.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/tasklog/HdfsTaskLogs.java index 2f1df6d1fabe..4e2b68fc67ec 100644 --- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/tasklog/HdfsTaskLogs.java +++ b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/tasklog/HdfsTaskLogs.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.storage.hdfs.tasklog; import com.google.common.base.Optional; diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java index 820ce2637c8a..01a4e0b65514 100644 --- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java +++ b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.storage.hdfs.tasklog; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/hadoop/fs/HadoopFsWrapper.java b/extensions-core/hdfs-storage/src/main/java/org/apache/hadoop/fs/HadoopFsWrapper.java index 334a212f1879..a408032aa3e3 100644 --- a/extensions-core/hdfs-storage/src/main/java/org/apache/hadoop/fs/HadoopFsWrapper.java +++ b/extensions-core/hdfs-storage/src/main/java/org/apache/hadoop/fs/HadoopFsWrapper.java @@ -17,9 +17,7 @@ * under the License. */ -//CHECKSTYLE.OFF: PackageName -package org.apache.hadoop.fs; -//CHECKSTYLE.ON: PackageName +package /*CHECKSTYLE.OFF: PackageName*/org.apache.hadoop.fs/*CHECKSTYLE.ON: PackageName*/; import com.google.common.base.Throwables; import org.apache.druid.java.util.common.logger.Logger; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index f440d1befc81..62d595c1d3e9 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.indexing.kafka; import com.fasterxml.jackson.core.type.TypeReference; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index e587917c36ae..c496f5fb5c50 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.indexing.kafka; import com.fasterxml.jackson.core.JsonProcessingException; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index e92ad874ec1e..328071d7ef12 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -377,6 +377,12 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) super.resetInternal(dataSourceMetadata); } + @Override + protected String baseTaskName() + { + return "index_kafka"; + } + @Override @VisibleForTesting protected void moveTaskGroupToPendingCompletion(int taskGroupId) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index eb0beac9755e..d49af6bb9391 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -72,8 +72,6 @@ public KinesisIndexTaskRunner( rowIngestionMetersFactory, false ); - - } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index e85975e42afc..0d21097fa9a4 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -316,6 +316,12 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) super.resetInternal(dataSourceMetadata); } + @Override + protected String baseTaskName() + { + return "index_kinesis"; + } + @Override @VisibleForTesting protected void moveTaskGroupToPendingCompletion(int taskGroupId) diff --git a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/UriCacheGenerator.java b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/UriCacheGenerator.java index eaa8c052e051..b7213d63af0a 100644 --- a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/UriCacheGenerator.java +++ b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/server/lookup/namespace/UriCacheGenerator.java @@ -113,8 +113,6 @@ public CacheScheduler.VersionedCache generateCache( uri = extractionNamespace.getUri(); } - final String uriPath = uri.getPath(); - return RetryUtils.retry( () -> { final String version = puller.getVersion(uri); diff --git a/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/PollingLookupTest.java b/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/PollingLookupTest.java index c9ab212745fa..ebccdbd5ee87 100644 --- a/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/PollingLookupTest.java +++ b/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/PollingLookupTest.java @@ -65,6 +65,7 @@ public class PollingLookupTest private static class MockDataFetcher implements DataFetcher { private int callNumber = 0; + @Override public Iterable fetchAll() { diff --git a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/firehose/sql/MySQLFirehoseDatabaseConnector.java b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/firehose/sql/MySQLFirehoseDatabaseConnector.java index 581f7ffdf76c..2c08a63f8d4c 100644 --- a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/firehose/sql/MySQLFirehoseDatabaseConnector.java +++ b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/firehose/sql/MySQLFirehoseDatabaseConnector.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.firehose.sql; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/DruidParquetAvroInputFormat.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/DruidParquetAvroInputFormat.java index 783ec983e0a3..109f315c4bea 100755 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/DruidParquetAvroInputFormat.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/DruidParquetAvroInputFormat.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.parquet.avro; import org.apache.avro.generic.GenericRecord; diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java index 330d9e11f375..a297b94e5a1f 100755 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/avro/ParquetAvroHadoopInputRowParser.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.data.input.parquet.avro; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/parquet/avro/DruidParquetAvroReadSupport.java b/extensions-core/parquet-extensions/src/main/java/org/apache/parquet/avro/DruidParquetAvroReadSupport.java index e8ffde7800bf..9493420448dc 100755 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/parquet/avro/DruidParquetAvroReadSupport.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/parquet/avro/DruidParquetAvroReadSupport.java @@ -17,9 +17,7 @@ * under the License. */ -//CHECKSTYLE.OFF: PackageName -package org.apache.parquet.avro; -//CHECKSTYLE.ON: PackageName +package /*CHECKSTYLE.OFF: PackageName*/org.apache.parquet.avro/*CHECKSTYLE.ON: PackageName*/; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -42,7 +40,7 @@ import java.util.Set; /** - * This class must in package org.apache.parquet.avro to access the AvroRecordMaterializer constructor + * This class must be in the package org.apache.parquet.avro to access {@link AvroRecordMaterializer}'s constructor */ public class DruidParquetAvroReadSupport extends AvroReadSupport { diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/CustomServerSideEncryption.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/CustomServerSideEncryption.java index d00c138b432f..647ba308f2f8 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/CustomServerSideEncryption.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/CustomServerSideEncryption.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.storage.s3; import com.amazonaws.services.s3.model.CopyObjectRequest; diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/KmsServerSideEncryption.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/KmsServerSideEncryption.java index cfd2eaf3d7b8..fb68f1d9f682 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/KmsServerSideEncryption.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/KmsServerSideEncryption.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.storage.s3; import com.amazonaws.services.s3.model.CopyObjectRequest; diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/NoopServerSideEncryption.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/NoopServerSideEncryption.java index 0a30d8d54aa4..b74ff32008fc 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/NoopServerSideEncryption.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/NoopServerSideEncryption.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.storage.s3; public class NoopServerSideEncryption implements ServerSideEncryption diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3SSECustomConfig.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3SSECustomConfig.java index 00c95488f081..4a970ffe2eea 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3SSECustomConfig.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3SSECustomConfig.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.storage.s3; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3SSEKmsConfig.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3SSEKmsConfig.java index 91d1b20c6aad..6a1b35cc63f5 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3SSEKmsConfig.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3SSEKmsConfig.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.storage.s3; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3ServerSideEncryption.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3ServerSideEncryption.java index 6ca6c5e66f9a..348b3d2aeabc 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3ServerSideEncryption.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3ServerSideEncryption.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.storage.s3; import com.amazonaws.services.s3.model.CopyObjectRequest; diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageConfig.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageConfig.java index 9881551703af..cfae0eb084b7 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageConfig.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageConfig.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.storage.s3; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TimestampVersionedDataFinder.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TimestampVersionedDataFinder.java index b26e4dcd357b..3b7598cbe223 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TimestampVersionedDataFinder.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TimestampVersionedDataFinder.java @@ -31,10 +31,6 @@ import java.util.Iterator; import java.util.regex.Pattern; -/** - * This is implemented explicitly for URIExtractionNamespaceFunctionFactory - * If you have a use case for this interface beyond URIExtractionNamespaceFunctionFactory please bring it up in the dev list. - */ public class S3TimestampVersionedDataFinder extends S3DataSegmentPuller implements SearchableVersionedDataFinder { private static final int MAX_LISTING_KEYS = 1000; diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3.java index 577c99dfc34e..f7e8fa61aa0e 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.storage.s3; import com.amazonaws.services.s3.AmazonS3; diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryption.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryption.java index b9a8ae8c7446..3e0402d1b60e 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryption.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryption.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.storage.s3; import com.amazonaws.services.s3.model.CopyObjectRequest; diff --git a/hll/src/test/java/org/apache/druid/hll/HyperLogLogSerdeBenchmarkTest.java b/hll/src/test/java/org/apache/druid/hll/HyperLogLogSerdeBenchmarkTest.java index bcfa936e0f8d..6658cb33b74e 100644 --- a/hll/src/test/java/org/apache/druid/hll/HyperLogLogSerdeBenchmarkTest.java +++ b/hll/src/test/java/org/apache/druid/hll/HyperLogLogSerdeBenchmarkTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.hll; import com.carrotsearch.junitbenchmarks.AbstractBenchmark; @@ -47,6 +48,7 @@ public class HyperLogLogSerdeBenchmarkTest extends AbstractBenchmark { private final HyperLogLogCollector collector; private final long NUM_HASHES; + public HyperLogLogSerdeBenchmarkTest(final HyperLogLogCollector collector, Long num_hashes) { this.collector = collector; diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java index 405e5d60fb14..cd110298c739 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java @@ -151,7 +151,6 @@ public HadoopTuningConfig( this.useExplicitVersion = useExplicitVersion; this.allowedHadoopPrefix = allowedHadoopPrefix == null ? ImmutableList.of() : allowedHadoopPrefix; - this.ignoreInvalidRows = ignoreInvalidRows == null ? false : ignoreInvalidRows; if (maxParseExceptions != null) { this.maxParseExceptions = maxParseExceptions; diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java index 10911e70212b..9161a1c55f7d 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -46,12 +46,12 @@ public class HadoopDruidIndexerConfigTest { private static final ObjectMapper jsonMapper; + static { jsonMapper = new DefaultObjectMapper(); jsonMapper.setInjectableValues(new InjectableValues.Std().addValue(ObjectMapper.class, jsonMapper)); } - @Test public void testHashedBucketSelection() { diff --git a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceModuleHelper.java b/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceModuleHelper.java index be3666a3cc98..5135c97cc93a 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceModuleHelper.java +++ b/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceModuleHelper.java @@ -29,6 +29,7 @@ public class IndexingServiceModuleHelper { public static final String INDEXER_RUNNER_PROPERTY_PREFIX = "druid.indexer.runner"; + public static void configureTaskRunnerConfigs(Binder binder) { JsonConfigProvider.bind(binder, INDEXER_RUNNER_PROPERTY_PREFIX, ForkingTaskRunnerConfig.class); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAuditLogConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAuditLogConfig.java index a4470999bff4..cfabcf18a6e9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAuditLogConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAuditLogConfig.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.indexing.common.actions; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorIngestionSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorIngestionSpec.java index 9e64e1a9839c..c244a0c509d6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorIngestionSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorIngestionSpec.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.indexing.common.index; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index a5c8e7d5032f..6f2052dbef72 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.indexing.common.index; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexingProgress.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexingProgress.java index edcc59507977..336bbb04bcae 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexingProgress.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexingProgress.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.indexing.common.task.batch.parallel; /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingProgress.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingProgress.java index aa27767c054b..28fd7c851387 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingProgress.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingProgress.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskHistory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskHistory.java index 48161e9cc5cd..f7b1bf082feb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskHistory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskHistory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.indexing.common.task.batch.parallel; import com.google.common.base.Preconditions; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index 006286ac8417..51b38d026095 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -221,9 +221,9 @@ public List> getActiveTaskInfo(@Nullable String dataS } @Override - public List> getRecentlyFinishedTaskInfo( + public List> getRecentlyCreatedAlreadyFinishedTaskInfo( @Nullable Integer maxTaskStatuses, - @Nullable Duration duration, + @Nullable Duration durationBeforeNow, @Nullable String datasource ) { @@ -240,18 +240,18 @@ public int compare(TaskStuff a, TaskStuff b) }.reverse(); return maxTaskStatuses == null ? - getRecentlyFinishedTaskInfoSince( - DateTimes.nowUtc().minus(duration == null ? config.getRecentlyFinishedThreshold() : duration), + getRecentlyCreatedAlreadyFinishedTaskInfoSince( + DateTimes.nowUtc().minus(durationBeforeNow == null ? config.getRecentlyFinishedThreshold() : durationBeforeNow), createdDateDesc ) : - getNRecentlyFinishedTaskInfo(maxTaskStatuses, createdDateDesc); + getNRecentlyCreatedAlreadyFinishedTaskInfo(maxTaskStatuses, createdDateDesc); } finally { giant.unlock(); } } - private List> getRecentlyFinishedTaskInfoSince( + private List> getRecentlyCreatedAlreadyFinishedTaskInfoSince( DateTime start, Ordering createdDateDesc ) @@ -262,7 +262,7 @@ private List> getRecentlyFinishedTaskInfoSince( List list = createdDateDesc .sortedCopy(tasks.values()) .stream() - .filter(taskStuff -> taskStuff.getStatus().isComplete()) + .filter(taskStuff -> taskStuff.getStatus().isComplete() && taskStuff.createdDate.isAfter(start)) .collect(Collectors.toList()); final ImmutableList.Builder> listBuilder = ImmutableList.builder(); for (final TaskStuff taskStuff : list) { @@ -283,7 +283,7 @@ private List> getRecentlyFinishedTaskInfoSince( } } - private List> getNRecentlyFinishedTaskInfo(int n, Ordering createdDateDesc) + private List> getNRecentlyCreatedAlreadyFinishedTaskInfo(int n, Ordering createdDateDesc) { giant.lock(); @@ -291,6 +291,7 @@ private List> getNRecentlyFinishedTaskInfo(int n, Ord List list = createdDateDesc .sortedCopy(tasks.values()) .stream() + .filter(taskStuff -> taskStuff.getStatus().isComplete()) .limit(n) .collect(Collectors.toList()); final ImmutableList.Builder> listBuilder = ImmutableList.builder(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 808fdb797295..f0d9d37167f9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -208,15 +208,15 @@ public List> getActiveTaskInfo(@Nullable String dataS } @Override - public List> getRecentlyFinishedTaskInfo( + public List> getRecentlyCreatedAlreadyFinishedTaskInfo( @Nullable Integer maxTaskStatuses, - @Nullable Duration duration, + @Nullable Duration durationBeforeNow, @Nullable String datasource ) { return ImmutableList.copyOf( handler.getCompletedTaskInfo( - DateTimes.nowUtc().minus(duration == null ? config.getRecentlyFinishedThreshold() : duration), + DateTimes.nowUtc().minus(durationBeforeNow == null ? config.getRecentlyFinishedThreshold() : durationBeforeNow), maxTaskStatuses, datasource ) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java index b2f55f0c9d43..1edd52c2c385 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java @@ -141,14 +141,14 @@ public interface TaskStorage * return nothing. * * @param maxTaskStatuses maxTaskStatuses - * @param duration duration + * @param durationBeforeNow duration * @param datasource datasource * * @return list of {@link TaskInfo} */ - List> getRecentlyFinishedTaskInfo( + List> getRecentlyCreatedAlreadyFinishedTaskInfo( @Nullable Integer maxTaskStatuses, - @Nullable Duration duration, + @Nullable Duration durationBeforeNow, @Nullable String datasource ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java index fd61752a7f2a..6a12b40bc11b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -58,13 +58,13 @@ public List> getActiveTaskInfo(@Nullable String dataS return storage.getActiveTaskInfo(dataSource); } - public List> getRecentlyCompletedTaskInfo( + public List> getCompletedTaskInfoByCreatedTimeDuration( @Nullable Integer maxTaskStatuses, @Nullable Duration duration, @Nullable String dataSource ) { - return storage.getRecentlyFinishedTaskInfo(maxTaskStatuses, duration, dataSource); + return storage.getRecentlyCreatedAlreadyFinishedTaskInfo(maxTaskStatuses, duration, dataSource); } public Optional getTask(final String taskid) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/WorkerTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/WorkerTaskRunner.java index 5948cbac0b7a..c0620f859408 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/WorkerTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/WorkerTaskRunner.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.indexing.overlord; import com.google.common.base.Predicate; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index f0748f9a3afe..92f01ac5a6f0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -627,7 +627,7 @@ public Response getCompleteTasks( public Response getTasks( @QueryParam("state") final String state, @QueryParam("datasource") final String dataSource, - @QueryParam("interval") final String interval, + @PathParam("createdTimeInterval") final String createdTimeInterval, @QueryParam("max") final Integer maxCompletedTasks, @QueryParam("type") final String type, @Context final HttpServletRequest req @@ -692,13 +692,13 @@ public Response getTasks( //checking for complete tasks first to avoid querying active tasks if user only wants complete tasks if (state == null || "complete".equals(StringUtils.toLowerCase(state))) { - Duration duration = null; - if (interval != null) { - final Interval theInterval = Intervals.of(interval.replace('_', '/')); - duration = theInterval.toDuration(); + Duration createdTimeDuration = null; + if (createdTimeInterval != null) { + final Interval theInterval = Intervals.of(StringUtils.replace(createdTimeInterval, "_", "/")); + createdTimeDuration = theInterval.toDuration(); } final List> taskInfoList = - taskStorageQueryAdapter.getRecentlyCompletedTaskInfo(maxCompletedTasks, duration, dataSource); + taskStorageQueryAdapter.getCompletedTaskInfoByCreatedTimeDuration(maxCompletedTasks, createdTimeDuration, dataSource); final List completedTasks = taskInfoList.stream() .map(completeTaskTransformFunc::apply) .collect(Collectors.toList()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index ddb22f507038..75e8f747c60c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.indexing.seekablestream; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 8267ae440486..03d8a81d907e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -565,7 +565,6 @@ public Optional getTaskStatus(String id) + IndexTaskClient.MAX_RETRY_WAIT_SECONDS) ); - int chatThreads = (this.tuningConfig.getChatThreads() != null ? this.tuningConfig.getChatThreads() : Math.min(10, this.ioConfig.getTaskCount() * this.ioConfig.getReplicas())); @@ -1677,6 +1676,7 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) } } + @VisibleForTesting protected String generateSequenceName( Map startPartitions, Optional minimumMessageTime, @@ -1709,9 +1709,11 @@ protected String generateSequenceName( + maxMsgTimeStr) .substring(0, 15); - return Joiner.on("_").join("index_seekable_streaming", dataSource, hashCode); + return Joiner.on("_").join(baseTaskName(), dataSource, hashCode); } + protected abstract String baseTaskName(); + private void updatePartitionDataFromStream() { Set partitionIds; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index 790e00f08460..d598b1f897b2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -695,11 +695,11 @@ public boolean isRevoked() } private static String TASK_NAME = "myModuleIsntLoadedTask"; + private static class TheModule extends SimpleModule { public TheModule() { - registerSubtypes(new NamedType(MyModuleIsntLoadedTask.class, TASK_NAME)); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java index 7964c764069c..e3885f755967 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java @@ -233,7 +233,7 @@ public void testSecuredGetCompleteTasks() new MockTaskRunnerWorkItem(tasksIds.get(1), null), new MockTaskRunnerWorkItem(tasksIds.get(2), null))); - EasyMock.expect(taskStorageQueryAdapter.getRecentlyCompletedTaskInfo(null, null, null)).andStubReturn( + EasyMock.expect(taskStorageQueryAdapter.getCompletedTaskInfoByCreatedTimeDuration(null, null, null)).andStubReturn( ImmutableList.of( new TaskInfo( "id_1", @@ -259,7 +259,7 @@ public void testSecuredGetCompleteTasks() ) ); EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req); - Assert.assertTrue(taskStorageQueryAdapter.getRecentlyCompletedTaskInfo(null, null, null).size() == 3); + Assert.assertTrue(taskStorageQueryAdapter.getCompletedTaskInfoByCreatedTimeDuration(null, null, null).size() == 3); Assert.assertTrue(taskRunner.getRunningTasks().size() == 3); List responseObjects = (List) overlordResource .getCompleteTasks(null, req).getEntity(); @@ -313,7 +313,7 @@ public void testGetTasks() { expectAuthorizationTokenCheck(); //completed tasks - EasyMock.expect(taskStorageQueryAdapter.getRecentlyCompletedTaskInfo(null, null, null)).andStubReturn( + EasyMock.expect(taskStorageQueryAdapter.getCompletedTaskInfoByCreatedTimeDuration(null, null, null)).andStubReturn( ImmutableList.of( new TaskInfo( "id_5", @@ -403,7 +403,7 @@ public void testGetTasksFilterDataSource() { expectAuthorizationTokenCheck(); //completed tasks - EasyMock.expect(taskStorageQueryAdapter.getRecentlyCompletedTaskInfo(null, null, "allow")).andStubReturn( + EasyMock.expect(taskStorageQueryAdapter.getCompletedTaskInfoByCreatedTimeDuration(null, null, "allow")).andStubReturn( ImmutableList.of( new TaskInfo( "id_5", @@ -667,7 +667,7 @@ public void testGetTasksFilterPendingState() public void testGetTasksFilterCompleteState() { expectAuthorizationTokenCheck(); - EasyMock.expect(taskStorageQueryAdapter.getRecentlyCompletedTaskInfo(null, null, null)).andStubReturn( + EasyMock.expect(taskStorageQueryAdapter.getCompletedTaskInfoByCreatedTimeDuration(null, null, null)).andStubReturn( ImmutableList.of( new TaskInfo( "id_1", @@ -707,7 +707,7 @@ public void testGetTasksFilterCompleteStateWithInterval() expectAuthorizationTokenCheck(); List tasksIds = ImmutableList.of("id_1", "id_2", "id_3"); Duration duration = new Period("PT86400S").toStandardDuration(); - EasyMock.expect(taskStorageQueryAdapter.getRecentlyCompletedTaskInfo(null, duration, null)).andStubReturn( + EasyMock.expect(taskStorageQueryAdapter.getCompletedTaskInfoByCreatedTimeDuration(null, duration, null)).andStubReturn( ImmutableList.of( new TaskInfo( "id_1", @@ -747,7 +747,7 @@ public void testGetTasksFilterCompleteStateWithInterval() public void testGetNullCompleteTask() { expectAuthorizationTokenCheck(); - EasyMock.expect(taskStorageQueryAdapter.getRecentlyCompletedTaskInfo(null, null, null)).andStubReturn( + EasyMock.expect(taskStorageQueryAdapter.getCompletedTaskInfoByCreatedTimeDuration(null, null, null)).andStubReturn( ImmutableList.of( new TaskInfo( "id_1", diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java index 7ae3b994aced..2bdddb87cef0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java @@ -383,6 +383,7 @@ public TestSupervisorSpec(String id, Supervisor supervisor, boolean suspended, S this.suspended = suspended; this.suspendedSupervisor = suspendedSupervisor; } + @Override public SupervisorSpec createSuspendedSpec() { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/guice/ITTLSCertificateCheckerModule.java b/integration-tests/src/main/java/org/apache/druid/testing/guice/ITTLSCertificateCheckerModule.java index f5c24a16da4f..6166359693a1 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/guice/ITTLSCertificateCheckerModule.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/guice/ITTLSCertificateCheckerModule.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.Module; import com.google.inject.Binder; - import com.google.inject.name.Names; import org.apache.druid.initialization.DruidModule; import org.apache.druid.server.security.TLSCertificateChecker; diff --git a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java index eb3a9bc15e6e..ed53e3a512dc 100644 --- a/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java +++ b/integration-tests/src/main/java/org/testng/DruidTestRunnerFactory.java @@ -17,10 +17,7 @@ * under the License. */ - -//CHECKSTYLE.OFF: PackageName -package org.testng; -//CHECKSTYLE.ON: PackageName +package /*CHECKSTYLE.OFF: PackageName*/org.testng/*CHECKSTYLE.ON: PackageName*/; import com.google.common.base.Throwables; import com.google.inject.Injector; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITRealtimeIndexTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITRealtimeIndexTaskTest.java index 9bf9de219ae6..176ea112590b 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITRealtimeIndexTaskTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITRealtimeIndexTaskTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.tests.indexer; import com.google.common.base.Throwables; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppenderatorDriverRealtimeIndexTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppenderatorDriverRealtimeIndexTaskTest.java index 54d28d9d91fe..12822959b445 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppenderatorDriverRealtimeIndexTaskTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITAppenderatorDriverRealtimeIndexTaskTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.tests.indexer; import com.google.common.base.Throwables; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITNestedQueryPushDownTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITNestedQueryPushDownTest.java index 1825c6d24501..90ba30419c2b 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITNestedQueryPushDownTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITNestedQueryPushDownTest.java @@ -50,6 +50,7 @@ public class ITNestedQueryPushDownTest extends AbstractIndexerTest @Inject ClientInfoResourceTestClient clientInfoResourceTestClient; + @Test public void testIndexData() { diff --git a/pom.xml b/pom.xml index 5f55b3a70f39..c917892c73c9 100644 --- a/pom.xml +++ b/pom.xml @@ -879,9 +879,9 @@ org.apache.maven.plugins maven-checkstyle-plugin - 2.17 + 3.0.0 - ${project.build.sourceDirectory} + ${project.build.sourceDirectory} true codestyle/checkstyle.xml codestyle/checkstyle-suppressions.xml @@ -895,7 +895,7 @@ com.puppycrawl.tools checkstyle - 8.0 + 8.14 diff --git a/processing/src/main/java/org/apache/druid/collections/bitmap/RoaringBitmapFactory.java b/processing/src/main/java/org/apache/druid/collections/bitmap/RoaringBitmapFactory.java index 66f7bf096b6c..8836d15bd87b 100644 --- a/processing/src/main/java/org/apache/druid/collections/bitmap/RoaringBitmapFactory.java +++ b/processing/src/main/java/org/apache/druid/collections/bitmap/RoaringBitmapFactory.java @@ -55,6 +55,7 @@ public class RoaringBitmapFactory implements BitmapFactory throw Throwables.propagate(e); } } + private static final WrappedImmutableRoaringBitmap WRAPPED_IMMUTABLE_ROARING_BITMAP = new WrappedImmutableRoaringBitmap(EMPTY_IMMUTABLE_BITMAP); diff --git a/processing/src/main/java/org/apache/druid/query/LegacyDataSource.java b/processing/src/main/java/org/apache/druid/query/LegacyDataSource.java index 79c4014413ad..d3812489be20 100644 --- a/processing/src/main/java/org/apache/druid/query/LegacyDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/LegacyDataSource.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/processing/src/main/java/org/apache/druid/query/SubqueryQueryRunner.java b/processing/src/main/java/org/apache/druid/query/SubqueryQueryRunner.java index 50dca91101c4..25eb1f6a982d 100644 --- a/processing/src/main/java/org/apache/druid/query/SubqueryQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/SubqueryQueryRunner.java @@ -19,8 +19,8 @@ package org.apache.druid.query; -import org.apache.druid.java.util.common.guava.Sequence; import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.groupby.GroupByQueryConfig; import java.util.Map; diff --git a/processing/src/main/java/org/apache/druid/query/TableDataSource.java b/processing/src/main/java/org/apache/druid/query/TableDataSource.java index bbebb51d593c..f26aa53b93a5 100644 --- a/processing/src/main/java/org/apache/druid/query/TableDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/TableDataSource.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/processing/src/main/java/org/apache/druid/query/dimension/PrefixFilteredDimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/PrefixFilteredDimensionSpec.java index d4904b5c5c02..d3f2919b1635 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/PrefixFilteredDimensionSpec.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/PrefixFilteredDimensionSpec.java @@ -22,14 +22,15 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; +import it.unimi.dsi.fastutil.ints.Int2IntMap; +import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.filter.DimFilterUtils; import org.apache.druid.segment.DimensionSelector; -import it.unimi.dsi.fastutil.ints.Int2IntMap; -import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap; -import java.nio.ByteBuffer; + import javax.annotation.Nullable; +import java.nio.ByteBuffer; /** */ diff --git a/processing/src/main/java/org/apache/druid/query/filter/SpatialDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/SpatialDimFilter.java index 1cd2c4624ac8..0d12ac5979e8 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/SpatialDimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/SpatialDimFilter.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java index c44605e8a0a5..6cb96346ac0f 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/DictionaryBuildingStringGroupByColumnSelectorStrategy.java @@ -43,6 +43,7 @@ public class DictionaryBuildingStringGroupByColumnSelectorStrategy extends Strin private int nextId = 0; private final List dictionary = new ArrayList<>(); private final Object2IntOpenHashMap reverseDictionary = new Object2IntOpenHashMap<>(); + { reverseDictionary.defaultReturnValue(-1); } diff --git a/processing/src/main/java/org/apache/druid/query/monomorphicprocessing/SpecializationService.java b/processing/src/main/java/org/apache/druid/query/monomorphicprocessing/SpecializationService.java index daca57542de7..43f1a1a5afdc 100644 --- a/processing/src/main/java/org/apache/druid/query/monomorphicprocessing/SpecializationService.java +++ b/processing/src/main/java/org/apache/druid/query/monomorphicprocessing/SpecializationService.java @@ -67,6 +67,7 @@ public final class SpecializationService private static final Logger LOG = new Logger(SpecializationService.class); private static final Unsafe UNSAFE; + static { try { Field theUnsafe = Unsafe.class.getDeclaredField("theUnsafe"); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 3bce2011cf62..e780d36e10bd 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.scan; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index 1502a502530f..341a499c447e 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.scan; import com.google.common.base.Preconditions; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java index b655b21a155d..3f56054b5f49 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.scan; import org.apache.druid.java.util.common.guava.Sequence; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index c627469016a8..73d9f680239c 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.scan; import com.google.common.base.Function; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java index 6fa38250478d..1c14d54cd2c1 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.scan; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/processing/src/main/java/org/apache/druid/segment/AbstractSegment.java b/processing/src/main/java/org/apache/druid/segment/AbstractSegment.java index a7fb22cca39c..6bf1f670440f 100644 --- a/processing/src/main/java/org/apache/druid/segment/AbstractSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/AbstractSegment.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment; public abstract class AbstractSegment implements Segment diff --git a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java index 0729ea7324b9..92f95121e2e6 100644 --- a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java @@ -49,6 +49,7 @@ public class BitmapOffset extends Offset private static final String DEFAULT_FULLNESS_FACTORIZATION_STOPS = "0.01,0.1,0.3,0.5,0.7,0.9,0.99"; private static final double[] BITMAP_FULLNESS_FACTORIZATION_STOPS; private static final String[] FACTORIZED_FULLNESS; + static { String stopString = System.getProperty("bitmapFullnessFactorizationStops", DEFAULT_FULLNESS_FACTORIZATION_STOPS); String[] stopsArray = stopString.split(","); @@ -67,7 +68,6 @@ public class BitmapOffset extends Offset } Arrays.sort(BITMAP_FULLNESS_FACTORIZATION_STOPS); - double firstStop = BITMAP_FULLNESS_FACTORIZATION_STOPS[0]; if (Double.isNaN(firstStop) || firstStop <= 0.0) { throw new RE("First bitmapFullnessFactorizationStop[%d] should be > 0", firstStop); diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java index 2353a9b8a05f..e68a119113ff 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java @@ -243,7 +243,6 @@ class RowIteratorImpl implements TransformableRowIterator offset::getOffset ); - markedDimensionValueSelectors = dimensionHandlers .stream() .map(DimensionHandler::makeNewSettableEncodedValueSelector) diff --git a/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java index 2ea703aa6294..37b228d0842e 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment.filter; import com.google.common.base.Preconditions; diff --git a/processing/src/main/java/org/apache/druid/segment/serde/ComplexColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/ComplexColumnPartSerde.java index 0e5592619e8b..c36235156812 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/ComplexColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/ComplexColumnPartSerde.java @@ -34,6 +34,7 @@ public class ComplexColumnPartSerde implements ColumnPartSerde private final String typeName; private final ComplexMetricSerde serde; private final Serializer serializer; + private ComplexColumnPartSerde(String typeName, Serializer serializer) { this.typeName = typeName; diff --git a/processing/src/main/java/org/apache/druid/segment/serde/SpatialIndexColumnPartSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/SpatialIndexColumnPartSupplier.java index c5a3653e7e64..8f117543658b 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/SpatialIndexColumnPartSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/SpatialIndexColumnPartSupplier.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment.serde; import com.google.common.base.Supplier; diff --git a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java index bc712b70a6f3..2ed7b7131f8f 100644 --- a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java +++ b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query; import com.google.common.base.Throwables; diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/JavaScriptAggregatorBenchmark.java b/processing/src/test/java/org/apache/druid/query/aggregation/JavaScriptAggregatorBenchmark.java index 79d63c3d4e93..e3024782f7b2 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/JavaScriptAggregatorBenchmark.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/JavaScriptAggregatorBenchmark.java @@ -26,7 +26,6 @@ import java.util.HashMap; import java.util.Map; - /** * TODO rewrite to use JMH and move to benchmarks project */ @@ -34,6 +33,7 @@ public class JavaScriptAggregatorBenchmark extends SimpleBenchmark { protected static final Map scriptDoubleSum = new HashMap<>(); + static { scriptDoubleSum.put("fnAggregate", "function aggregate(current, a) { return current + a }"); scriptDoubleSum.put("fnReset", "function reset() { return 0 }"); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java index ec56024ef51e..c17e645aa8bd 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java @@ -340,22 +340,18 @@ public CardinalityAggregatorTest() false ); - String superJsFn = "function(str) { return 'super-' + str; }"; ExtractionFn superFn = new JavaScriptExtractionFn(superJsFn, false, JavaScriptConfig.getEnabledInstance()); dim1WithExtraction = new TestDimensionSelector(values1, superFn); dim2WithExtraction = new TestDimensionSelector(values2, superFn); - selectorListWithExtraction = Lists.newArrayList( - (DimensionSelector) dim1WithExtraction, - dim2WithExtraction - ); + selectorListWithExtraction = Lists.newArrayList(dim1WithExtraction, dim2WithExtraction); dimInfoListWithExtraction = Lists.newArrayList( - new ColumnSelectorPlus( + new ColumnSelectorPlus<>( dimSpec1.getDimension(), dimSpec1.getOutputName(), new StringCardinalityAggregatorColumnSelectorStrategy(), dim1WithExtraction ), - new ColumnSelectorPlus( + new ColumnSelectorPlus<>( dimSpec1.getDimension(), dimSpec1.getOutputName(), new StringCardinalityAggregatorColumnSelectorStrategy(), dim2WithExtraction @@ -366,17 +362,14 @@ public CardinalityAggregatorTest() ExtractionFn helloFn = new JavaScriptExtractionFn(helloJsFn, false, JavaScriptConfig.getEnabledInstance()); dim1ConstantVal = new TestDimensionSelector(values1, helloFn); dim2ConstantVal = new TestDimensionSelector(values2, helloFn); - selectorListConstantVal = Lists.newArrayList( - (DimensionSelector) dim1ConstantVal, - dim2ConstantVal - ); + selectorListConstantVal = Lists.newArrayList(dim1ConstantVal, dim2ConstantVal); dimInfoListConstantVal = Lists.newArrayList( - new ColumnSelectorPlus( + new ColumnSelectorPlus<>( dimSpec1.getDimension(), dimSpec1.getOutputName(), new StringCardinalityAggregatorColumnSelectorStrategy(), dim1ConstantVal ), - new ColumnSelectorPlus( + new ColumnSelectorPlus<>( dimSpec1.getDimension(), dimSpec1.getOutputName(), new StringCardinalityAggregatorColumnSelectorStrategy(), dim2ConstantVal diff --git a/processing/src/test/java/org/apache/druid/query/dimension/PrefixFilteredDimensionSpecTest.java b/processing/src/test/java/org/apache/druid/query/dimension/PrefixFilteredDimensionSpecTest.java index 2dc8fecb0255..9538a5381563 100644 --- a/processing/src/test/java/org/apache/druid/query/dimension/PrefixFilteredDimensionSpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/dimension/PrefixFilteredDimensionSpecTest.java @@ -23,10 +23,11 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.data.IndexedInts; -import java.util.Arrays; import org.junit.Assert; import org.junit.Test; +import java.util.Arrays; + /** */ public class PrefixFilteredDimensionSpecTest diff --git a/processing/src/test/java/org/apache/druid/query/extraction/LowerExtractionFnTest.java b/processing/src/test/java/org/apache/druid/query/extraction/LowerExtractionFnTest.java index 76609ac3f8f8..d33c1ee6622d 100644 --- a/processing/src/test/java/org/apache/druid/query/extraction/LowerExtractionFnTest.java +++ b/processing/src/test/java/org/apache/druid/query/extraction/LowerExtractionFnTest.java @@ -28,6 +28,7 @@ public class LowerExtractionFnTest { ExtractionFn extractionFn = new LowerExtractionFn(null); + @Test public void testApply() { diff --git a/processing/src/test/java/org/apache/druid/query/extraction/UpperExtractionFnTest.java b/processing/src/test/java/org/apache/druid/query/extraction/UpperExtractionFnTest.java index 05a2590091f2..e9d6e2339b5f 100644 --- a/processing/src/test/java/org/apache/druid/query/extraction/UpperExtractionFnTest.java +++ b/processing/src/test/java/org/apache/druid/query/extraction/UpperExtractionFnTest.java @@ -28,6 +28,7 @@ public class UpperExtractionFnTest { ExtractionFn extractionFn = new UpperExtractionFn(null); + @Test public void testApply() { diff --git a/processing/src/test/java/org/apache/druid/query/filter/AndDimFilterTest.java b/processing/src/test/java/org/apache/druid/query/filter/AndDimFilterTest.java index 4e430c70c13b..f2d6c4348849 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/AndDimFilterTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/AndDimFilterTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.filter; import com.google.common.collect.Lists; diff --git a/processing/src/test/java/org/apache/druid/query/filter/InDimFilterSerDesrTest.java b/processing/src/test/java/org/apache/druid/query/filter/InDimFilterSerDesrTest.java index d16fb3240d90..9e550903799f 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/InDimFilterSerDesrTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/InDimFilterSerDesrTest.java @@ -37,7 +37,9 @@ public class InDimFilterSerDesrTest { private static ObjectMapper mapper; - private final String actualInFilter = "{\"type\":\"in\",\"dimension\":\"dimTest\",\"values\":[\"bad\",\"good\"],\"extractionFn\":null}"; + private final String actualInFilter = + "{\"type\":\"in\",\"dimension\":\"dimTest\",\"values\":[\"bad\",\"good\"],\"extractionFn\":null}"; + @Before public void setUp() { diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java index b013195f6b3a..e65aaa295688 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -60,6 +60,7 @@ public class GroupByQueryMergeBufferTest { private static final long TIMEOUT = 5000; + private static class TestBlockingPool extends CloseableDefaultBlockingPool { private int minRemainBufferNum; diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java index 1bf936dbec67..ce9a36e3e2bf 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.groupby; import com.fasterxml.jackson.databind.InjectableValues; diff --git a/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java index 058d0bcb50e6..f740679b22b1 100644 --- a/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/select/SelectQueryRunnerTest.java @@ -113,9 +113,11 @@ public class SelectQueryRunnerTest private static final boolean DEFAULT_FROM_NEXT = true; private static final SelectQueryConfig config = new SelectQueryConfig(true); - { + + static { config.setEnableFromNextDefault(DEFAULT_FROM_NEXT); } + private static final Supplier configSupplier = Suppliers.ofInstance(config); private static final SelectQueryQueryToolChest toolChest = new SelectQueryQueryToolChest( diff --git a/processing/src/test/java/org/apache/druid/query/select/SelectQuerySpecTest.java b/processing/src/test/java/org/apache/druid/query/select/SelectQuerySpecTest.java index 49887c2cd935..2915f6947a57 100644 --- a/processing/src/test/java/org/apache/druid/query/select/SelectQuerySpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/select/SelectQuerySpecTest.java @@ -38,12 +38,10 @@ public class SelectQuerySpecTest { private final ObjectMapper objectMapper = new DefaultObjectMapper(); + { objectMapper.setInjectableValues( - new InjectableValues.Std().addValue( - SelectQueryConfig.class, - new SelectQueryConfig(true) - ) + new InjectableValues.Std().addValue(SelectQueryConfig.class, new SelectQueryConfig(true)) ); } diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9CompatibilityTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9CompatibilityTest.java index c25fd76ef7dc..68835084c03c 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9CompatibilityTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9CompatibilityTest.java @@ -108,7 +108,6 @@ public IndexMergerV9CompatibilityTest(SegmentWriteOutMediumFactory segmentWriteO DIMS.get(1), "dim10" ); - final Map map3 = ImmutableMap.of( DIMS.get(0), ImmutableList.of("dim00", "dim01") diff --git a/processing/src/test/java/org/apache/druid/segment/data/BitmapCreationBenchmark.java b/processing/src/test/java/org/apache/druid/segment/data/BitmapCreationBenchmark.java index 777576043252..f01dc5d388ef 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/BitmapCreationBenchmark.java +++ b/processing/src/test/java/org/apache/druid/segment/data/BitmapCreationBenchmark.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment.data; import com.carrotsearch.junitbenchmarks.AbstractBenchmark; diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java index acd7498eb5d2..1d48b377338c 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java @@ -99,14 +99,11 @@ public boolean supportsBitmapIndex(BitmapIndexSelector selector) private static class NoBitmapSelectorDimFilter extends SelectorDimFilter { - public NoBitmapSelectorDimFilter( - String dimension, - String value, - ExtractionFn extractionFn - ) + NoBitmapSelectorDimFilter(String dimension, String value, ExtractionFn extractionFn) { super(dimension, value, extractionFn); } + @Override public Filter toFilter() { diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowSizeTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowSizeTest.java index 0952c436c4fa..0e5283bc7bd0 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowSizeTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowSizeTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment.incremental; import com.google.common.collect.Lists; diff --git a/server/src/main/java/org/apache/druid/metadata/SQLFirehoseDatabaseConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLFirehoseDatabaseConnector.java index 59860265cbe3..242f04bccfcb 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLFirehoseDatabaseConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLFirehoseDatabaseConnector.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.metadata; import com.fasterxml.jackson.annotation.JsonTypeInfo; diff --git a/server/src/main/java/org/apache/druid/query/lookup/MapLookupExtractorFactory.java b/server/src/main/java/org/apache/druid/query/lookup/MapLookupExtractorFactory.java index d89e197049a9..3f39e149ba3e 100644 --- a/server/src/main/java/org/apache/druid/query/lookup/MapLookupExtractorFactory.java +++ b/server/src/main/java/org/apache/druid/query/lookup/MapLookupExtractorFactory.java @@ -123,6 +123,7 @@ public int hashCode() public static class MapLookupIntrospectionHandler implements LookupIntrospectHandler { private final Map map; + public MapLookupIntrospectionHandler(Map map) { this.map = map; diff --git a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfigs.java b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfigs.java index cc0f005d0ae0..9c470ab677d0 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfigs.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfigs.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment.indexing; public class TuningConfigs diff --git a/server/src/main/java/org/apache/druid/segment/loading/LocalFileTimestampVersionFinder.java b/server/src/main/java/org/apache/druid/segment/loading/LocalFileTimestampVersionFinder.java index 4ca982db0fb8..9cb47e9338cb 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/LocalFileTimestampVersionFinder.java +++ b/server/src/main/java/org/apache/druid/segment/loading/LocalFileTimestampVersionFinder.java @@ -31,14 +31,11 @@ import java.nio.file.Path; import java.util.regex.Pattern; -/** - * This is implemented explicitly for URIExtractionNamespaceFunctionFactory - * If you have a use case for this interface beyond URIExtractionNamespaceFunctionFactory please bring it up in the dev list. - */ public class LocalFileTimestampVersionFinder extends LocalDataSegmentPuller implements SearchableVersionedDataFinder { public static final String URI_SCHEME = "file"; + private URI mostRecentInDir(final Path dir, final Pattern pattern) { long latestModified = Long.MIN_VALUE; @@ -68,7 +65,8 @@ public boolean accept(File pathname) * Matches based on a pattern in the file name. Returns the file with the latest timestamp. * * @param uri If it is a file, then the parent is searched. If it is a directory, then the directory is searched. - * @param pattern The matching filter to down-select the file names in the directory of interest. Passing `null` results in matching any file + * @param pattern The matching filter to down-select the file names in the directory of interest. Passing `null` + * results in matching any file * * @return The URI of the most recently modified file which matches the pattern, or `null` if it cannot be found */ diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java index fc380aa99246..43738f8f6842 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment.realtime.firehose; import com.fasterxml.jackson.annotation.JacksonInject; diff --git a/server/src/main/java/org/apache/druid/server/audit/SQLAuditManagerProvider.java b/server/src/main/java/org/apache/druid/server/audit/SQLAuditManagerProvider.java index a9309f01c4c1..fdb9900ab29b 100644 --- a/server/src/main/java/org/apache/druid/server/audit/SQLAuditManagerProvider.java +++ b/server/src/main/java/org/apache/druid/server/audit/SQLAuditManagerProvider.java @@ -38,6 +38,7 @@ public class SQLAuditManagerProvider implements AuditManagerProvider private final ServiceEmitter emitter; private final ObjectMapper mapper; private final SQLAuditManagerConfig config; + @Inject public SQLAuditManagerProvider( Supplier dbTables, diff --git a/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestsSnapshot.java b/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestsSnapshot.java index 96b2a24f8550..2710ff3dcd55 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestsSnapshot.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ChangeRequestsSnapshot.java @@ -55,7 +55,6 @@ public ChangeRequestsSnapshot( Preconditions.checkNotNull(resetCause, "NULL resetCause when resetCounter is true."); } - this.counter = counter; this.requests = requests; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategyFactory.java index b168ab1161e4..7e79c9a4455a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategyFactory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategyFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.server.coordinator; import com.fasterxml.jackson.annotation.JsonSubTypes; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/BytesAccumulatingResponseHandler.java b/server/src/main/java/org/apache/druid/server/coordinator/BytesAccumulatingResponseHandler.java index c9e89c3c35d7..88d82264d0ce 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/BytesAccumulatingResponseHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/BytesAccumulatingResponseHandler.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.server.coordinator; import org.apache.druid.java.util.http.client.io.AppendableByteArrayInputStream; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java index 8e8df480bdfc..726f2fa07dbd 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.server.coordinator; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategyFactory.java index 08cdd05b6eb7..c846acf3e8a6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategyFactory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategyFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.server.coordinator; import com.google.common.util.concurrent.ListeningExecutorService; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyFactory.java index 90fc6f9673bf..10a32bce2740 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyFactory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.server.coordinator; import com.google.common.util.concurrent.ListeningExecutorService; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategyFactory.java index e5770c5073da..c02554050eb8 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategyFactory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategyFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.server.coordinator; import com.google.common.util.concurrent.ListeningExecutorService; diff --git a/server/src/main/java/org/apache/druid/server/http/OverlordProxyServlet.java b/server/src/main/java/org/apache/druid/server/http/OverlordProxyServlet.java index 8ed6ac087ca9..0fa0f3e2c645 100644 --- a/server/src/main/java/org/apache/druid/server/http/OverlordProxyServlet.java +++ b/server/src/main/java/org/apache/druid/server/http/OverlordProxyServlet.java @@ -20,6 +20,7 @@ package org.apache.druid.server.http; import com.google.inject.Inject; +import com.google.inject.Provider; import org.apache.druid.client.indexing.IndexingService; import org.apache.druid.discovery.DruidLeaderClient; import org.apache.druid.guice.annotations.Global; @@ -27,7 +28,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.server.JettyUtils; import org.apache.druid.server.security.AuthConfig; -import com.google.inject.Provider; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.client.api.Request; import org.eclipse.jetty.proxy.ProxyServlet; diff --git a/server/src/main/java/org/apache/druid/server/initialization/TLSServerConfig.java b/server/src/main/java/org/apache/druid/server/initialization/TLSServerConfig.java index 4e8a625023b4..f6b8c4a19423 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/TLSServerConfig.java +++ b/server/src/main/java/org/apache/druid/server/initialization/TLSServerConfig.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.server.initialization; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/org/apache/druid/server/metrics/QueryCountStatsMonitor.java b/server/src/main/java/org/apache/druid/server/metrics/QueryCountStatsMonitor.java index e217e8de8046..c3b956c23d92 100644 --- a/server/src/main/java/org/apache/druid/server/metrics/QueryCountStatsMonitor.java +++ b/server/src/main/java/org/apache/druid/server/metrics/QueryCountStatsMonitor.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.server.metrics; import com.google.common.collect.ImmutableMap; diff --git a/server/src/main/java/org/apache/druid/server/metrics/QueryCountStatsProvider.java b/server/src/main/java/org/apache/druid/server/metrics/QueryCountStatsProvider.java index a7bd8762b24e..9c920857f801 100644 --- a/server/src/main/java/org/apache/druid/server/metrics/QueryCountStatsProvider.java +++ b/server/src/main/java/org/apache/druid/server/metrics/QueryCountStatsProvider.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.server.metrics; public interface QueryCountStatsProvider diff --git a/server/src/main/java/org/apache/druid/server/router/ConsistentHasher.java b/server/src/main/java/org/apache/druid/server/router/ConsistentHasher.java index 8af81f05c54e..ed0b9676eed0 100644 --- a/server/src/main/java/org/apache/druid/server/router/ConsistentHasher.java +++ b/server/src/main/java/org/apache/druid/server/router/ConsistentHasher.java @@ -43,16 +43,16 @@ public class ConsistentHasher private static final HashFunction DEFAULT_HASH_FN = Hashing.murmur3_128(); private final Long2ObjectRBTreeMap nodeKeySlots = new Long2ObjectRBTreeMap<>(); + { nodeKeySlots.defaultReturnValue(null); } + private final HashFunction hashFn; private final Map nodeKeyHashes = new HashMap<>(); private Set previousKeys = new HashSet<>(); - public ConsistentHasher( - final HashFunction hashFunction - ) + public ConsistentHasher(final HashFunction hashFunction) { this.hashFn = hashFunction == null ? DEFAULT_HASH_FN : hashFunction; } diff --git a/server/src/main/java/org/apache/druid/server/security/DefaultTLSCertificateCheckerModule.java b/server/src/main/java/org/apache/druid/server/security/DefaultTLSCertificateCheckerModule.java index 8b2384b2eef4..3c2745ac352d 100644 --- a/server/src/main/java/org/apache/druid/server/security/DefaultTLSCertificateCheckerModule.java +++ b/server/src/main/java/org/apache/druid/server/security/DefaultTLSCertificateCheckerModule.java @@ -21,7 +21,6 @@ import com.google.inject.Binder; import com.google.inject.Module; - import com.google.inject.name.Names; public class DefaultTLSCertificateCheckerModule implements Module diff --git a/server/src/main/java/org/apache/druid/server/security/TLSUtils.java b/server/src/main/java/org/apache/druid/server/security/TLSUtils.java index 69fcdd234bb9..83d1e507f16e 100644 --- a/server/src/main/java/org/apache/druid/server/security/TLSUtils.java +++ b/server/src/main/java/org/apache/druid/server/security/TLSUtils.java @@ -23,7 +23,6 @@ import com.google.common.base.Throwables; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.PasswordProvider; - import org.eclipse.jetty.util.ssl.AliasedX509ExtendedKeyManager; import javax.annotation.Nullable; diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java index 47407215af0b..d05cfc809aca 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.client; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java index 3609e2af7778..5a283f893978 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTestUtils.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.client; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java index f42199c25b82..a86c07e1c5f0 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.metadata; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java index b5f37a6cca3c..09e621ba5ee0 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment.realtime.appenderator; import org.apache.druid.data.input.InputRow; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentWithStateTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentWithStateTest.java index 8f037ac24033..c19ec56b6d76 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentWithStateTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentWithStateTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.segment.realtime.appenderator; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java index d6c51a91c52f..1bb58ab40905 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java @@ -133,8 +133,8 @@ public void testReadFromIndexAndWriteAnotherIndex() throws Exception .withMetrics(AGGREGATORS_REINDEX.toArray(new AggregatorFactory[0])) .build() ) - .setMaxRowCount(5000) - .buildOnheap() + .setMaxRowCount(5000) + .buildOnheap() ) { final StorageAdapter sa = new QueryableIndexStorageAdapter(qi); final WindowedStorageAdapter wsa = new WindowedStorageAdapter(sa, sa.getInterval()); @@ -223,8 +223,8 @@ private void createTestIndex(File segmentDir) throws Exception .withMetrics(AGGREGATORS.toArray(new AggregatorFactory[0])) .build() ) - .setMaxRowCount(5000) - .buildOnheap() + .setMaxRowCount(5000) + .buildOnheap() ) { for (String line : rows) { index.add(parser.parse(line)); diff --git a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java index dcfeb0bf341e..e6a341fc544f 100644 --- a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java @@ -19,9 +19,9 @@ package org.apache.druid.server; -import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.ListenableFuture; diff --git a/server/src/test/java/org/apache/druid/server/metrics/ExceptionCapturingServiceEmitter.java b/server/src/test/java/org/apache/druid/server/metrics/ExceptionCapturingServiceEmitter.java index f70595a2fa05..c28744220f17 100644 --- a/server/src/test/java/org/apache/druid/server/metrics/ExceptionCapturingServiceEmitter.java +++ b/server/src/test/java/org/apache/druid/server/metrics/ExceptionCapturingServiceEmitter.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.server.metrics; import org.apache.druid.java.util.emitter.EmittingLogger; diff --git a/services/src/test/java/org/apache/druid/cli/MainTest.java b/services/src/test/java/org/apache/druid/cli/MainTest.java index 9d8a891a4187..1a35b6664877 100644 --- a/services/src/test/java/org/apache/druid/cli/MainTest.java +++ b/services/src/test/java/org/apache/druid/cli/MainTest.java @@ -55,11 +55,14 @@ public static Iterable constructorFeeder() new Object[]{new CliRouter()} ); } + private final GuiceRunnable runnable; + public MainTest(GuiceRunnable runnable) { this.runnable = runnable; } + @Test public void testSimpleInjection() { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/SortProject.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/SortProject.java index f55fa18fb03a..f7c503b83f2f 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/SortProject.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/SortProject.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.sql.calcite.rel; import com.google.common.base.Preconditions; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataHolder.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataHolder.java index cb446e91109c..b2666e149b58 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataHolder.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataHolder.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.sql.calcite.schema; import org.apache.druid.sql.calcite.table.RowSignature; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java index a004e80b8103..431f4b1730eb 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.sql.calcite.schema; import com.fasterxml.jackson.core.JsonProcessingException; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index 851adff94548..9dc5a0e55f35 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.sql.calcite.schema; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java index e0b6825cc738..363d595f38d3 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java @@ -34,7 +34,10 @@ import javax.annotation.Nullable; import java.util.List; import java.util.concurrent.Executor; -// this class is used for testing and benchmark + +/** + * This class is used for testing and benchmark + */ public class TestServerInventoryView implements TimelineServerView { private static final DruidServerMetadata DUMMY_SERVER = new DruidServerMetadata( From 836a5a7124b2ce9a800af781a4aca41776c52121 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 27 Nov 2018 16:38:43 -0800 Subject: [PATCH 52/87] fix kinesis offset discrepancy with kafka --- ...ementalPublishingKafkaIndexTaskRunner.java | 65 +++++- .../druid/indexing/kafka/KafkaIOConfig.java | 13 +- .../kafka/LegacyKafkaIndexTaskRunner.java | 34 ++- .../indexing/kafka/KafkaIOConfigTest.java | 5 + .../indexing/kinesis/KinesisIOConfig.java | 15 +- .../kinesis/KinesisIndexTaskRunner.java | 71 +++++- .../kinesis/KinesisIndexTaskTest.java | 203 +++++------------- .../src/test/resources/log4j2.xml | 35 +++ .../SeekableStreamIOConfig.java | 24 ++- .../SeekableStreamIndexTaskRunner.java | 111 +++++++--- .../SeekableStreamPartitions.java | 1 + .../supervisor/SeekableStreamSupervisor.java | 7 +- 12 files changed, 349 insertions(+), 235 deletions(-) create mode 100644 extensions-core/kinesis-indexing-service/src/test/resources/log4j2.xml diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 62d595c1d3e9..30614eb06aa1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -51,6 +51,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -88,7 +89,7 @@ public IncrementalPublishingKafkaIndexTaskRunner( @Override protected Long getNextSequenceNumber( - RecordSupplier recordSupplier, StreamPartition partition, @NotNull Long sequenceNumber + @NotNull Long sequenceNumber ) { return sequenceNumber + 1; @@ -203,6 +204,32 @@ protected OrderedSequenceNumber createSequenceNumber(Long sequenceNumber) return KafkaSequenceNumber.of(sequenceNumber); } + @Override + protected Type getRunnerType() + { + return Type.KAFKA; + } + + @Override + protected SequenceMetadata createSequenceMetaData( + int sequenceId, + String sequenceName, + Map startOffsets, + Map endOffsets, + boolean checkpointed, + Set exclusiveStartPartition + ) + { + return new KafkaSequenceMetaData( + sequenceId, + sequenceName, + startOffsets, + endOffsets, + checkpointed, + null + ); + } + @Nullable @Override protected TreeMap> getCheckPointsFromContext( @@ -223,4 +250,40 @@ protected TreeMap> getCheckPointsFromContext( return null; } } + + private class KafkaSequenceMetaData extends SequenceMetadata + { + + public KafkaSequenceMetaData( + int sequenceId, + String sequenceName, + Map startOffsets, + Map endOffsets, + boolean checkpointed, + Set exclusiveStartPartitions + ) + { + super(sequenceId, sequenceName, startOffsets, endOffsets, checkpointed, null); + } + + @Override + protected boolean canHandle(OrderedPartitionableRecord record) + { + lock.lock(); + try { + final OrderedSequenceNumber partitionEndOffset = createSequenceNumber(endOffsets.get(record.getPartitionId())); + final OrderedSequenceNumber partitionStartOffset = createSequenceNumber(startOffsets.get(record.getPartitionId())); + final OrderedSequenceNumber recordOffset = createSequenceNumber(record.getSequenceNumber()); + return isOpen() + && recordOffset != null + && partitionEndOffset != null + && partitionStartOffset != null + && recordOffset.compareTo(partitionStartOffset) >= 0 + && recordOffset.compareTo(partitionEndOffset) < 0; + } + finally { + lock.unlock(); + } + } + } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java index 443da7c0ba56..34e38e91f9c1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java @@ -28,7 +28,6 @@ import javax.annotation.Nullable; import java.util.Map; -import java.util.Set; public class KafkaIOConfig extends SeekableStreamIOConfig { @@ -55,7 +54,8 @@ public KafkaIOConfig( useTransaction, minimumMessageTime, maximumMessageTime, - skipOffsetGaps + skipOffsetGaps, + null ); this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); @@ -71,15 +71,6 @@ public KafkaIOConfig( } } - // exclusive starting sequence partitions are used only for kinesis where the starting - // sequence number for certain partitions are discarded because they've already been - // read by a previous task - @Override - public Set getExclusiveStartSequenceNumberPartitions() - { - return null; - } - @JsonProperty public Map getConsumerProperties() { diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index c496f5fb5c50..da53a50d16ce 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -52,7 +52,6 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; -import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -625,14 +624,6 @@ public void run() ); } - @Override - protected Long getNextSequenceNumber( - RecordSupplier recordSupplier, StreamPartition partition, Long sequenceNumber - ) - { - return sequenceNumber + 1; - } - @Override protected List> getRecords( RecordSupplier recordSupplier, TaskToolbox toolbox @@ -802,6 +793,12 @@ protected TreeMap> getCheckPointsFromContext( return null; } + @Override + protected Long getNextSequenceNumber(Long sequenceNumber) + { + return null; + } + private void handleParseException(ParseException pe, ConsumerRecord record) { if (pe.isFromPartiallyValidRow()) { @@ -1211,6 +1208,12 @@ protected OrderedSequenceNumber createSequenceNumber(Long sequenceNumber) return null; } + @Override + protected Type getRunnerType() + { + return Type.KAFKA; + } + @Override @GET @Path("/time/start") @@ -1220,4 +1223,17 @@ public DateTime getStartTime(@Context final HttpServletRequest req) authorizationCheck(req, Action.WRITE); return startTime; } + + @Override + protected SequenceMetadata createSequenceMetaData( + int sequenceId, + String sequenceName, + Map startOffsets, + Map endOffsets, + boolean checkpointed, + Set exclusiveStartPartitions + ) + { + return null; + } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java index e54b9ac0a014..c40a3e73bff6 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java @@ -32,6 +32,8 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.Collections; + public class KafkaIOConfigTest { private final ObjectMapper mapper; @@ -76,6 +78,7 @@ public void testSerdeWithDefaults() throws Exception Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent()); Assert.assertFalse("maximumMessageTime", config.getMaximumMessageTime().isPresent()); Assert.assertFalse("skipOffsetGaps", config.isSkipOffsetGaps()); + Assert.assertEquals(Collections.EMPTY_SET, config.getExclusiveStartSequenceNumberPartitions()); } @Test @@ -113,6 +116,8 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(DateTimes.of("2016-05-31T12:00Z"), config.getMinimumMessageTime().get()); Assert.assertEquals(DateTimes.of("2016-05-31T14:00Z"), config.getMaximumMessageTime().get()); Assert.assertTrue("skipOffsetGaps", config.isSkipOffsetGaps()); + Assert.assertEquals(Collections.EMPTY_SET, config.getExclusiveStartSequenceNumberPartitions()); + } @Test diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java index 326f30b64bde..95287f73cf99 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java @@ -42,7 +42,7 @@ public class KinesisIOConfig extends SeekableStreamIOConfig private final Integer fetchDelayMillis; private final String awsAccessKeyId; private final String awsSecretAccessKey; - private final Set exclusiveStartSequenceNumberPartitions; + private final String awsAssumedRoleArn; private final String awsExternalId; private final boolean deaggregate; @@ -76,7 +76,8 @@ public KinesisIOConfig( useTransaction, minimumMessageTime, maximumMessageTime, - true + true, + exclusiveStartSequenceNumberPartitions ); Preconditions.checkArgument(endPartitions.getPartitionSequenceNumberMap() .values() @@ -89,7 +90,6 @@ public KinesisIOConfig( this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : DEFAULT_FETCH_DELAY_MILLIS; this.awsAccessKeyId = awsAccessKeyId; this.awsSecretAccessKey = awsSecretAccessKey; - this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions; this.awsAssumedRoleArn = awsAssumedRoleArn; this.awsExternalId = awsExternalId; this.deaggregate = deaggregate; @@ -131,13 +131,6 @@ public String getAwsSecretAccessKey() return awsSecretAccessKey; } - @Override - @JsonProperty - public Set getExclusiveStartSequenceNumberPartitions() - { - return exclusiveStartSequenceNumberPartitions; - } - @JsonProperty public String getAwsAssumedRoleArn() { @@ -172,7 +165,7 @@ public String toString() ", fetchDelayMillis=" + fetchDelayMillis + ", awsAccessKeyId='" + awsAccessKeyId + '\'' + ", awsSecretAccessKey=" + "************************" + - ", exclusiveStartSequenceNumberPartitions=" + exclusiveStartSequenceNumberPartitions + + ", exclusiveStartSequenceNumberPartitions=" + getExclusiveStartSequenceNumberPartitions() + ", awsAssumedRoleArn='" + awsAssumedRoleArn + '\'' + ", awsExternalId='" + awsExternalId + '\'' + ", deaggregate=" + deaggregate + diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index d49af6bb9391..cb9d1a12f003 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -32,7 +32,6 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; -import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; @@ -44,6 +43,7 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeMap; import java.util.concurrent.CopyOnWriteArrayList; @@ -77,11 +77,10 @@ public KinesisIndexTaskRunner( @Override protected String getNextSequenceNumber( - RecordSupplier recordSupplier, StreamPartition partition, String sequenceNumber + String sequenceNumber ) { - String sequence = recordSupplier.getPosition(partition); - return sequence == null ? sequenceNumber : sequence; + return sequenceNumber; } @Nonnull @@ -121,6 +120,32 @@ protected OrderedSequenceNumber createSequenceNumber(String sequenceNumb return KinesisSequenceNumber.of(sequenceNumber); } + @Override + protected Type getRunnerType() + { + return Type.KINESIS; + } + + @Override + protected SequenceMetadata createSequenceMetaData( + int sequenceId, + String sequenceName, + Map startOffsets, + Map endOffsets, + boolean checkpointed, + Set exclusiveStartPartitions + ) + { + return new KinesisSequenceMetaData( + sequenceId, + sequenceName, + startOffsets, + endOffsets, + checkpointed, + exclusiveStartPartitions + ); + } + @Nullable @Override protected TreeMap> getCheckPointsFromContext( @@ -142,5 +167,43 @@ protected TreeMap> getCheckPointsFromContext( } } + private class KinesisSequenceMetaData extends SequenceMetadata + { + + public KinesisSequenceMetaData( + int sequenceId, + String sequenceName, + Map startOffsets, + Map endOffsets, + boolean checkpointed, + Set exclusiveStartPartitions + ) + { + super(sequenceId, sequenceName, startOffsets, endOffsets, checkpointed, exclusiveStartPartitions); + } + + @Override + protected boolean canHandle(OrderedPartitionableRecord record) + { + lock.lock(); + try { + final OrderedSequenceNumber partitionEndOffset = createSequenceNumber(endOffsets.get(record.getPartitionId())); + final OrderedSequenceNumber partitionStartOffset = createSequenceNumber(startOffsets.get(record.getPartitionId())); + final OrderedSequenceNumber recordOffset = createSequenceNumber(record.getSequenceNumber()); + return isOpen() + && recordOffset != null + && partitionEndOffset != null + && partitionStartOffset != null + && recordOffset.compareTo(partitionStartOffset) + >= (getExclusiveStartPartitions().contains(record.getPartitionId()) ? 1 : 0) + && recordOffset.compareTo(partitionEndOffset) <= 0; + } + finally { + lock.unlock(); + } + } + + } + } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index f5da32c71ed7..12490f2d0433 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -281,6 +281,7 @@ public class KinesisIndexTaskTest private Long maxTotalRows = null; private Period intermediateHandoffPeriod = null; private int maxRecordsPerPoll; + private boolean skipAvailabilityCheck = false; private TaskToolboxFactory toolboxFactory; private IndexerMetadataStorageCoordinator metadataStorageCoordinator; @@ -385,6 +386,7 @@ public void setupTest() throws IOException, InterruptedException logParseExceptions = true; maxParseExceptions = null; maxSavedParseExceptions = null; + skipAvailabilityCheck = false; doHandoff = true; stream = getStreamName(); reportsFile = File.createTempFile("KinesisIndexTaskTestReports-" + System.currentTimeMillis(), "json"); @@ -435,7 +437,7 @@ public void testRunAfterDataInserted() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) )), true, null, @@ -473,7 +475,7 @@ public void testRunAfterDataInserted() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -610,7 +612,7 @@ public void testIncrementalHandOff() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5), + getSequenceNumber(res, shardId1, 4), shardId0, getSequenceNumber(res, shardId0, 0) ) @@ -619,18 +621,18 @@ public void testIncrementalHandOff() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4), + getSequenceNumber(res, shardId1, 3), shardId0, - getSequenceNumber(res, shardId0, 2) + getSequenceNumber(res, shardId0, 1) ) ); final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 10), + getSequenceNumber(res, shardId1, 9), shardId0, - getSequenceNumber(res, shardId0, 2) + getSequenceNumber(res, shardId0, 1) ) ); final KinesisIndexTask task = createTask( @@ -696,9 +698,9 @@ public void testIncrementalHandOff() throws Exception Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 10), + getSequenceNumber(res, shardId1, 9), shardId0, - getSequenceNumber(res, shardId0, 2) + getSequenceNumber(res, shardId0, 1) ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -739,21 +741,21 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3) + getSequenceNumber(res, shardId1, 2) ) ); final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 10) + getSequenceNumber(res, shardId1, 9) ) ); final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 11) + getSequenceNumber(res, shardId1, 10) ) ); @@ -839,7 +841,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 11) + getSequenceNumber(res, shardId1, 10) ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -853,115 +855,6 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); } - @Test(timeout = 120_000L) - public void testTimeBasedIncrementalHandOff() throws Exception - { - - final String baseSequenceName = "sequence0"; - // as soon as any segment hits maxRowsPerSegment or intermediateHandoffPeriod, incremental publishing should happen - maxRowsPerSegment = Integer.MAX_VALUE; - intermediateHandoffPeriod = new Period().withSeconds(0); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream, 0, 13)); - - final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( - stream, - ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 0) - ) - ); - // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering - // of events fetched across two partitions from Kafka - final SeekableStreamPartitions checkpoint = new SeekableStreamPartitions<>( - stream, - ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 1) - ) - ); - - final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( - stream, - ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 2) - ) - ); - - - final KinesisIndexTask task = createTask( - null, - new KinesisIOConfig( - null, - baseSequenceName, - startPartitions, - endPartitions, - true, - null, - null, - null, - LocalstackTestRunner.getEndpointKinesis(), - null, - null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, - null, - null, - null, - false - ) - ); - - final ListenableFuture future = runTask(task); - - // task will pause for checkpointing - while (task.getRunner().getStatus() != Status.PAUSED) { - Thread.sleep(10); - } - final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertEquals(checkpoint.getPartitionSequenceNumberMap(), currentOffsets); - task.getRunner().setEndOffsets(currentOffsets, false); - Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - - Assert.assertEquals(1, checkpointRequestsHash.size()); - Assert.assertTrue( - checkpointRequestsHash.contains( - Objects.hash( - DATA_SCHEMA.getDataSource(), - 0, - new KinesisDataSourceMetadata(startPartitions), - new KinesisDataSourceMetadata(new SeekableStreamPartitions<>( - stream, - checkpoint.getPartitionSequenceNumberMap() - )) - ) - ) - ); - - // Check metrics - Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); - Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - - // Check published metadata - SegmentDescriptor desc1 = SD(task, "2008/P1D", 0); - SegmentDescriptor desc2 = SD(task, "2009/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); - Assert.assertEquals( - new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 2) - ))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) - ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); - } - - @Test(timeout = 120_000L) public void testRunWithMinimumMessageTime() throws Exception { @@ -979,7 +872,7 @@ public void testRunWithMinimumMessageTime() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) )), true, null, @@ -1022,7 +915,7 @@ public void testRunWithMinimumMessageTime() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1050,7 +943,7 @@ public void testRunWithMaximumMessageTime() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) )), true, null, @@ -1094,7 +987,7 @@ public void testRunWithMaximumMessageTime() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1131,7 +1024,7 @@ public void testRunWithTransformSpec() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) )), true, null, @@ -1173,7 +1066,7 @@ public void testRunWithTransformSpec() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1253,7 +1146,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) )), true, null, @@ -1288,7 +1181,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1320,7 +1213,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) )), true, null, @@ -1355,7 +1248,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1389,7 +1282,7 @@ public void testReportParseExceptions() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 6) + getSequenceNumber(res, shardId1, 5) )), true, null, @@ -1444,7 +1337,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 13) + getSequenceNumber(res, shardId1, 12) )), true, null, @@ -1489,7 +1382,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 13) + getSequenceNumber(res, shardId1, 12) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1546,7 +1439,7 @@ public void testMultipleParseExceptionsFailure() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 10) + getSequenceNumber(res, shardId1, 9) )), true, null, @@ -1624,7 +1517,7 @@ public void testRunReplicas() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) )), true, null, @@ -1652,7 +1545,7 @@ public void testRunReplicas() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) )), true, null, @@ -1695,7 +1588,7 @@ public void testRunReplicas() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1724,7 +1617,7 @@ public void testRunConflicting() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) )), true, null, @@ -1752,7 +1645,7 @@ public void testRunConflicting() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 10) + getSequenceNumber(res, shardId1, 9) )), true, null, @@ -1796,7 +1689,7 @@ public void testRunConflicting() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1825,7 +1718,7 @@ public void testRunConflictingWithoutTransactions() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) )), false, null, @@ -1853,7 +1746,7 @@ public void testRunConflictingWithoutTransactions() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 10) + getSequenceNumber(res, shardId1, 9) )), false, null, @@ -1926,9 +1819,9 @@ public void testRunOneTaskTwoPartitions() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5), + getSequenceNumber(res, shardId1, 4), shardId0, - getSequenceNumber(res, shardId0, 2) + getSequenceNumber(res, shardId0, 1) )), true, null, @@ -1970,9 +1863,9 @@ public void testRunOneTaskTwoPartitions() throws Exception Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5), + getSequenceNumber(res, shardId1, 4), shardId0, - getSequenceNumber(res, shardId0, 2) + getSequenceNumber(res, shardId0, 1) ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -2006,7 +1899,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) )), true, null, @@ -2084,7 +1977,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception new KinesisDataSourceMetadata( new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5), + getSequenceNumber(res, shardId1, 4), shardId0, OrderedPartitionableRecord.END_OF_SHARD_MARKER ))), @@ -2349,7 +2242,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) )), true, null, @@ -2384,7 +2277,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + getSequenceNumber(res, shardId1, 4) ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -2476,7 +2369,7 @@ private KinesisIndexTask createTask( reportParseExceptions, handoffConditionTimeout, resetOffsetAutomatically, - true, + skipAvailabilityCheck, null, null, null, @@ -2907,13 +2800,13 @@ private AmazonKinesis getKinesisClientInstance() throws InterruptedException return kinesis; } - private static String getSequenceNumber(List entries, String shardId, int stream) + private static String getSequenceNumber(List entries, String shardId, int offset) { List sortedEntries = entries.stream() .filter(e -> e.getShardId().equals(shardId)) .sorted(Comparator.comparing(e -> KinesisSequenceNumber.of(e.getSequenceNumber()))) .collect(Collectors.toList()); - return sortedEntries.get(stream).getSequenceNumber(); + return sortedEntries.get(offset).getSequenceNumber(); } diff --git a/extensions-core/kinesis-indexing-service/src/test/resources/log4j2.xml b/extensions-core/kinesis-indexing-service/src/test/resources/log4j2.xml new file mode 100644 index 000000000000..bca6c69fdb1b --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/test/resources/log4j2.xml @@ -0,0 +1,35 @@ + + + + + + + + + + + + + + + + + + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java index 6d701d2203c9..a6fc881fc532 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java @@ -27,6 +27,7 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; +import java.util.Collections; import java.util.Set; public abstract class SeekableStreamIOConfig implements IOConfig @@ -43,7 +44,7 @@ public abstract class SeekableStreamIOConfig implem private final Optional minimumMessageTime; private final Optional maximumMessageTime; private final boolean skipOffsetGaps; - + private final Set exclusiveStartSequenceNumberPartitions; @JsonCreator public SeekableStreamIOConfig( @@ -54,7 +55,8 @@ public SeekableStreamIOConfig( @JsonProperty("useTransaction") Boolean useTransaction, @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, - @JsonProperty("skipOffsetGaps") Boolean skipOffsetGaps + @JsonProperty("skipOffsetGaps") Boolean skipOffsetGaps, + @JsonProperty("exclusiveStartSequenceNumberPartitions") Set exclusiveStartSequenceNumberPartitions ) { this.taskGroupId = taskGroupId; @@ -65,6 +67,9 @@ public SeekableStreamIOConfig( this.minimumMessageTime = Optional.fromNullable(minimumMessageTime); this.maximumMessageTime = Optional.fromNullable(maximumMessageTime); this.skipOffsetGaps = skipOffsetGaps != null ? skipOffsetGaps : DEFAULT_SKIP_OFFSET_GAPS; + this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions == null + ? Collections.emptySet() + : exclusiveStartSequenceNumberPartitions; Preconditions.checkArgument( startPartitions.getStream().equals(endPartitions.getStream()), @@ -79,6 +84,15 @@ public SeekableStreamIOConfig( ); } + // exclusive starting sequence partitions are used only for kinesis where the starting + // sequence number for certain partitions are discarded because they've already been + // read by a previous task + @JsonProperty + public Set getExclusiveStartSequenceNumberPartitions() + { + return exclusiveStartSequenceNumberPartitions; + } + @Nullable @JsonProperty public Integer getTaskGroupId() @@ -128,12 +142,6 @@ public boolean isSkipOffsetGaps() return skipOffsetGaps; } - // exclusive starting sequence partitions are used only for kinesis where the starting - // sequence number for certain partitions are discarded because they've already been - // read by a previous task - @JsonProperty - public abstract Set getExclusiveStartSequenceNumberPartitions(); - @Override public abstract String toString(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 75e8f747c60c..ae214c184704 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -198,6 +198,7 @@ public enum Status private final List> publishWaitList = new ArrayList<>(); private final List> handOffWaitList = new ArrayList<>(); private final Map initialOffsetsSnapshot = new HashMap<>(); + private final Set exclusiveStartingPartitions = new HashSet<>(); // true for kafka, falsse for kinesis private final boolean isSkipSegmentLineageCheck; @@ -272,34 +273,39 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception if (!restoreSequences()) { final TreeMap> checkpoints = getCheckPointsFromContext(toolbox, task); if (checkpoints != null) { + boolean exclusive = false; Iterator>> sequenceOffsets = checkpoints.entrySet() .iterator(); Map.Entry> previous = sequenceOffsets.next(); while (sequenceOffsets.hasNext()) { Map.Entry> current = sequenceOffsets.next(); - sequences.add(new SequenceMetadata( + sequences.add(createSequenceMetaData( previous.getKey(), StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), previous.getValue(), current.getValue(), - true + true, + exclusive ? previous.getValue().keySet() : null )); previous = current; + exclusive = true; } - sequences.add(new SequenceMetadata( + sequences.add(createSequenceMetaData( previous.getKey(), StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), previous.getValue(), endOffsets, - false + false, + exclusive ? previous.getValue().keySet() : null )); } else { - sequences.add(new SequenceMetadata( + sequences.add(createSequenceMetaData( 0, StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0), ioConfig.getStartPartitions().getPartitionSequenceNumberMap(), endOffsets, - false + false, + null )); } } @@ -452,6 +458,7 @@ public void run() Throwable caughtExceptionInner = null; initialOffsetsSnapshot.putAll(currOffsets); + exclusiveStartingPartitions.addAll(ioConfig.getExclusiveStartSequenceNumberPartitions()); try { while (stillReading) { @@ -496,6 +503,12 @@ public void run() SequenceMetadata sequenceToCheckpoint = null; for (OrderedPartitionableRecord record : records) { + // for Kafka, the end offsets are exclusive, so skip it + if (getRunnerType() == Type.KAFKA && createSequenceNumber(record.getSequenceNumber()).equals( + createSequenceNumber(endOffsets.get(record.getPartitionId())))) { + continue; + } + // for the first message we receive, check that we were given a message with a sequenceNumber that matches our // expected starting sequenceNumber if (!verifyInitialRecordAndSkipExclusiveParition(record, initialOffsetsSnapshot)) { @@ -515,7 +528,7 @@ public void run() // shard is closed currOffsets.put(record.getPartitionId(), record.getSequenceNumber()); } else if (createSequenceNumber(record.getSequenceNumber()).compareTo( - createSequenceNumber(endOffsets.get(record.getPartitionId()))) < 0) { + createSequenceNumber(endOffsets.get(record.getPartitionId()))) <= 0) { if (!record.getSequenceNumber().equals(currOffsets.get(record.getPartitionId()))) { @@ -618,16 +631,10 @@ public void onFailure(@ParametersAreNonnullByDefault Throwable t) handleParseException(e, record); } - - currOffsets.put( - record.getPartitionId(), - getNextSequenceNumber( - recordSupplier, - record.getStreamPartition(), - currOffsets.get(record.getPartitionId()) - ) - ); - + // in kafka, we can easily get the next offset by adding 1, but for kinesis, there's no way + // to get the next sequence number without having to make an expensive api call. So the behavior + // here for kafka is to +1 while for kinesis we simply save the current sequence numberkkkkk + currOffsets.put(record.getPartitionId(), getNextSequenceNumber(record.getSequenceNumber())); } if ((currOffsets.get(record.getPartitionId()).equals(endOffsets.get(record.getPartitionId())) @@ -1432,7 +1439,17 @@ public Response setEndOffsets( Preconditions.checkState(sequenceNumbers.size() > 0, "WTH?! No Sequences found to set end sequences"); final SequenceMetadata latestSequence = sequences.get(sequences.size() - 1); - if ((latestSequence.getStartOffsets().equals(sequenceNumbers) && !finish) || + // if a partition has not been read yet (contained in initialOffsetsSnapshot), then + // do not mark the starting sequence number as exclusive + Set exclusivePartitions = sequenceNumbers.keySet() + .stream() + .filter(x -> !initialOffsetsSnapshot.containsKey(x) + || ioConfig.getExclusiveStartSequenceNumberPartitions() + .contains(x)) + .collect(Collectors.toSet()); + + if ((latestSequence.getStartOffsets().equals(sequenceNumbers) && latestSequence.exclusiveStartPartitions.equals( + exclusivePartitions) && !finish) || (latestSequence.getEndOffsets().equals(sequenceNumbers) && finish)) { log.warn("Ignoring duplicate request, end sequences already set for sequences [%s]", sequenceNumbers); return Response.ok(sequenceNumbers).build(); @@ -1471,18 +1488,20 @@ public Response setEndOffsets( log.info("Updating endOffsets from [%s] to [%s]", endOffsets, sequenceNumbers); endOffsets.putAll(sequenceNumbers); } else { + exclusiveStartingPartitions.addAll(exclusivePartitions); + // create new sequence - final SequenceMetadata newSequence = new SequenceMetadata( + final SequenceMetadata newSequence = createSequenceMetaData( latestSequence.getSequenceId() + 1, StringUtils.format("%s_%d", ioConfig.getBaseSequenceName(), latestSequence.getSequenceId() + 1), sequenceNumbers, endOffsets, - false + false, + exclusivePartitions ); sequences.add(newSequence); initialOffsetsSnapshot.putAll(sequenceNumbers); } - persistSequences(); } catch (Exception e) { @@ -1633,29 +1652,41 @@ public DateTime getStartTime(@Context final HttpServletRequest req) return startTime; } - protected class SequenceMetadata + protected abstract SequenceMetadata createSequenceMetaData( + int sequenceId, + String sequenceName, + Map startOffsets, + Map endOffsets, + boolean checkpointed, + Set exclusiveStartingPartitions + ); + + protected abstract class SequenceMetadata { /** * Lock for accessing {@link #endOffsets} and {@link #checkpointed}. This lock is required because * {@link #setEndOffsets)} can be called by both the main thread and the HTTP thread. */ - private final ReentrantLock lock = new ReentrantLock(); + protected final ReentrantLock lock = new ReentrantLock(); private final int sequenceId; private final String sequenceName; - private final Map startOffsets; - private final Map endOffsets; + protected final Map startOffsets; + protected final Map endOffsets; + private final Set exclusiveStartPartitions; private final Set assignments; private final boolean sentinel; private boolean checkpointed; + @JsonCreator public SequenceMetadata( @JsonProperty("sequenceId") int sequenceId, @JsonProperty("sequenceName") String sequenceName, @JsonProperty("startOffsets") Map startOffsets, @JsonProperty("endOffsets") Map endOffsets, - @JsonProperty("checkpointed") boolean checkpointed + @JsonProperty("checkpointed") boolean checkpointed, + @JsonProperty("exclusiveStartParittions") Set exclusiveStartPartitions ) { Preconditions.checkNotNull(sequenceName); @@ -1668,6 +1699,15 @@ public SequenceMetadata( this.assignments = new HashSet<>(startOffsets.keySet()); this.checkpointed = checkpointed; this.sentinel = false; + this.exclusiveStartPartitions = exclusiveStartPartitions == null + ? Collections.emptySet() + : exclusiveStartPartitions; + } + + @JsonProperty + public Set getExclusiveStartPartitions() + { + return exclusiveStartPartitions; } @JsonProperty @@ -1748,12 +1788,14 @@ void updateAssignments(Map nextPartitionOffset) } } - boolean isOpen() + protected boolean isOpen() { return !assignments.isEmpty(); } - boolean canHandle(OrderedPartitionableRecord record) + protected abstract boolean canHandle(OrderedPartitionableRecord record); + + boolean acanHandle(OrderedPartitionableRecord record) { lock.lock(); try { @@ -1917,8 +1959,7 @@ private boolean verifyInitialRecordAndSkipExclusiveParition( } // check exclusive starting sequence - if (ioConfig.getExclusiveStartSequenceNumberPartitions() != null - && ioConfig.getExclusiveStartSequenceNumberPartitions().contains(record.getPartitionId())) { + if (getRunnerType() == Type.KINESIS && exclusiveStartingPartitions.contains(record.getPartitionId())) { log.info("Skipping starting sequenceNumber for partition [%s] marked exclusive", record.getPartitionId()); return false; @@ -1935,8 +1976,6 @@ protected abstract TreeMap> getCheckPo ) throws IOException; protected abstract SequenceType getNextSequenceNumber( - RecordSupplier recordSupplier, - StreamPartition partition, SequenceType sequenceNumber ); @@ -1956,4 +1995,12 @@ protected abstract SeekableStreamDataSourceMetadata ); protected abstract OrderedSequenceNumber createSequenceNumber(SequenceType sequenceNumber); + + protected abstract Type getRunnerType(); + + protected enum Type + { + KAFKA, + KINESIS + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index bb473af06971..f270198cb2ba 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -47,6 +47,7 @@ public class SeekableStreamPartitions { public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER"; + public static final String BEGINNING_SEQUENCE_NUMBER = "BEGINNING_SEQUENCE_NUMBER"; // stream/topic private final String stream; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 03d8a81d907e..891662864a4b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -33,7 +33,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; @@ -134,7 +133,6 @@ public abstract class SeekableStreamSupervisor private static final long MAX_RUN_FREQUENCY_MILLIS = 1000; private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120; private static final int MAX_INITIALIZATION_RETRIES = 20; - private static final CopyOnWriteArrayList EMPTY_LIST = Lists.newCopyOnWriteArrayList(); private final EmittingLogger log = new EmittingLogger(this.getClass()); @@ -1531,7 +1529,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) return sequence.equals(latestOffset); } ) && earliestConsistentSequenceId.compareAndSet(-1, sequenceCheckpoint.getKey())) || ( - pendingCompletionTaskGroups.getOrDefault(groupId, EMPTY_LIST).size() > 0 + pendingCompletionTaskGroups.getOrDefault(groupId, new CopyOnWriteArrayList<>()).size() > 0 && earliestConsistentSequenceId.compareAndSet(-1, taskCheckpoints.firstKey()))) { final SortedMap> latestCheckpoints = new TreeMap<>( taskCheckpoints.tailMap(earliestConsistentSequenceId.get()) @@ -1568,7 +1566,8 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) } if ((tasksToKill.size() > 0 && tasksToKill.size() == taskGroup.tasks.size()) || - (taskGroup.tasks.size() == 0 && pendingCompletionTaskGroups.getOrDefault(groupId, EMPTY_LIST).size() == 0)) { + (taskGroup.tasks.size() == 0 + && pendingCompletionTaskGroups.getOrDefault(groupId, new CopyOnWriteArrayList<>()).size() == 0)) { // killing all tasks or no task left in the group ? // clear state about the taskgroup so that get latest sequence information is fetched from metadata store log.warn("Clearing task group [%d] information as no valid tasks left the group", groupId); From ebde13b9f5ce1f1df7ca7d2b7998fec11ede45e6 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 27 Nov 2018 17:27:08 -0800 Subject: [PATCH 53/87] kinesis record supplier disable getPosition --- .../kafka/supervisor/KafkaSupervisor.java | 15 +++ .../kinesis/KinesisRecordSupplier.java | 45 +------- .../kinesis/supervisor/KinesisSupervisor.java | 9 ++ .../indexing/kinesis/KinesisIOConfigTest.java | 4 +- .../kinesis/KinesisRecordSupplierTest.java | 107 ------------------ .../supervisor/SeekableStreamSupervisor.java | 14 +-- 6 files changed, 35 insertions(+), 159 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 328071d7ef12..22eb21d4652e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -46,6 +46,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; @@ -88,6 +89,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor private final ServiceEmitter emitter; private final DruidMonitorSchedulerConfig monitorSchedulerConfig; + private volatile Map latestSequenceFromStream; private final KafkaSupervisorSpec spec; @@ -363,6 +365,19 @@ protected Runnable updateCurrentAndLatestOffsets() return super.updateCurrentAndLatestOffsets(); } + @Override + protected void updateLatestSequenceFromStream( + RecordSupplier recordSupplier, + Set> partitions + ) + { + latestSequenceFromStream = partitions.stream() + .collect(Collectors.toMap( + StreamPartition::getPartitionId, + recordSupplier::getPosition + )); + } + @Override @VisibleForTesting protected void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index e6651b4911a0..7622e137d09a 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -42,7 +42,6 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; @@ -57,7 +56,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -84,7 +82,6 @@ private class PartitionResource private final AmazonKinesis kinesisProxy; private final Object startLock = new Object(); - private String currIterator; // tracks current position private volatile String shardIterator; private volatile boolean started; private volatile boolean stopRequested; @@ -458,27 +455,6 @@ public List> poll(long timeout) .filter(x -> partitionResources.containsKey(x.getStreamPartition())) .collect(Collectors.toList()); - // update currIterator in each PartitionResource - // first, build a map of shardId -> latest record we've polled - // since polledRecords is ordered from earliest to latest, the final ordering of partitionSequenceMap - // is guranteed to be latest - Map> partitionSequenceMap = new HashMap<>(); - polledRecords.forEach(x -> partitionSequenceMap.put(x.getPartitionId(), x)); - - // then get the next shardIterator for each shard and update currIterator - partitionSequenceMap.forEach((shardId, record) -> partitionResources.get(record.getStreamPartition()).currIterator = - record.getSequenceNumber().equals(OrderedPartitionableRecord.END_OF_SHARD_MARKER) ? - null : - getKinesisProxy(shardId) - .getShardIterator( - record.getStream(), - record.getPartitionId(), - ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(), - record.getSequenceNumber() - ) - .getShardIterator()); - - return polledRecords; } catch (InterruptedException e) { @@ -506,24 +482,7 @@ public String getEarliestSequenceNumber(StreamPartition partition) @Override public String getPosition(StreamPartition partition) { - checkIfClosed(); - if (partitionResources.containsKey(partition)) { - String iter = partitionResources.get(partition).currIterator; - if (iter == null) { - log.warn( - "attempting to get position in shard[%s], stream[%s] with null sharditerator, is shard closed or did you forget to seek?", - partition.getPartitionId(), - partition.getStream() - ); - } - return getSequenceNumberInternal(partition, iter); - } else { - throw new IAE( - "attempting to get position in unassigned shard[%s], stream[%s]", - partition.getPartitionId(), - partition.getStream() - ); - } + throw new UnsupportedOperationException("getPosition is not supported in Kinesiss"); } @Override @@ -592,8 +551,6 @@ private void seekInternal(StreamPartition partition, String sequenceNumb sequenceNumber ).getShardIterator(); - resource.currIterator = resource.shardIterator; - resource.start(); checkPartitionsStarted = true; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 0d21097fa9a4..10f1c84c340e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -47,6 +47,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; @@ -302,6 +303,14 @@ protected Runnable updateCurrentAndLatestOffsets() return super.updateCurrentAndLatestOffsets(); } + @Override + protected void updateLatestSequenceFromStream( + RecordSupplier recordSupplier, Set> streamPartitions + ) + { + // do nothing + } + @Override @VisibleForTesting protected void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index 7557f639bed2..4a13de87ffea 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -33,6 +33,8 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.Collections; + public class KinesisIOConfigTest { private final ObjectMapper mapper; @@ -86,7 +88,7 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(config.getFetchDelayMillis(), 0); Assert.assertNull(config.getAwsAccessKeyId()); Assert.assertNull(config.getAwsSecretAccessKey()); - Assert.assertNull(config.getExclusiveStartSequenceNumberPartitions()); + Assert.assertEquals(Collections.emptySet(), config.getExclusiveStartSequenceNumberPartitions()); Assert.assertNull(config.getAwsAssumedRoleArn()); Assert.assertNull(config.getAwsExternalId()); Assert.assertFalse(config.isDeaggregate()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index 63a04f8b056f..3072454b39fd 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -554,113 +554,6 @@ public void testPollAfterSeek() Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 2), record3.getSequenceNumber()); } - @Test - public void testPosition() - throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException - { - AmazonKinesis kinesis = getKinesisClientInstance(); - List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); - - StreamPartition partition1 = StreamPartition.of(stream, shardId1); - Set> partitions = ImmutableSet.of( - partition1 - ); - - recordSupplier = new KinesisRecordSupplier( - LocalstackTestRunner.getEndpointKinesis(), - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, - 100, - 0, - 2, - null, - null, - false, - 100, - 5000, - 5000, - 60000, - 1 - ); - - recordSupplier.assign(partitions); - recordSupplier.seekToEarliest(partitions); - - Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 0), recordSupplier.getPosition(partition1)); - - recordSupplier.seek(partition1, getSequenceNumber(insertDataResults, shardId1, 3)); - Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 3), recordSupplier.getPosition(partition1)); - - Assert.assertEquals(new OrderedPartitionableRecord<>( - stream, - shardId1, - getSequenceNumber(insertDataResults, shardId1, 3), - null - ), recordSupplier.poll(poll_timeout_millis).get(0)); - - Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 4), recordSupplier.getPosition(partition1)); - - Assert.assertEquals( - getSequenceNumber(insertDataResults, shardId1, 4), - recordSupplier.poll(poll_timeout_millis).get(0).getSequenceNumber() - ); - - Assert.assertEquals( - getSequenceNumber(insertDataResults, shardId1, 5), - recordSupplier.poll(poll_timeout_millis).get(0).getSequenceNumber() - ); - - Assert.assertEquals( - getSequenceNumber(insertDataResults, shardId1, 6), - recordSupplier.poll(poll_timeout_millis).get(0).getSequenceNumber() - ); - - Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 7), recordSupplier.getPosition(partition1)); - } - - @Test - public void testPositionAfterPollBatch() - throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException - { - AmazonKinesis kinesis = getKinesisClientInstance(); - List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); - - StreamPartition partition1 = StreamPartition.of(stream, shardId1); - Set> partitions = ImmutableSet.of( - partition1 - ); - - recordSupplier = new KinesisRecordSupplier( - LocalstackTestRunner.getEndpointKinesis(), - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, - 100, - 0, - 2, - null, - null, - false, - 100, - 5000, - 5000, - 60000, - 3 - ); - - recordSupplier.assign(partitions); - recordSupplier.seekToEarliest(partitions); - - Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 0), recordSupplier.getPosition(partition1)); - - int i = 0; - while (recordSupplier.bufferSize() < 3 && i++ < pollRetry) { - Thread.sleep(100); - } - - Assert.assertEquals(3, recordSupplier.poll(poll_timeout_millis).size()); - - Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 3), recordSupplier.getPosition(partition1)); - } @Test public void testDeaggregate() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 891662864a4b..dfe004db48c8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -453,7 +453,6 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) protected final ObjectMapper sortingMapper; protected final List partitionIds = new CopyOnWriteArrayList<>(); - protected volatile Map latestSequenceFromStream; protected volatile DateTime sequenceLastUpdated; @@ -2542,7 +2541,7 @@ private void updateCurrentOffsets() throws InterruptedException, ExecutionExcept Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); } - private void updateLatestOffsetsFromStream() + protected void updateLatestOffsetsFromStream() { synchronized (recordSupplierLock) { Set partitionIds = null; @@ -2562,15 +2561,16 @@ private void updateLatestOffsetsFromStream() recordSupplier.assign(partitions); recordSupplier.seekToLatest(partitions); - latestSequenceFromStream = partitions.stream() - .collect(Collectors.toMap( - StreamPartition::getPartitionId, - x -> recordSupplier.getPosition(x) - )); + updateLatestSequenceFromStream(recordSupplier, partitions); } } + protected abstract void updateLatestSequenceFromStream( + RecordSupplier recordSupplier, + Set> partitions + ); + protected Map getHighestCurrentOffsets() { return activelyReadingTaskGroups From cc72cc9b52ba921cbcd48c93ed63b33db3aec54d Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 27 Nov 2018 17:45:39 -0800 Subject: [PATCH 54/87] pr comments --- .../druid/indexing/kafka/KafkaIndexTask.java | 1 + .../kafka/LegacyKafkaIndexTaskRunner.java | 18 +++++++++--------- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 54511b7b76b5..569e780be744 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -141,6 +141,7 @@ KafkaConsumer newConsumer() } } + @Deprecated static void assignPartitions( final KafkaConsumer consumer, final String topic, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index da53a50d16ce..3f2918563b26 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -629,7 +629,7 @@ protected List> getRecords( RecordSupplier recordSupplier, TaskToolbox toolbox ) { - return null; + throw new UnsupportedOperationException(); } private Set assignPartitionsAndSeekToNext(KafkaConsumer consumer, String topic) @@ -703,7 +703,7 @@ protected SeekableStreamPartitions createSeekableStreamPartitions Object obeject ) { - return null; + throw new UnsupportedOperationException(); } private void possiblyResetOffsetsOrWait( @@ -734,7 +734,7 @@ private void possiblyResetOffsetsOrWait( } if (doReset) { - sendResetRequestAndAndWait(resetPartitions, taskToolbox); + sendResetRequestAndWaitLegacy(resetPartitions, taskToolbox); } else { log.warn("Retrying in %dms", task.getPollRetryMs()); pollRetryLock.lockInterruptibly(); @@ -750,7 +750,7 @@ private void possiblyResetOffsetsOrWait( } } - private void sendResetRequestAndAndWait(Map outOfRangePartitions, TaskToolbox taskToolbox) + private void sendResetRequestAndWaitLegacy(Map outOfRangePartitions, TaskToolbox taskToolbox) throws IOException { Map partitionOffsetMap = new HashMap<>(); @@ -790,13 +790,13 @@ protected TreeMap> getCheckPointsFromContext( SeekableStreamIndexTask task ) { - return null; + throw new UnsupportedOperationException(); } @Override protected Long getNextSequenceNumber(Long sequenceNumber) { - return null; + throw new UnsupportedOperationException(); } private void handleParseException(ParseException pe, ConsumerRecord record) @@ -1199,13 +1199,13 @@ protected SeekableStreamDataSourceMetadata createDataSourceMetada SeekableStreamPartitions partitions ) { - return null; + throw new UnsupportedOperationException(); } @Override protected OrderedSequenceNumber createSequenceNumber(Long sequenceNumber) { - return null; + throw new UnsupportedOperationException(); } @Override @@ -1234,6 +1234,6 @@ protected SequenceMetadata createSequenceMetaData( Set exclusiveStartPartitions ) { - return null; + throw new UnsupportedOperationException(); } } From e35ab1659bc8b03dc59a93c4dc5e1b915dcaa23b Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Fri, 30 Nov 2018 13:07:57 -0800 Subject: [PATCH 55/87] mock for kinesis tests and remove docker dependency for unit tests --- .travis.yml | 230 ++--- ...ementalPublishingKafkaIndexTaskRunner.java | 2 + .../druid/indexing/kafka/KafkaIndexTask.java | 45 +- .../kafka/LegacyKafkaIndexTaskRunner.java | 2 + .../kafka/supervisor/KafkaSupervisor.java | 6 +- .../indexing/kafka/KafkaIndexTaskTest.java | 8 +- .../kafka/supervisor/KafkaSupervisorTest.java | 13 +- .../indexing/kinesis/KinesisIndexTask.java | 55 +- .../kinesis/KinesisIndexTaskRunner.java | 2 + .../kinesis/KinesisRecordSupplier.java | 119 +-- .../kinesis/supervisor/KinesisSupervisor.java | 18 +- .../kinesis/KinesisIndexTaskTest.java | 951 +++++++++++------- .../kinesis/KinesisRecordSupplierTest.java | 687 ++++++------- .../supervisor/KinesisSupervisorTest.java | 922 ++++++++++------- .../SeekableStreamIndexTask.java | 10 +- .../SeekableStreamIndexTaskRunner.java | 8 +- .../common/OrderedPartitionableRecord.java | 1 + .../supervisor/SeekableStreamSupervisor.java | 17 +- 18 files changed, 1765 insertions(+), 1331 deletions(-) diff --git a/.travis.yml b/.travis.yml index 084aad442f82..c7d3d7da4544 100644 --- a/.travis.yml +++ b/.travis.yml @@ -23,138 +23,130 @@ dist: trusty group: deprecated-2017Q4 jdk: - - oraclejdk8 +- oraclejdk8 cache: directories: - - $HOME/.m2 + - $HOME/.m2 matrix: include: - # license checks - - env: - - NAME="license checks" - install: true - script: MAVEN_OPTS='-Xmx3000m' mvn clean verify -Prat -DskipTests -B -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn + # license checks + - env: + - NAME="license checks" + install: true + script: MAVEN_OPTS='-Xmx3000m' mvn clean verify -Prat -DskipTests -B -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn - # strict compilation - - env: - - NAME="strict compilation" - install: true - # Strict compilation requires more than 2 GB - script: MAVEN_OPTS='-Xmx3000m' mvn clean -Pstrict -pl '!benchmarks' compile test-compile -B --fail-at-end + # strict compilation + - env: + - NAME="strict compilation" + install: true + # Strict compilation requires more than 2 GB + script: MAVEN_OPTS='-Xmx3000m' mvn clean -Pstrict -pl '!benchmarks' compile test-compile -B --fail-at-end - # processing module test - - env: - - NAME="processing module test" - install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B - before_script: unset _JAVA_OPTIONS - script: - # Set MAVEN_OPTS for Surefire launcher - - MAVEN_OPTS='-Xmx512m' mvn test -B -pl processing - - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" - - free -m + # processing module test + - env: + - NAME="processing module test" + install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B + before_script: unset _JAVA_OPTIONS + script: + # Set MAVEN_OPTS for Surefire launcher + - MAVEN_OPTS='-Xmx512m' mvn test -B -pl processing + - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" + - free -m - # processing module tests with SQL Compatibility enabled - - env: - - NAME="processing module test with SQL Compatibility" - install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B - before_script: unset _JAVA_OPTIONS - script: - # Set MAVEN_OPTS for Surefire launcher - - MAVEN_OPTS='-Xmx512m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl processing - - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" - - free -m + # processing module tests with SQL Compatibility enabled + - env: + - NAME="processing module test with SQL Compatibility" + install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B + before_script: unset _JAVA_OPTIONS + script: + # Set MAVEN_OPTS for Surefire launcher + - MAVEN_OPTS='-Xmx512m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl processing + - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" + - free -m - # server module test - - env: - - NAME="server module test" - install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B - before_script: unset _JAVA_OPTIONS - script: - # Set MAVEN_OPTS for Surefire launcher - - MAVEN_OPTS='-Xmx512m' mvn test -B -pl server + # server module test + - env: + - NAME="server module test" + install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B + before_script: unset _JAVA_OPTIONS + script: + # Set MAVEN_OPTS for Surefire launcher + - MAVEN_OPTS='-Xmx512m' mvn test -B -pl server - # server module test with SQL Compatibility enabled - - env: - - NAME="server module test with SQL Compatibility enabled" - install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B - before_script: unset _JAVA_OPTIONS - script: - # Set MAVEN_OPTS for Surefire launcher - - MAVEN_OPTS='-Xmx512m' mvn test -B -pl server -Ddruid.generic.useDefaultValueForNull=false + # server module test with SQL Compatibility enabled + - env: + - NAME="server module test with SQL Compatibility enabled" + install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B + before_script: unset _JAVA_OPTIONS + script: + # Set MAVEN_OPTS for Surefire launcher + - MAVEN_OPTS='-Xmx512m' mvn test -B -pl server -Ddruid.generic.useDefaultValueForNull=false - # other modules test - - sudo: required - services: - - docker - env: - - NAME="other modules test" - - AWS_REGION=us-east-1 # set a aws region for unit tests - - DOCKER_IP=127.0.0.1 - install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B - before_script: unset _JAVA_OPTIONS - script: - # Set MAVEN_OPTS for Surefire launcher - - MAVEN_OPTS='-Xmx512m' mvn test -B -pl '!processing,!server' - - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" - - free -m + # other modules test + - env: + - NAME="other modules test" + - AWS_REGION=us-east-1 # set a aws region for unit tests + install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B + before_script: unset _JAVA_OPTIONS + script: + # Set MAVEN_OPTS for Surefire launcher + - MAVEN_OPTS='-Xmx512m' mvn test -B -pl '!processing,!server' + - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" + - free -m - # other modules test with SQL Compatibility enabled - - sudo: required - services: - - docker - env: - - NAME="other modules test with SQL Compatibility" - - AWS_REGION=us-east-1 # set a aws region for unit tests - - DOCKER_IP=127.0.0.1 - install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B - before_script: unset _JAVA_OPTIONS - script: - # Set MAVEN_OPTS for Surefire launcher - - MAVEN_OPTS='-Xmx512m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl '!processing,!server' - - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" - - free -m + # other modules test with SQL Compatibility enabled + - env: + - NAME="other modules test with SQL Compatibility" + - AWS_REGION=us-east-1 # set a aws region for unit tests + install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B + before_script: unset _JAVA_OPTIONS + script: + # Set MAVEN_OPTS for Surefire launcher + - MAVEN_OPTS='-Xmx512m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl '!processing,!server' + - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" + - free -m - # run integration tests - - sudo: required - services: - - docker - env: - - NAME="integration test part 1" - - DOCKER_IP=127.0.0.1 - install: - # Only errors will be shown with the -q option. This is to avoid generating too many logs which make travis build failed. - - mvn install -q -ff -DskipTests -B - script: - - $TRAVIS_BUILD_DIR/ci/travis_script_integration.sh - after_failure: - - for v in ~/shared/logs/*.log ; do - echo $v logtail ======================== ; tail -100 $v ; - done - - for v in broker middlemanager overlord router coordinator historical ; do - echo $v dmesg ======================== ; - docker exec -it druid-$v sh -c 'dmesg | tail -3' ; - done + # run integration tests + - sudo: required + services: + - docker + env: + - NAME="integration test part 1" + - DOCKER_IP=127.0.0.1 + install: + # Only errors will be shown with the -q option. This is to avoid generating too many logs which make travis build failed. + - mvn install -q -ff -DskipTests -B + script: + - $TRAVIS_BUILD_DIR/ci/travis_script_integration.sh + after_failure: + - for v in ~/shared/logs/*.log ; do + echo $v logtail ======================== ; tail -100 $v ; + done + - for v in broker middlemanager overlord router coordinator historical ; do + echo $v dmesg ======================== ; + docker exec -it druid-$v sh -c 'dmesg | tail -3' ; + done - # run integration tests - - sudo: required - services: - - docker - env: - - NAME="integration test part 2" - - DOCKER_IP=127.0.0.1 - install: - # Only errors will be shown with the -q option. This is to avoid generating too many logs which make travis build failed. - - mvn install -q -ff -DskipTests -B - script: - - $TRAVIS_BUILD_DIR/ci/travis_script_integration_part2.sh - after_failure: - - for v in ~/shared/logs/*.log ; do - echo $v logtail ======================== ; tail -100 $v ; - done - - for v in broker middlemanager overlord router coordinator historical ; do - echo $v dmesg ======================== ; - docker exec -it druid-$v sh -c 'dmesg | tail -3' ; - done + # run integration tests + - sudo: required + services: + - docker + env: + - NAME="integration test part 2" + - DOCKER_IP=127.0.0.1 + install: + # Only errors will be shown with the -q option. This is to avoid generating too many logs which make travis build failed. + - mvn install -q -ff -DskipTests -B + script: + - $TRAVIS_BUILD_DIR/ci/travis_script_integration_part2.sh + after_failure: + - for v in ~/shared/logs/*.log ; do + echo $v logtail ======================== ; tail -100 $v ; + done + - for v in broker middlemanager overlord router coordinator historical ; do + echo $v dmesg ======================== ; + docker exec -it druid-$v sh -c 'dmesg | tail -3' ; + done diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 30614eb06aa1..02cea4f1a195 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -67,6 +67,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd public IncrementalPublishingKafkaIndexTaskRunner( KafkaIndexTask task, + KafkaRecordSupplier recordSupplier, InputRowParser parser, AuthorizerMapper authorizerMapper, Optional chatHandlerProvider, @@ -76,6 +77,7 @@ public IncrementalPublishingKafkaIndexTaskRunner( { super( task, + recordSupplier, parser, authorizerMapper, chatHandlerProvider, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 569e780be744..2ec6db18b7ad 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -28,7 +28,6 @@ import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; -import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.indexing.DataSchema; @@ -70,7 +69,7 @@ public KafkaIndexTask( @JacksonInject AuthorizerMapper authorizerMapper, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, @JacksonInject ObjectMapper configMapper - ) + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { super( id, @@ -94,26 +93,6 @@ long getPollRetryMs() return pollRetryMs; } - @Override - protected RecordSupplier newRecordSupplier() - { - ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); - try { - Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); - - final Map props = new HashMap<>(ioConfig.getConsumerProperties()); - - props.put("auto.offset.reset", "none"); - props.put("key.deserializer", ByteArrayDeserializer.class.getName()); - props.put("value.deserializer", ByteArrayDeserializer.class.getName()); - - return new KafkaRecordSupplier(props, configMapper); - } - finally { - Thread.currentThread().setContextClassLoader(currCtxCl); - } - } - @Deprecated KafkaConsumer newConsumer() { @@ -162,6 +141,7 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() && ((boolean) context.get(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED))) { return new IncrementalPublishingKafkaIndexTaskRunner( this, + newTaskRecordSupplier(), parser, authorizerMapper, chatHandlerProvider, @@ -171,6 +151,7 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() } else { return new LegacyKafkaIndexTaskRunner( this, + newTaskRecordSupplier(), parser, authorizerMapper, chatHandlerProvider, @@ -180,6 +161,26 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() } } + @VisibleForTesting + protected KafkaRecordSupplier newTaskRecordSupplier() + { + ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + + final Map props = new HashMap<>(((KafkaIOConfig) super.ioConfig).getConsumerProperties()); + + props.put("auto.offset.reset", "none"); + props.put("key.deserializer", ByteArrayDeserializer.class.getName()); + props.put("value.deserializer", ByteArrayDeserializer.class.getName()); + + return new KafkaRecordSupplier(props, configMapper); + } + finally { + Thread.currentThread().setContextClassLoader(currCtxCl); + } + } + @Override @JsonProperty public KafkaTuningConfig getTuningConfig() diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 3f2918563b26..ed100828d675 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -181,6 +181,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner parser, AuthorizerMapper authorizerMapper, Optional chatHandlerProvider, @@ -190,6 +191,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner> createIndexTasks( SeekableStreamIOConfig taskIoConfig, SeekableStreamTuningConfig taskTuningConfig, RowIngestionMetersFactory rowIngestionMetersFactory - ) throws JsonProcessingException + ) throws JsonProcessingException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException { final String checkpoints = sortingMapper.writerWithType(new TypeReference>>() { @@ -353,7 +352,8 @@ protected Long getEndOfPartitionMarker() @Override @VisibleForTesting protected void runInternal() - throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException + throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException, NoSuchMethodException, + IllegalAccessException, ClassNotFoundException { super.runInternal(); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index f9165694e995..d51f116674f9 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -2146,7 +2146,7 @@ public boolean apply(TaskLock lock) private KafkaIndexTask createTask( final String taskId, final KafkaIOConfig ioConfig - ) + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { return createTask(taskId, DATA_SCHEMA, ioConfig); } @@ -2155,7 +2155,7 @@ private KafkaIndexTask createTask( final String taskId, final KafkaIOConfig ioConfig, final Map context - ) + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { return createTask(taskId, DATA_SCHEMA, ioConfig, context); } @@ -2164,7 +2164,7 @@ private KafkaIndexTask createTask( final String taskId, final DataSchema dataSchema, final KafkaIOConfig ioConfig - ) + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( 1000, @@ -2213,7 +2213,7 @@ private KafkaIndexTask createTask( final DataSchema dataSchema, final KafkaIOConfig ioConfig, final Map context - ) + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( 1000, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index e8574fdf4d15..b4833e6c501d 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -2161,7 +2161,8 @@ public void testNoDataIngestionTasks() throws Exception @Test(timeout = 60_000L) public void testCheckpointForInactiveTaskGroup() - throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException + throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException, NoSuchMethodException, + IllegalAccessException, ClassNotFoundException { supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); //not adding any events @@ -2263,7 +2264,8 @@ public void testCheckpointForInactiveTaskGroup() } @Test(timeout = 60_000L) - public void testCheckpointForUnknownTaskGroup() throws InterruptedException + public void testCheckpointForUnknownTaskGroup() + throws InterruptedException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException { supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); //not adding any events @@ -2342,7 +2344,8 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException @Test(timeout = 60_000L) public void testCheckpointWithNullTaskGroupId() - throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException + throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException, NoSuchMethodException, + IllegalAccessException, ClassNotFoundException { supervisor = getSupervisor(1, 3, true, "PT1S", null, null, false); //not adding any events @@ -2889,7 +2892,7 @@ private KafkaIndexTask createKafkaIndexTask( SeekableStreamPartitions endPartitions, DateTime minimumMessageTime, DateTime maximumMessageTime - ) + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { return new KafkaIndexTask( id, @@ -2901,7 +2904,7 @@ private KafkaIndexTask createKafkaIndexTask( "sequenceName-" + taskGroupId, startPartitions, endPartitions, - ImmutableMap.of(), + ImmutableMap.of("bootstrap.servers", kafkaHost), true, minimumMessageTime, maximumMessageTime, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 3c2d7b559bde..1005be20c385 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -22,11 +22,11 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; -import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; @@ -35,9 +35,6 @@ public class KinesisIndexTask extends SeekableStreamIndexTask { - private final KinesisIOConfig ioConfig; - private final KinesisTuningConfig tuningConfig; - @JsonCreator public KinesisIndexTask( @JsonProperty("id") String id, @@ -49,7 +46,7 @@ public KinesisIndexTask( @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory - ) + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { super( id, @@ -63,28 +60,45 @@ public KinesisIndexTask( rowIngestionMetersFactory, "index_kinesis" ); - - this.ioConfig = ioConfig; - this.tuningConfig = tuningConfig; } + @Override - protected RecordSupplier newRecordSupplier() + protected SeekableStreamIndexTaskRunner createTaskRunner() + throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + { + return new KinesisIndexTaskRunner( + this, + newTaskRecordSupplier(), + parser, + authorizerMapper, + chatHandlerProvider, + savedParseExceptions, + rowIngestionMetersFactory + ); + } + + @VisibleForTesting + protected KinesisRecordSupplier newTaskRecordSupplier() throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException { + KinesisIOConfig ioConfig = ((KinesisIOConfig) super.ioConfig); + KinesisTuningConfig tuningConfig = ((KinesisTuningConfig) super.tuningConfig); int fetchThreads = tuningConfig.getFetchThreads() != null ? tuningConfig.getFetchThreads() : Math.max(1, ioConfig.getStartPartitions().getPartitionSequenceNumberMap().size()); return new KinesisRecordSupplier( - ioConfig.getEndpoint(), - ioConfig.getAwsAccessKeyId(), - ioConfig.getAwsSecretAccessKey(), + KinesisRecordSupplier.getAmazonKinesisClient( + ioConfig.getEndpoint(), + ioConfig.getAwsAccessKeyId(), + ioConfig.getAwsSecretAccessKey(), + ioConfig.getAwsAssumedRoleArn(), + ioConfig.getAwsExternalId() + ), ioConfig.getRecordsPerFetch(), ioConfig.getFetchDelayMillis(), fetchThreads, - ioConfig.getAwsAssumedRoleArn(), - ioConfig.getAwsExternalId(), ioConfig.isDeaggregate(), tuningConfig.getRecordBufferSize(), tuningConfig.getRecordBufferOfferTimeout(), @@ -94,19 +108,6 @@ protected RecordSupplier newRecordSupplier() ); } - @Override - protected SeekableStreamIndexTaskRunner createTaskRunner() - { - return new KinesisIndexTaskRunner( - this, - parser, - authorizerMapper, - chatHandlerProvider, - savedParseExceptions, - rowIngestionMetersFactory - ); - } - @Override @JsonProperty("ioConfig") public KinesisIOConfig getIOConfig() diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index cb9d1a12f003..195edd066fef 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -56,6 +56,7 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner parser, AuthorizerMapper authorizerMapper, Optional chatHandlerProvider, @@ -65,6 +66,7 @@ public KinesisIndexTaskRunner( { super( task, + recordSupplier, parser, authorizerMapper, chatHandlerProvider, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 7622e137d09a..6135953e59b9 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -35,6 +35,7 @@ import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder; import com.amazonaws.util.AwsHostNameUtils; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Queues; import org.apache.druid.common.aws.AWSCredentialsUtils; @@ -79,23 +80,20 @@ public class KinesisRecordSupplier implements RecordSupplier private class PartitionResource { private final StreamPartition streamPartition; - private final AmazonKinesis kinesisProxy; private final Object startLock = new Object(); private volatile String shardIterator; private volatile boolean started; private volatile boolean stopRequested; - public PartitionResource( - StreamPartition streamPartition, - AmazonKinesis kinesisProxy + PartitionResource( + StreamPartition streamPartition ) { this.streamPartition = streamPartition; - this.kinesisProxy = kinesisProxy; } - public void start() + void start() { synchronized (startLock) { if (started) { @@ -160,7 +158,7 @@ private Runnable getRecordRunnable() return; } - GetRecordsResult recordsResult = kinesisProxy.getRecords(new GetRecordsRequest().withShardIterator( + GetRecordsResult recordsResult = kinesis.getRecords(new GetRecordsRequest().withShardIterator( shardIterator).withLimit(recordsPerFetch)); // list will come back empty if there are no records @@ -213,7 +211,7 @@ private Runnable getRecordRunnable() recordBufferFullWait ); - shardIterator = kinesisProxy.getShardIterator( + shardIterator = kinesis.getShardIterator( record.getStream(), record.getPartitionId(), ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), @@ -264,6 +262,8 @@ private void rescheduleRunnable(long delayMillis) private final MethodHandle deaggregateHandle; private final MethodHandle getDataHandle; + private final AmazonKinesis kinesis; + private final int recordsPerFetch; private final int fetchDelayMillis; private final boolean deaggregate; @@ -274,10 +274,8 @@ private void rescheduleRunnable(long delayMillis) private final int fetchThreads; private final int recordBufferSize; - private final AmazonKinesisClientBuilder kinesisBuilder; private ScheduledExecutorService scheduledExec; - private final Map kinesisProxies = new ConcurrentHashMap<>(); private final Map, PartitionResource> partitionResources = new ConcurrentHashMap<>(); private BlockingQueue> records; @@ -285,14 +283,10 @@ private void rescheduleRunnable(long delayMillis) private volatile boolean closed = false; public KinesisRecordSupplier( - String endpoint, - String awsAccessKeyId, - String awsSecretAccessKey, + AmazonKinesis amazonKinesis, int recordsPerFetch, int fetchDelayMillis, int fetchThreads, - String awsAssumedRoleArn, - String awsExternalId, boolean deaggregate, int recordBufferSize, int recordBufferOfferTimeout, @@ -301,6 +295,8 @@ public KinesisRecordSupplier( int maxRecordsPerPoll ) throws ClassNotFoundException, IllegalAccessException, NoSuchMethodException { + Preconditions.checkNotNull(amazonKinesis); + this.kinesis = amazonKinesis; this.recordsPerFetch = recordsPerFetch; this.fetchDelayMillis = fetchDelayMillis; this.deaggregate = deaggregate; @@ -333,6 +329,29 @@ public KinesisRecordSupplier( getDataHandle = null; } + + log.info( + "Creating fetch thread pool of size [%d] (Runtime.availableProcessors=%d)", + fetchThreads, + Runtime.getRuntime().availableProcessors() + ); + + scheduledExec = Executors.newScheduledThreadPool( + fetchThreads, + Execs.makeThreadFactory("KinesisRecordSupplier-Worker-%d") + ); + + records = new LinkedBlockingQueue<>(recordBufferSize); + } + + public static AmazonKinesis getAmazonKinesisClient( + String endpoint, + String awsAccessKeyId, + String awsSecretAccessKey, + String awsAssumedRoleArn, + String awsExternalId + ) + { AWSCredentialsProvider awsCredentialsProvider = AWSCredentialsUtils.defaultAWSCredentialsProviderChain( new ConstructibleAWSCredentialsConfig(awsAccessKeyId, awsSecretAccessKey) ); @@ -352,32 +371,30 @@ public KinesisRecordSupplier( awsCredentialsProvider = builder.build(); } - kinesisBuilder = AmazonKinesisClientBuilder.standard() - .withCredentials(awsCredentialsProvider) - .withClientConfiguration(new ClientConfiguration()) - .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration( - endpoint, - AwsHostNameUtils.parseRegion( - endpoint, - null - ) - )); - log.info( - "Creating fetch thread pool of size [%d] (Runtime.availableProcessors=%d)", - fetchThreads, - Runtime.getRuntime().availableProcessors() - ); + return AmazonKinesisClientBuilder.standard() + .withCredentials(awsCredentialsProvider) + .withClientConfiguration(new ClientConfiguration()) + .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration( + endpoint, + AwsHostNameUtils.parseRegion( + endpoint, + null + ) + )).build(); + } - scheduledExec = Executors.newScheduledThreadPool( - fetchThreads, - Execs.makeThreadFactory("KinesisRecordSupplier-Worker-%d") - ); - records = new LinkedBlockingQueue<>(recordBufferSize); + @VisibleForTesting + public void start() + { + checkIfClosed(); + if (checkPartitionsStarted) { + partitionResources.values().forEach(PartitionResource::start); + checkPartitionsStarted = false; + } } - @Override public void assign(Set> collection) { @@ -386,7 +403,7 @@ public void assign(Set> collection) collection.forEach( streamPartition -> partitionResources.putIfAbsent( streamPartition, - new PartitionResource(streamPartition, getKinesisProxy(streamPartition.getStream())) + new PartitionResource(streamPartition) ) ); @@ -489,11 +506,11 @@ public String getPosition(StreamPartition partition) public Set getPartitionIds(String stream) { checkIfClosed(); - return getKinesisProxy(stream).describeStream(stream) - .getStreamDescription() - .getShards() - .stream() - .map(Shard::getShardId).collect(Collectors.toSet()); + return kinesis.describeStream(stream) + .getStreamDescription() + .getShards() + .stream() + .map(Shard::getShardId).collect(Collectors.toSet()); } @Override @@ -519,16 +536,6 @@ public void close() this.closed = true; } - private AmazonKinesis getKinesisProxy(String streamName) - { - if (!kinesisProxies.containsKey(streamName)) { - AmazonKinesis kinesis = kinesisBuilder.build(); - kinesisProxies.put(streamName, kinesis); - } - - return kinesisProxies.get(streamName); - } - private void seekInternal(StreamPartition partition, String sequenceNumber, ShardIteratorType iteratorEnum) { PartitionResource resource = partitionResources.get(partition); @@ -542,8 +549,6 @@ private void seekInternal(StreamPartition partition, String sequenceNumb sequenceNumber != null ? sequenceNumber : iteratorEnum.toString() ); - AmazonKinesis kinesis = getKinesisProxy(partition.getStream()); - resource.shardIterator = kinesis.getShardIterator( partition.getStream(), partition.getPartitionId(), @@ -551,8 +556,6 @@ private void seekInternal(StreamPartition partition, String sequenceNumb sequenceNumber ).getShardIterator(); - resource.start(); - checkPartitionsStarted = true; } @@ -590,7 +593,7 @@ private void filterBufferAndResetFetchRunnable(Set> part private String getSequenceNumberInternal(StreamPartition partition, ShardIteratorType iteratorEnum) { - AmazonKinesis kinesis = getKinesisProxy(partition.getStream()); + String shardIterator = null; try { shardIterator = kinesis.getShardIterator( @@ -609,8 +612,6 @@ private String getSequenceNumberInternal(StreamPartition partition, Shar private String getSequenceNumberInternal(StreamPartition partition, String shardIterator) { long timeoutMillis = System.currentTimeMillis() + fetchSequenceNumberTimeout; - AmazonKinesis kinesis = getKinesisProxy(partition.getStream()); - while (shardIterator != null && System.currentTimeMillis() < timeoutMillis) { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 10f1c84c340e..005bf490e0a9 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -146,7 +146,7 @@ protected List> createIndexTasks( SeekableStreamIOConfig taskIoConfig, SeekableStreamTuningConfig taskTuningConfig, RowIngestionMetersFactory rowIngestionMetersFactory - ) throws JsonProcessingException + ) throws JsonProcessingException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException { final String checkpoints = sortingMapper.writerFor(new TypeReference>>() { @@ -180,6 +180,7 @@ protected List> createIndexTasks( return taskList; } + @Override protected RecordSupplier setupRecordSupplier() throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException @@ -188,14 +189,16 @@ protected RecordSupplier setupRecordSupplier() KinesisTuningConfig taskTuningConfig = spec.getTuningConfig(); return new KinesisRecordSupplier( - ioConfig.getEndpoint(), - ioConfig.getAwsAccessKeyId(), - ioConfig.getAwsSecretAccessKey(), + KinesisRecordSupplier.getAmazonKinesisClient( + ioConfig.getEndpoint(), + ioConfig.getAwsAccessKeyId(), + ioConfig.getAwsSecretAccessKey(), + ioConfig.getAwsAssumedRoleArn(), + ioConfig.getAwsExternalId() + ), ioConfig.getRecordsPerFetch(), ioConfig.getFetchDelayMillis(), 1, - ioConfig.getAwsAssumedRoleArn(), - ioConfig.getAwsExternalId(), ioConfig.isDeaggregate(), taskTuningConfig.getRecordBufferSize(), taskTuningConfig.getRecordBufferOfferTimeout(), @@ -291,7 +294,8 @@ protected OrderedSequenceNumber makeSequenceNumber( @Override @VisibleForTesting protected void runInternal() - throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException + throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException, NoSuchMethodException, + IllegalAccessException, ClassNotFoundException { super.runInternal(); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 12490f2d0433..c35b4e92121d 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -21,14 +21,10 @@ import cloud.localstack.LocalstackTestRunner; import cloud.localstack.TestUtils; -import cloud.localstack.docker.LocalstackDockerTestRunner; -import cloud.localstack.docker.annotation.LocalstackDockerProperties; -import com.amazonaws.http.SdkHttpMetadata; -import com.amazonaws.services.kinesis.AmazonKinesis; -import com.amazonaws.services.kinesis.model.PutRecordsRequest; -import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry; -import com.amazonaws.services.kinesis.model.PutRecordsResult; -import com.amazonaws.services.kinesis.model.PutRecordsResultEntry; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; @@ -77,13 +73,14 @@ import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskTest; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.MetadataTaskStorage; import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; @@ -141,6 +138,7 @@ import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; +import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.transform.ExpressionTransform; @@ -148,8 +146,10 @@ import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; +import org.easymock.EasyMockSupport; import org.joda.time.Interval; import org.joda.time.Period; import org.junit.After; @@ -160,110 +160,68 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; -import java.util.Comparator; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Random; import java.util.Set; import java.util.TreeMap; import java.util.concurrent.Executor; import java.util.concurrent.Executors; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; -@RunWith(LocalstackDockerTestRunner.class) -@LocalstackDockerProperties(services = {"kinesis"}) -public class KinesisIndexTaskTest -{ - static { - TestUtils.setEnv("AWS_CBOR_DISABLE", "1"); - if (LocalstackTestRunner.useSSL()) { - TestUtils.disableSslCertChecking(); - } - } +import static org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; +import static org.easymock.EasyMock.anyLong; +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.anyString; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.expectLastCall; +import static org.easymock.EasyMock.reset; +public class KinesisIndexTaskTest extends EasyMockSupport +{ private static final Logger log = new Logger(KinesisIndexTaskTest.class); private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); // private static final long POLL_RETRY_MS = 100; - private static int streamPosFix = 0; - private static String shardId1 = "shardId-000000000001"; - private static String shardId0 = "shardId-000000000000"; - private static final List records = ImmutableList.of( - generateRequestEntry( - "1", - JB("2008", "a", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2009", "b", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2010", "c", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2011", "d", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2011", "e", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( + private static String stream = "stream"; + private static String shardId1 = "1"; + private static String shardId0 = "0"; + private static KinesisRecordSupplier recordSupplier; + private static List> records = ImmutableList.of( + new OrderedPartitionableRecord<>(stream, "1", "0", JB("2008", "a", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "1", JB("2009", "b", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "2", JB("2010", "c", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "3", JB("2011", "d", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "4", JB("2011", "e", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>( + stream, "1", + "5", JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0") ), - generateRequestEntry("1", StringUtils.toUtf8("unparseable")), - generateRequestEntry( - "1", - StringUtils.toUtf8("unparseable2") - ), - generateRequestEntry("1", StringUtils.toUtf8("{}")), - generateRequestEntry( - "1", - JB("2013", "f", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2049", "f", "y", "notanumber", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2049", "f", "y", "10", "notanumber", "1.0") - ), - generateRequestEntry( - "1", - JB("2049", "f", "y", "10", "20.0", "notanumber") - ), - generateRequestEntry("1", StringUtils.toUtf8("END_OF_PARTITION")), - generateRequestEntry( - "123123", - JB("2012", "g", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "123123", - JB("2011", "h", "y", "10", "20.0", "1.0") - ), - generateRequestEntry("123123", StringUtils.toUtf8("END_OF_PARTITION")) + new OrderedPartitionableRecord<>(stream, "1", "6", Collections.singletonList(StringUtils.toUtf8("unparseable"))), + new OrderedPartitionableRecord<>(stream, "1", "7", Collections.singletonList(StringUtils.toUtf8("unparseable2"))), + new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(StringUtils.toUtf8("{}"))), + new OrderedPartitionableRecord<>(stream, "1", "9", JB("2013", "f", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "10", JB("2049", "f", "y", "notanumber", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "11", JB("2049", "f", "y", "10", "notanumber", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "12", JB("2049", "f", "y", "10", "20.0", "notanumber")), + new OrderedPartitionableRecord<>(stream, "0", "0", JB("2012", "g", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "0", "1", JB("2011", "h", "y", "10", "20.0", "1.0")) ); - private static ServiceEmitter emitter; private static ListeningExecutorService taskExec; @@ -288,14 +246,10 @@ public class KinesisIndexTaskTest private TaskStorage taskStorage; private TaskLockbox taskLockbox; private File directory; - private String stream; - private final boolean isIncrementalHandoffSupported = false; private final Set checkpointRequestsHash = new HashSet<>(); private File reportsFile; private RowIngestionMetersFactory rowIngestionMetersFactory; - private int handoffCount = 0; - private static final DataSchema DATA_SCHEMA = new DataSchema( "test_ds", objectMapper.convertValue( @@ -329,32 +283,6 @@ public class KinesisIndexTaskTest objectMapper ); - private static PutRecordsRequestEntry generateRequestEntry(String partition, byte[] data) - { - return new PutRecordsRequestEntry().withPartitionKey(partition) - .withData(ByteBuffer.wrap(data)); - } - - private static PutRecordsRequest generateRecordsRequests(String stream, int first, int last) - { - return new PutRecordsRequest() - .withStreamName(stream) - .withRecords(records.subList(first, last)); - } - - private static PutRecordsRequest generateRecordsRequests(String stream) - { - return new PutRecordsRequest() - .withStreamName(stream) - .withRecords(records); - } - - private static String getStreamName() - { - return "stream-" + streamPosFix++; - } - - @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); @@ -388,10 +316,11 @@ public void setupTest() throws IOException, InterruptedException maxSavedParseExceptions = null; skipAvailabilityCheck = false; doHandoff = true; - stream = getStreamName(); reportsFile = File.createTempFile("KinesisIndexTaskTestReports-" + System.currentTimeMillis(), "json"); maxRecordsPerPoll = 1; + recordSupplier = mock(KinesisRecordSupplier.class); + // sleep required because of kinesalite Thread.sleep(500); makeToolboxFactory(); @@ -423,8 +352,20 @@ public static void tearDownClass() throws Exception @Test(timeout = 120_000L) public void testRunAfterDataInserted() throws Exception { - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 5)).once(); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); final KinesisIndexTask task = createTask( null, @@ -433,11 +374,11 @@ public void testRunAfterDataInserted() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" )), true, null, @@ -461,6 +402,8 @@ public void testRunAfterDataInserted() throws Exception // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + verifyAll(); + // Check metrics Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); @@ -475,7 +418,7 @@ public void testRunAfterDataInserted() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -489,21 +432,24 @@ public void testRunAfterDataInserted() throws Exception @Test(timeout = 120_000L) public void testRunBeforeDataInserted() throws Exception { - AmazonKinesis kinesis = getKinesisClientInstance(); - - // insert 1 row to get starting seq number - List res = insertData(kinesis, new PutRecordsRequest() - .withStreamName(stream) - .withRecords( - ImmutableList.of( - generateRequestEntry( - "123123", - JB("2055", "z", "y", "10", "20.0", "1.0") - ) - ) - ) - ); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(Collections.emptyList()) + .times(5) + .andReturn(records.subList(13, 15)) + .once(); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); final KinesisIndexTask task = createTask( null, @@ -512,11 +458,11 @@ public void testRunBeforeDataInserted() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + "1" )), true, null, @@ -537,43 +483,26 @@ public void testRunBeforeDataInserted() throws Exception final ListenableFuture future = runTask(task); - - // Wait for the task to start readingtestRunConfli - while (task.getRunner().getStatus() != Status.READING) { - Thread.sleep(10); - } - - // Insert data - List res2 = insertData(kinesis, generateRecordsRequests(stream)); - - while (task.getRunner().getRowIngestionMeters().getProcessed() < 3) { - Thread.sleep(10); - } - - // force shard to close - kinesis.splitShard(stream, shardId0, "somerandomshardidhah1213123"); - // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - + verifyAll(); // Check metrics - Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata SegmentDescriptor desc1 = SD(task, "2011/P1D", 0); SegmentDescriptor desc2 = SD(task, "2012/P1D", 0); - SegmentDescriptor desc3 = SD(task, "2055/P1D", 0); - Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors()); + Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors()); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId0, - OrderedPartitionableRecord.END_OF_SHARD_MARKER + "1" ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -582,9 +511,9 @@ public void testRunBeforeDataInserted() throws Exception // Check segments in deep storage Assert.assertEquals(ImmutableList.of("h"), readSegmentColumn("dim1", desc1)); Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc2)); - Assert.assertEquals(ImmutableList.of("z"), readSegmentColumn("dim1", desc3)); } + @Test(timeout = 120_000L) public void testIncrementalHandOff() throws Exception { @@ -593,46 +522,51 @@ public void testIncrementalHandOff() throws Exception maxRowsPerSegment = 2; maxRecordsPerPoll = 1; - // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(0, 5)) + .once() + .andReturn(records.subList(4, records.size())) + .once(); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" ) ); - // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering - // of events fetched across two partitions from Kafka + final SeekableStreamPartitions checkpoint1 = new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4), + "4", shardId0, - getSequenceNumber(res, shardId0, 0) - ) - ); - final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>( - stream, - ImmutableMap.of( - shardId1, - getSequenceNumber(res, shardId1, 3), - shardId0, - getSequenceNumber(res, shardId0, 1) + "0" ) ); + final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 9), + "9", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ) ); final KinesisIndexTask task = createTask( @@ -658,17 +592,17 @@ public void testIncrementalHandOff() throws Exception ) ); final ListenableFuture future = runTask(task); - while (task.getRunner().getStatus() != Status.PAUSED) { + while (task.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.PAUSED) { Thread.sleep(10); } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); - Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets) - || checkpoint2.getPartitionSequenceNumberMap() - .equals(currentOffsets)); + Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets)); task.getRunner().setEndOffsets(currentOffsets, false); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + verifyAll(); + Assert.assertEquals(1, checkpointRequestsHash.size()); Assert.assertTrue( checkpointRequestsHash.contains( @@ -698,9 +632,9 @@ public void testIncrementalHandOff() throws Exception Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 9), + "9", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -717,6 +651,7 @@ public void testIncrementalHandOff() throws Exception Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); } + @Test(timeout = 120_000L) public void testIncrementalHandOffMaxTotalRows() throws Exception { @@ -725,14 +660,31 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception maxRowsPerSegment = Integer.MAX_VALUE; maxTotalRows = 3L; + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(0, 3)) + .once() + .andReturn(records.subList(2, 10)) + .once() + .andReturn(records.subList(9, 11)); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); + // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream, 0, 13)); final SeekableStreamPartitions startPartitions = new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0) + "0" ) ); // Checkpointing will happen at either checkpoint1 or checkpoint2 depending on ordering @@ -741,21 +693,21 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" ) ); final SeekableStreamPartitions checkpoint2 = new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 9) + "9" ) ); final SeekableStreamPartitions endPartitions = new SeekableStreamPartitions<>( stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 10) + "10" ) ); @@ -803,6 +755,8 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + verifyAll(); + Assert.assertEquals(2, checkpointRequestsHash.size()); Assert.assertTrue( checkpointRequestsHash.contains( @@ -841,7 +795,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 10) + "10" ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -855,11 +809,24 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7)); } + @Test(timeout = 120_000L) public void testRunWithMinimumMessageTime() throws Exception { - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(0, 13)).once(); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); final KinesisIndexTask task = createTask( null, @@ -868,11 +835,11 @@ public void testRunWithMinimumMessageTime() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" )), true, null, @@ -900,6 +867,8 @@ public void testRunWithMinimumMessageTime() throws Exception // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + verifyAll(); + // Check metrics Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); @@ -915,7 +884,7 @@ public void testRunWithMinimumMessageTime() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -926,11 +895,24 @@ public void testRunWithMinimumMessageTime() throws Exception Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } + @Test(timeout = 120_000L) public void testRunWithMaximumMessageTime() throws Exception { - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(0, 13)).once(); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); final KinesisIndexTask task = createTask( null, @@ -939,11 +921,11 @@ public void testRunWithMaximumMessageTime() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" )), true, null, @@ -971,6 +953,8 @@ public void testRunWithMaximumMessageTime() throws Exception // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + verifyAll(); + // Check metrics Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); @@ -987,7 +971,7 @@ public void testRunWithMaximumMessageTime() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -999,11 +983,24 @@ public void testRunWithMaximumMessageTime() throws Exception Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); } + @Test(timeout = 120_000L) public void testRunWithTransformSpec() throws Exception { - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(0, 13)).once(); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); final KinesisIndexTask task = createTask( null, @@ -1020,11 +1017,11 @@ public void testRunWithTransformSpec() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" )), true, null, @@ -1052,6 +1049,8 @@ public void testRunWithTransformSpec() throws Exception // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + verifyAll(); + // Check metrics Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); @@ -1066,7 +1065,7 @@ public void testRunWithTransformSpec() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1077,11 +1076,22 @@ public void testRunWithTransformSpec() throws Exception Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", desc1)); } + @Test(timeout = 120_000L) public void testRunOnNothing() throws Exception { - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); final KinesisIndexTask task = createTask( null, @@ -1090,11 +1100,11 @@ public void testRunOnNothing() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), true, null, @@ -1117,6 +1127,8 @@ public void testRunOnNothing() throws Exception // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + verifyAll(); + // Check metrics Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); @@ -1126,14 +1138,26 @@ public void testRunOnNothing() throws Exception Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); } + @Test(timeout = 60_000L) public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception { handoffConditionTimeout = 5_000; - // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).once(); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); final KinesisIndexTask task = createTask( null, @@ -1142,11 +1166,11 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" )), true, null, @@ -1169,6 +1193,8 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + verifyAll(); + // Check metrics Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); @@ -1181,7 +1207,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1198,9 +1224,20 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio doHandoff = false; handoffConditionTimeout = 100; - // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).once(); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); final KinesisIndexTask task = createTask( null, @@ -1209,11 +1246,11 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" )), true, null, @@ -1236,6 +1273,8 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + verifyAll(); + // Check metrics Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); @@ -1248,7 +1287,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1268,8 +1307,17 @@ public void testReportParseExceptions() throws Exception maxParseExceptions = 1000; maxSavedParseExceptions = 2; - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).once(); + + replayAll(); final KinesisIndexTask task = createTask( null, @@ -1278,11 +1326,11 @@ public void testReportParseExceptions() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 5) + "5" )), true, null, @@ -1305,6 +1353,8 @@ public void testReportParseExceptions() throws Exception // Wait for task to exit Assert.assertEquals(TaskState.FAILED, future.get().getStatusCode()); + verifyAll(); + // Check metrics Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getUnparseable()); @@ -1315,6 +1365,7 @@ public void testReportParseExceptions() throws Exception Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); } + @Test(timeout = 120_000L) public void testMultipleParseExceptionsSuccess() throws Exception { @@ -1322,9 +1373,20 @@ public void testMultipleParseExceptionsSuccess() throws Exception maxParseExceptions = 7; maxSavedParseExceptions = 7; - // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).once(); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); final KinesisIndexTask task = createTask( null, @@ -1333,11 +1395,11 @@ public void testMultipleParseExceptionsSuccess() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 12) + "12" )), true, null, @@ -1357,13 +1419,14 @@ public void testMultipleParseExceptionsSuccess() throws Exception final ListenableFuture future = runTask(task); - kinesis.splitShard(stream, shardId1, "somerandomshardidhah1213123"); - TaskStatus status = future.get(); // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, status.getStatusCode()); - Assert.assertEquals(null, status.getErrorMsg()); + + verifyAll(); + + Assert.assertNull(status.getErrorMsg()); // Check metrics Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getProcessed()); @@ -1382,7 +1445,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 12) + "12" ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1417,6 +1480,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); } + @Test(timeout = 120_000L) public void testMultipleParseExceptionsFailure() throws Exception { @@ -1424,9 +1488,18 @@ public void testMultipleParseExceptionsFailure() throws Exception maxParseExceptions = 2; maxSavedParseExceptions = 2; - // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).once(); + + + replayAll(); final KinesisIndexTask task = createTask( null, @@ -1435,11 +1508,11 @@ public void testMultipleParseExceptionsFailure() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 9) + "9" )), true, null, @@ -1463,6 +1536,7 @@ public void testMultipleParseExceptionsFailure() throws Exception // Wait for task to exit Assert.assertEquals(TaskState.FAILED, status.getStatusCode()); + verifyAll(); IndexTaskTest.checkTaskStatusErrorMsgForParseExceptionsExceeded(status); // Check metrics @@ -1499,12 +1573,25 @@ public void testMultipleParseExceptionsFailure() throws Exception Assert.assertEquals(unparseableEvents, reportData.getUnparseableEvents()); } + @Test(timeout = 120_000L) public void testRunReplicas() throws Exception { // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)).times(2); + + recordSupplier.close(); + expectLastCall().times(2); + + replayAll(); final KinesisIndexTask task1 = createTask( null, @@ -1513,11 +1600,11 @@ public void testRunReplicas() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" )), true, null, @@ -1541,11 +1628,11 @@ public void testRunReplicas() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" )), true, null, @@ -1570,6 +1657,8 @@ public void testRunReplicas() throws Exception Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); + verifyAll(); + // Check metrics Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); @@ -1588,7 +1677,7 @@ public void testRunReplicas() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1599,12 +1688,27 @@ public void testRunReplicas() throws Exception Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } + @Test(timeout = 120_000L) public void testRunConflicting() throws Exception { - // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)) + .once() + .andReturn(records.subList(3, 13)) + .once(); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); final KinesisIndexTask task1 = createTask( null, @@ -1613,11 +1717,11 @@ public void testRunConflicting() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" )), true, null, @@ -1641,11 +1745,11 @@ public void testRunConflicting() throws Exception "sequence1", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3) + "3" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 9) + "9" )), true, null, @@ -1671,6 +1775,7 @@ public void testRunConflicting() throws Exception final ListenableFuture future2 = runTask(task2); Assert.assertEquals(TaskState.FAILED, future2.get().getStatusCode()); + verifyAll(); // Check metrics Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); @@ -1689,7 +1794,7 @@ public void testRunConflicting() throws Exception stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" ) )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1700,12 +1805,27 @@ public void testRunConflicting() throws Exception Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } + @Test(timeout = 120_000L) public void testRunConflictingWithoutTransactions() throws Exception { - // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)) + .once() + .andReturn(records.subList(3, 13)) + .once(); + + recordSupplier.close(); + expectLastCall().times(2); + + replayAll(); final KinesisIndexTask task1 = createTask( null, @@ -1714,11 +1834,11 @@ public void testRunConflictingWithoutTransactions() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" )), false, null, @@ -1742,11 +1862,11 @@ public void testRunConflictingWithoutTransactions() throws Exception "sequence1", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3) + "3" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 9) + "9" )), false, null, @@ -1778,6 +1898,8 @@ public void testRunConflictingWithoutTransactions() throws Exception final ListenableFuture future2 = runTask(task2); Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); + verifyAll(); + // Check metrics Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); @@ -1799,12 +1921,25 @@ public void testRunConflictingWithoutTransactions() throws Exception Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4)); } + @Test(timeout = 120_000L) public void testRunOneTaskTwoPartitions() throws Exception { // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, records.size())).once(); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); final KinesisIndexTask task = createTask( null, @@ -1813,15 +1948,15 @@ public void testRunOneTaskTwoPartitions() throws Exception "sequence1", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4), + "4", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )), true, null, @@ -1845,11 +1980,11 @@ public void testRunOneTaskTwoPartitions() throws Exception Thread.sleep(10); } - kinesis.splitShard(stream, shardId0, "somerandomshardidhah1213123"); - // Wait for tasks to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + verifyAll(); + // Check metrics Assert.assertEquals(5, task.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); @@ -1863,9 +1998,9 @@ public void testRunOneTaskTwoPartitions() throws Exception Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4), + "4", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1881,12 +2016,27 @@ public void testRunOneTaskTwoPartitions() throws Exception ); } + @Test(timeout = 120_000L) public void testRunTwoTasksTwoPartitions() throws Exception { - // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)) + .once() + .andReturn(records.subList(13, 15)) + .once(); + + recordSupplier.close(); + expectLastCall().times(2); + + replayAll(); final KinesisIndexTask task1 = createTask( null, @@ -1895,11 +2045,11 @@ public void testRunTwoTasksTwoPartitions() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" )), true, null, @@ -1923,11 +2073,11 @@ public void testRunTwoTasksTwoPartitions() throws Exception "sequence1", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + "1" )), true, null, @@ -1946,24 +2096,19 @@ public void testRunTwoTasksTwoPartitions() throws Exception ); final ListenableFuture future1 = runTask(task1); - final ListenableFuture future2 = runTask(task2); - - while (countEvents(task2) < 2) { - Thread.sleep(10); - } - - kinesis.splitShard(stream, shardId0, "somerandomshardidhah1213123"); - - // Wait for tasks to exit Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); + + final ListenableFuture future2 = runTask(task2); Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); + verifyAll(); + // Check metrics Assert.assertEquals(3, task1.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(2, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata @@ -1977,9 +2122,9 @@ public void testRunTwoTasksTwoPartitions() throws Exception new KinesisDataSourceMetadata( new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4), + "4", shardId0, - OrderedPartitionableRecord.END_OF_SHARD_MARKER + "1" ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -1994,12 +2139,24 @@ public void testRunTwoTasksTwoPartitions() throws Exception Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4)); } + @Test(timeout = 120_000L) public void testRestore() throws Exception { - // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream, 0, 4)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 4)) + .once() + .andReturn(Collections.emptyList()) + .anyTimes(); + + replayAll(); final KinesisIndexTask task1 = createTask( "task1", @@ -2008,11 +2165,11 @@ public void testRestore() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + "5" )), true, null, @@ -2044,7 +2201,23 @@ public void testRestore() throws Exception Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); - insertData(kinesis, generateRecordsRequests(stream, 4, 5)); + verifyAll(); + reset(recordSupplier); + + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(3, 6)).once(); + + recordSupplier.close(); + expectLastCall(); + + replayAll(); // Start a new task final KinesisIndexTask task2 = createTask( @@ -2054,11 +2227,11 @@ public void testRestore() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + "5" )), true, null, @@ -2084,17 +2257,17 @@ public void testRestore() throws Exception Assert.assertEquals(3, countEvents(task2)); - kinesis.splitShard(stream, shardId1, "somerandomshardidhah1213123"); - // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future2.get().getStatusCode()); + verifyAll(); + // Check metrics Assert.assertEquals(2, task1.getRunner().getRowIngestionMeters().getProcessed()); Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task1.getRunner().getRowIngestionMeters().getThrownAway()); Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getProcessed()); - Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata @@ -2105,7 +2278,7 @@ public void testRestore() throws Exception new KinesisDataSourceMetadata( new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - OrderedPartitionableRecord.END_OF_SHARD_MARKER + "5" ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -2115,12 +2288,24 @@ public void testRestore() throws Exception Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } + @Test(timeout = 120_000L) public void testRunWithPauseAndResume() throws Exception { - // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream, 0, 5)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 5)) + .once() + .andReturn(Collections.emptyList()) + .anyTimes(); + + replayAll(); final KinesisIndexTask task = createTask( "task1", @@ -2129,11 +2314,11 @@ public void testRunWithPauseAndResume() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2) + "2" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER + "13" )), true, null, @@ -2168,6 +2353,8 @@ public void testRunWithPauseAndResume() throws Exception } Assert.assertEquals(Status.PAUSED, task.getRunner().getStatus()); + verifyAll(); + Map currentOffsets = task.getRunner().getCurrentOffsets(); try { @@ -2180,9 +2367,26 @@ public void testRunWithPauseAndResume() throws Exception Assert.assertEquals(currentOffsets, task.getRunner().getCurrentOffsets()); + reset(recordSupplier); + + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + recordSupplier.close(); + expectLastCall().once(); + + replayAll(); + task.getRunner().setEndOffsets(currentOffsets, true); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + verifyAll(); Assert.assertEquals(task.getRunner().getEndOffsets(), task.getRunner().getCurrentOffsets()); // Check metrics @@ -2218,13 +2422,26 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception // which has done some incremental handoffs, thus the context will contain starting // sequence sequences from which the task should start reading and ignore the start sequences // Insert data - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream, 0, 6)); + recordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + + expect(recordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + + expect(recordSupplier.poll(anyLong())).andReturn(records.subList(2, 13)) + .once(); + + recordSupplier.close(); + expectLastCall(); + + replayAll(); final TreeMap> sequences = new TreeMap<>(); // Here the sequence number is 1 meaning that one incremental handoff was done by the failed task // and this task should start reading from stream 2 for partition 0 - sequences.put(1, ImmutableMap.of(shardId1, getSequenceNumber(res, shardId1, 2))); + sequences.put(1, ImmutableMap.of(shardId1, "2")); final Map context = new HashMap<>(); context.put("checkpoints", objectMapper.writerWithType(new TypeReference>>() { @@ -2238,11 +2455,11 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" )), true, null, @@ -2277,7 +2494,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 4) + "4" ))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); @@ -2336,7 +2553,7 @@ public boolean apply(TaskLock lock) private KinesisIndexTask createTask( final String taskId, final KinesisIOConfig ioConfig - ) + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { return createTask(taskId, DATA_SCHEMA, ioConfig); } @@ -2345,7 +2562,7 @@ private KinesisIndexTask createTask( final String taskId, final KinesisIOConfig ioConfig, final Map context - ) + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { return createTask(taskId, DATA_SCHEMA, ioConfig, context); } @@ -2354,7 +2571,7 @@ private KinesisIndexTask createTask( final String taskId, final DataSchema dataSchema, final KinesisIOConfig ioConfig - ) + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { final KinesisTuningConfig tuningConfig = new KinesisTuningConfig( 1000, @@ -2383,7 +2600,7 @@ private KinesisIndexTask createTask( intermediateHandoffPeriod ); final Map context = null; - final KinesisIndexTask task = new KinesisIndexTask( + final KinesisIndexTask task = new TestableKinesisIndexTask( taskId, null, cloneDataSchema(dataSchema), @@ -2394,6 +2611,7 @@ private KinesisIndexTask createTask( null, rowIngestionMetersFactory ); + return task; } @@ -2403,7 +2621,7 @@ private KinesisIndexTask createTask( final DataSchema dataSchema, final KinesisIOConfig ioConfig, final Map context - ) + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { final KinesisTuningConfig tuningConfig = new KinesisTuningConfig( maxRowsInMemory, @@ -2433,7 +2651,7 @@ private KinesisIndexTask createTask( ); context.put(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true); - final KinesisIndexTask task = new KinesisIndexTask( + final KinesisIndexTask task = new TestableKinesisIndexTask( taskId, null, cloneDataSchema(dataSchema), @@ -2444,6 +2662,7 @@ private KinesisIndexTask createTask( null, rowIngestionMetersFactory ); + return task; } @@ -2717,22 +2936,6 @@ private List readSegmentColumn(final String column, final SegmentDescrip return values; } - private void stopShards(Set shards, AmazonKinesis kinesis) - { - shards.forEach(s -> kinesis.splitShard(stream, s, "somerandomshardidhah" + getRandomShardName())); - } - - private String getRandomShardName() - { - Random random = ThreadLocalRandom.current(); - StringBuilder s = new StringBuilder(); - for (int i = 0; i < 10; i++) { - s.append((char) random.nextInt(27) + 'a'); - } - return s.toString(); - } - - public long countEvents(final Task task) { // Do a query. @@ -2752,10 +2955,17 @@ public long countEvents(final Task task) return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); } - private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) + private static List JB( + String timestamp, + String dim1, + String dim2, + String dimLong, + String dimFloat, + String met1 + ) { try { - return new ObjectMapper().writeValueAsBytes( + return Collections.singletonList(new ObjectMapper().writeValueAsBytes( ImmutableMap.builder() .put("timestamp", timestamp) .put("dim1", dim1) @@ -2764,7 +2974,7 @@ private static byte[] JB(String timestamp, String dim1, String dim2, String dimL .put("dimFloat", dimFloat) .put("met1", met1) .build() - ); + )); } catch (Exception e) { throw Throwables.propagate(e); @@ -2790,39 +3000,40 @@ private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOExcep ); } - private AmazonKinesis getKinesisClientInstance() throws InterruptedException + @JsonTypeName("index_kinesis") + private class TestableKinesisIndexTask extends KinesisIndexTask { - AmazonKinesis kinesis = TestUtils.getClientKinesis(); - SdkHttpMetadata createRes = kinesis.createStream(stream, 2).getSdkHttpMetadata(); - // sleep required because of kinesalite - Thread.sleep(500); - Assert.assertTrue(isResponseOk(createRes)); - return kinesis; - } - - private static String getSequenceNumber(List entries, String shardId, int offset) - { - List sortedEntries = entries.stream() - .filter(e -> e.getShardId().equals(shardId)) - .sorted(Comparator.comparing(e -> KinesisSequenceNumber.of(e.getSequenceNumber()))) - .collect(Collectors.toList()); - return sortedEntries.get(offset).getSequenceNumber(); - } - + @JsonCreator + public TestableKinesisIndexTask( + @JsonProperty("id") String id, + @JsonProperty("resource") TaskResource taskResource, + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("tuningConfig") KinesisTuningConfig tuningConfig, + @JsonProperty("ioConfig") KinesisIOConfig ioConfig, + @JsonProperty("context") Map context, + @JacksonInject ChatHandlerProvider chatHandlerProvider, + @JacksonInject AuthorizerMapper authorizerMapper, + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + { + super( + id, + taskResource, + dataSchema, + tuningConfig, + ioConfig, + context, + chatHandlerProvider, + authorizerMapper, + rowIngestionMetersFactory + ); + } - private static boolean isResponseOk(SdkHttpMetadata sdkHttpMetadata) - { - return sdkHttpMetadata.getHttpStatusCode() == 200; + @Override + protected KinesisRecordSupplier newTaskRecordSupplier() + { + return recordSupplier; + } } - private static List insertData( - AmazonKinesis kinesis, - PutRecordsRequest req - ) - { - PutRecordsResult res = kinesis.putRecords(req); - Assert.assertTrue(isResponseOk(res.getSdkHttpMetadata())); - Assert.assertEquals((int) res.getFailedRecordCount(), 0); - return res.getRecords(); - } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index 3072454b39fd..c454d49432b7 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -19,16 +19,15 @@ package org.apache.druid.indexing.kinesis; -import cloud.localstack.LocalstackTestRunner; -import cloud.localstack.TestUtils; -import cloud.localstack.docker.LocalstackDockerTestRunner; -import cloud.localstack.docker.annotation.LocalstackDockerProperties; -import com.amazonaws.http.SdkHttpMetadata; import com.amazonaws.services.kinesis.AmazonKinesis; -import com.amazonaws.services.kinesis.model.PutRecordsRequest; -import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry; -import com.amazonaws.services.kinesis.model.PutRecordsResult; -import com.amazonaws.services.kinesis.model.PutRecordsResultEntry; +import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.services.kinesis.model.DescribeStreamResult; +import com.amazonaws.services.kinesis.model.GetRecordsRequest; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.GetShardIteratorResult; +import com.amazonaws.services.kinesis.model.Record; +import com.amazonaws.services.kinesis.model.Shard; +import com.amazonaws.services.kinesis.model.StreamDescription; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -38,91 +37,93 @@ import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.logger.Logger; +import org.easymock.Capture; +import org.easymock.EasyMockSupport; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.junit.runner.RunWith; import java.nio.ByteBuffer; import java.util.Collections; -import java.util.Comparator; import java.util.List; import java.util.Set; import java.util.stream.Collectors; -@RunWith(LocalstackDockerTestRunner.class) -@LocalstackDockerProperties(services = {"kinesis"}) -public class KinesisRecordSupplierTest -{ - static { - TestUtils.setEnv("AWS_CBOR_DISABLE", "1"); - } +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.anyString; +import static org.easymock.EasyMock.capture; +import static org.easymock.EasyMock.eq; +import static org.easymock.EasyMock.expect; - private static final Logger log = new Logger(KinesisRecordSupplierTest.class); - private static String stream = "streamm"; +public class KinesisRecordSupplierTest extends EasyMockSupport +{ + private static final String stream = "stream"; private static long poll_timeout_millis = 2000; - private static String shardId1 = "shardId-000000000001"; - private static String shardId0 = "shardId-000000000000"; - private static int streamPosFix = 0; - private static int pollRetry = 10; + private static int recordsPerFetch; + private static String shardId1 = "1"; + private static String shardId0 = "0"; + private static String shard1Iterator = "1"; + private static String shard0Iterator = "0"; + private static AmazonKinesis kinesis; + private static DescribeStreamResult describeStreamResult; + private static GetShardIteratorResult getShardIteratorResult0; + private static GetShardIteratorResult getShardIteratorResult1; + private static GetRecordsResult getRecordsResult0; + private static GetRecordsResult getRecordsResult1; + private static StreamDescription streamDescription; + private static Shard shard0, shard1; private static KinesisRecordSupplier recordSupplier; - private static final List records = ImmutableList.of( - generateRequestEntry( - "1", - JB("2011", "d", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2011", "e", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0") - ), - generateRequestEntry("1", StringUtils.toUtf8("unparseable")), - generateRequestEntry( - "1", - StringUtils.toUtf8("unparseable2") - ), - generateRequestEntry("1", StringUtils.toUtf8("{}")), - generateRequestEntry( - "1", - JB("2013", "f", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2049", "f", "y", "notanumber", "20.0", "1.0") - ), - generateRequestEntry( - "123123", - JB("2012", "g", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "123123", - JB("2011", "h", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "123123", - JB("2008", "a", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "123123", - JB("2009", "b", "y", "10", "20.0", "1.0") - ) + private static List shard1Records = ImmutableList.of( + new Record().withData(JB("2011", "d", "y", "10", "20.0", "1.0")).withSequenceNumber("0"), + new Record().withData(JB("2011", "e", "y", "10", "20.0", "1.0")).withSequenceNumber("1"), + new Record().withData(JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")).withSequenceNumber("2"), + new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable"))).withSequenceNumber("3"), + new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable2"))).withSequenceNumber("4"), + new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("{}"))).withSequenceNumber("5"), + new Record().withData(JB("2013", "f", "y", "10", "20.0", "1.0")).withSequenceNumber("6"), + new Record().withData(JB("2049", "f", "y", "notanumber", "20.0", "1.0")).withSequenceNumber("7"), + new Record().withData(JB("2012", "g", "y", "10", "20.0", "1.0")).withSequenceNumber("8"), + new Record().withData(JB("2011", "h", "y", "10", "20.0", "1.0")).withSequenceNumber("9") ); - - private static PutRecordsRequestEntry generateRequestEntry(String partition, byte[] data) - { - return new PutRecordsRequestEntry().withPartitionKey(partition) - .withData(ByteBuffer.wrap(data)); - } - - private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) + private static List shard0Records = ImmutableList.of( + new Record().withData(JB("2008", "a", "y", "10", "20.0", "1.0")).withSequenceNumber("0"), + new Record().withData(JB("2009", "b", "y", "10", "20.0", "1.0")).withSequenceNumber("1") + ); + private static List allRecords = ImmutableList.builder() + .addAll(shard0Records.stream() + .map(x -> new OrderedPartitionableRecord<>( + stream, + shardId0, + x.getSequenceNumber(), + Collections + .singletonList( + toByteArray( + x.getData())) + )) + .collect( + Collectors + .toList())) + .addAll(shard1Records.stream() + .map(x -> new OrderedPartitionableRecord<>( + stream, + shardId1, + x.getSequenceNumber(), + Collections + .singletonList( + toByteArray( + x.getData())) + )) + .collect( + Collectors + .toList())) + .build(); + ; + + private static ByteBuffer JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) { try { - return new ObjectMapper().writeValueAsBytes( + return ByteBuffer.wrap(new ObjectMapper().writeValueAsBytes( ImmutableMap.builder() .put("timestamp", timestamp) .put("dim1", dim1) @@ -131,56 +132,26 @@ private static byte[] JB(String timestamp, String dim1, String dim2, String dimL .put("dimFloat", dimFloat) .put("met1", met1) .build() - ); + )); } catch (Exception e) { throw Throwables.propagate(e); } } - - private AmazonKinesis getKinesisClientInstance() throws InterruptedException - { - AmazonKinesis kinesis = TestUtils.getClientKinesis(); - SdkHttpMetadata createRes = kinesis.createStream(stream, 2).getSdkHttpMetadata(); - // sleep required because of kinesalite - Thread.sleep(500); - return kinesis; - } - - private static PutRecordsRequest generateRecordsRequests(String stream) - { - return new PutRecordsRequest() - .withStreamName(stream) - .withRecords(records); - } - - private static PutRecordsRequest generateRecordsRequests(String stream, int first, int last) - { - return new PutRecordsRequest() - .withStreamName(stream) - .withRecords(records.subList(first, last)); - } - - private static List insertData( - AmazonKinesis kinesis, - PutRecordsRequest req - ) - { - PutRecordsResult res = kinesis.putRecords(req); - Assert.assertEquals((int) res.getFailedRecordCount(), 0); - return res.getRecords(); - } - - private static String getStreamName() - { - return "stream-" + streamPosFix++; - } - @Before public void setupTest() { - stream = getStreamName(); + kinesis = createMock(AmazonKinesisClient.class); + describeStreamResult = createMock(DescribeStreamResult.class); + getShardIteratorResult0 = createMock(GetShardIteratorResult.class); + getShardIteratorResult1 = createMock(GetShardIteratorResult.class); + getRecordsResult0 = createMock(GetRecordsResult.class); + getRecordsResult1 = createMock(GetRecordsResult.class); + streamDescription = createMock(StreamDescription.class); + shard0 = createMock(Shard.class); + shard1 = createMock(Shard.class); + recordsPerFetch = 1; } @After @@ -192,10 +163,16 @@ public void tearDownTest() @Test public void testSupplierSetup() - throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException + throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException { + Capture captured = Capture.newInstance(); + expect(kinesis.describeStream(capture(captured))).andReturn(describeStreamResult).once(); + expect(describeStreamResult.getStreamDescription()).andReturn(streamDescription).once(); + expect(streamDescription.getShards()).andReturn(ImmutableList.of(shard0, shard1)).once(); + expect(shard0.getShardId()).andReturn(shardId0).once(); + expect(shard1.getShardId()).andReturn(shardId1).once(); - getKinesisClientInstance(); + replayAll(); Set> partitions = ImmutableSet.of( StreamPartition.of(stream, shardId0), @@ -203,14 +180,10 @@ public void testSupplierSetup() ); recordSupplier = new KinesisRecordSupplier( - LocalstackTestRunner.getEndpointKinesis(), - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, - 1, + kinesis, + recordsPerFetch, 0, 2, - null, - null, false, 100, 5000, @@ -226,197 +199,197 @@ public void testSupplierSetup() Assert.assertEquals(partitions, recordSupplier.getAssignment()); Assert.assertEquals(ImmutableSet.of(shardId1, shardId0), recordSupplier.getPartitionIds(stream)); Assert.assertEquals(Collections.emptyList(), recordSupplier.poll(100)); + + verifyAll(); + Assert.assertEquals(stream, captured.getValue()); } - @Test - public void testPoll() - throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException + private static GetRecordsRequest generateGetRecordsReq(String shardIterator, int limit) { - AmazonKinesis kinesis = getKinesisClientInstance(); - List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); - Set> initialRecords = insertDataResults.stream() - .map(r -> new OrderedPartitionableRecord<>( - stream, - r.getShardId(), - r.getSequenceNumber(), - null - )) - .collect(Collectors.toSet()); - - Set> partitions = ImmutableSet.of( - StreamPartition.of(stream, shardId0), - StreamPartition.of(stream, shardId1) - ); - - recordSupplier = new KinesisRecordSupplier( - LocalstackTestRunner.getEndpointKinesis(), - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, - 100, - 0, - 2, - null, - null, - false, - 100, - 5000, - 5000, - 60000, - 100 - ); - - recordSupplier.assign(partitions); - recordSupplier.seekToEarliest(partitions); - - List> polledRecords = recordSupplier.poll(poll_timeout_millis); - for (int i = 0; polledRecords.size() != initialRecords.size() && i < pollRetry; i++) { - polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); - Thread.sleep(200); - } + return new GetRecordsRequest().withShardIterator(shardIterator).withLimit(limit); + } - Assert.assertEquals(partitions, recordSupplier.getAssignment()); - Assert.assertEquals(initialRecords.size(), polledRecords.size()); - Assert.assertTrue(polledRecords.containsAll(initialRecords)); + // filter out EOS markers + private static List> cleanRecords(List> records) + { + return records.stream() + .filter(x -> !x.getSequenceNumber() + .equals(OrderedPartitionableRecord.END_OF_SHARD_MARKER)) + .collect(Collectors.toList()); } @Test - public void testPollAfterMoreDataAdded() - throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException + public void testPoll() + throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException, InterruptedException { - AmazonKinesis kinesis = getKinesisClientInstance(); - List insertDataResults1 = insertData(kinesis, generateRecordsRequests(stream, 0, 5)); - Set> initialRecords = insertDataResults1.stream() - .map(r -> new OrderedPartitionableRecord<>( - stream, - r.getShardId(), - r.getSequenceNumber(), - null - )) - .collect(Collectors.toSet()); + recordsPerFetch = 100; + + expect(kinesis.getShardIterator( + anyObject(), + eq(shardId0), + anyString(), + anyString() + )).andReturn( + getShardIteratorResult0).anyTimes(); + + expect(kinesis.getShardIterator( + anyObject(), + eq(shardId1), + anyString(), + anyString() + )).andReturn( + getShardIteratorResult1).anyTimes(); + + expect(getShardIteratorResult0.getShardIterator()).andReturn(shard0Iterator).anyTimes(); + expect(getShardIteratorResult1.getShardIterator()).andReturn(shard1Iterator).anyTimes(); + expect(kinesis.getRecords(generateGetRecordsReq(shard0Iterator, recordsPerFetch))).andReturn(getRecordsResult0) + .anyTimes(); + expect(kinesis.getRecords(generateGetRecordsReq(shard1Iterator, recordsPerFetch))).andReturn(getRecordsResult1) + .anyTimes(); + expect(getRecordsResult0.getRecords()).andReturn(shard0Records).once(); + expect(getRecordsResult1.getRecords()).andReturn(shard1Records).once(); + expect(getRecordsResult0.getNextShardIterator()).andReturn(null).anyTimes(); + expect(getRecordsResult1.getNextShardIterator()).andReturn(null).anyTimes(); + + replayAll(); Set> partitions = ImmutableSet.of( StreamPartition.of(stream, shardId0), StreamPartition.of(stream, shardId1) ); + recordSupplier = new KinesisRecordSupplier( - LocalstackTestRunner.getEndpointKinesis(), - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, - 1, + kinesis, + recordsPerFetch, 0, 2, - null, - null, false, 100, 5000, 5000, 60000, - 5 + 100 ); recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); + recordSupplier.start(); - List> polledRecords = recordSupplier.poll(poll_timeout_millis); - for (int i = 0; polledRecords.size() != initialRecords.size() && i < pollRetry; i++) { - polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); - Thread.sleep(200); + while (recordSupplier.bufferSize() < 12) { + Thread.sleep(100); } - List insertDataResults2 = insertData(kinesis, generateRecordsRequests(stream, 5, 12)); - insertDataResults2.forEach(entry -> initialRecords.add(new OrderedPartitionableRecord<>( - stream, - entry.getShardId(), - entry.getSequenceNumber(), - null - ))); - - for (int i = 0; polledRecords.size() != initialRecords.size() && i < pollRetry; i++) { - polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); - Thread.sleep(200); - } + List> polledRecords = cleanRecords(recordSupplier.poll( + poll_timeout_millis)); + + verifyAll(); - Assert.assertEquals(initialRecords.size(), polledRecords.size()); - Assert.assertTrue(polledRecords.containsAll(initialRecords)); + Assert.assertEquals(partitions, recordSupplier.getAssignment()); + Assert.assertTrue(polledRecords.containsAll(allRecords)); } @Test public void testSeek() throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { - AmazonKinesis kinesis = getKinesisClientInstance(); - List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); - - StreamPartition shard0 = StreamPartition.of(stream, shardId0); - StreamPartition shard1 = StreamPartition.of(stream, shardId1); + recordsPerFetch = 100; + + expect(kinesis.getShardIterator( + anyObject(), + eq(shardId0), + anyString(), + anyString() + )).andReturn( + getShardIteratorResult0).anyTimes(); + + expect(kinesis.getShardIterator( + anyObject(), + eq(shardId1), + anyString(), + anyString() + )).andReturn( + getShardIteratorResult1).anyTimes(); + + expect(getShardIteratorResult0.getShardIterator()).andReturn(shard0Iterator).anyTimes(); + expect(getShardIteratorResult1.getShardIterator()).andReturn(shard1Iterator).anyTimes(); + expect(kinesis.getRecords(generateGetRecordsReq(shard0Iterator, recordsPerFetch))).andReturn(getRecordsResult0) + .anyTimes(); + expect(kinesis.getRecords(generateGetRecordsReq(shard1Iterator, recordsPerFetch))).andReturn(getRecordsResult1) + .anyTimes(); + expect(getRecordsResult0.getRecords()).andReturn(shard0Records.subList(1, shard0Records.size())).once(); + expect(getRecordsResult1.getRecords()).andReturn(shard1Records.subList(2, shard1Records.size())).once(); + expect(getRecordsResult0.getNextShardIterator()).andReturn(null).anyTimes(); + expect(getRecordsResult1.getNextShardIterator()).andReturn(null).anyTimes(); + + replayAll(); + + StreamPartition shard0Partition = StreamPartition.of(stream, shardId0); + StreamPartition shard1Partition = StreamPartition.of(stream, shardId1); Set> partitions = ImmutableSet.of( - shard0, - shard1 + shard0Partition, + shard1Partition ); recordSupplier = new KinesisRecordSupplier( - LocalstackTestRunner.getEndpointKinesis(), - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, - 1, + kinesis, + recordsPerFetch, 0, 2, - null, - null, false, 100, 5000, 5000, 60000, - 5 + 100 ); recordSupplier.assign(partitions); + recordSupplier.seek(shard1Partition, shard1Records.get(2).getSequenceNumber()); + recordSupplier.seek(shard0Partition, shard0Records.get(1).getSequenceNumber()); + recordSupplier.start(); - Assert.assertEquals(insertDataResults.get(0).getSequenceNumber(), recordSupplier.getEarliestSequenceNumber(shard1)); - Assert.assertEquals(insertDataResults.get(8).getSequenceNumber(), recordSupplier.getEarliestSequenceNumber(shard0)); - - recordSupplier.seek(shard1, insertDataResults.get(2).getSequenceNumber()); - recordSupplier.seek(shard0, insertDataResults.get(10).getSequenceNumber()); - - Set> initialRecords1 = insertDataResults.subList(2, 8).stream() - .map(r -> new OrderedPartitionableRecord<>( - stream, - r.getShardId(), - r.getSequenceNumber(), - null - )) - .collect(Collectors.toSet()); - - Set> initialRecords2 = insertDataResults.subList(10, 12).stream() - .map(r -> new OrderedPartitionableRecord<>( - stream, - r.getShardId(), - r.getSequenceNumber(), - null - )) - .collect(Collectors.toSet()); - - List> polledRecords = recordSupplier.poll(poll_timeout_millis); - for (int i = 0; polledRecords.size() != 8 && i < pollRetry; i++) { - polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); - Thread.sleep(200); + while (recordSupplier.bufferSize() < 9) { + Thread.sleep(100); } - Assert.assertEquals(8, polledRecords.size()); - Assert.assertTrue(polledRecords.containsAll(initialRecords1)); - Assert.assertTrue(polledRecords.containsAll(initialRecords2)); + List> polledRecords = cleanRecords(recordSupplier.poll( + poll_timeout_millis)); + + verifyAll(); + Assert.assertEquals(9, polledRecords.size()); + Assert.assertTrue(polledRecords.containsAll(allRecords.subList(4, 12))); + Assert.assertTrue(polledRecords.containsAll(allRecords.subList(1, 2))); } + @Test public void testSeekToLatest() throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { - AmazonKinesis kinesis = getKinesisClientInstance(); - List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); + recordsPerFetch = 100; + + expect(kinesis.getShardIterator( + anyObject(), + eq(shardId0), + anyString(), + anyString() + )).andReturn( + getShardIteratorResult0).anyTimes(); + + expect(kinesis.getShardIterator( + anyObject(), + eq(shardId1), + anyString(), + anyString() + )).andReturn( + getShardIteratorResult1).anyTimes(); + + expect(getShardIteratorResult0.getShardIterator()).andReturn(null).once(); + expect(getShardIteratorResult1.getShardIterator()).andReturn(null).once(); + + replayAll(); StreamPartition shard0 = StreamPartition.of(stream, shardId0); StreamPartition shard1 = StreamPartition.of(stream, shardId1); @@ -426,39 +399,34 @@ public void testSeekToLatest() ); recordSupplier = new KinesisRecordSupplier( - LocalstackTestRunner.getEndpointKinesis(), - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, - 1, + kinesis, + recordsPerFetch, 0, 2, - null, - null, false, 100, 5000, 5000, 60000, - 5 + 100 ); - recordSupplier.assign(partitions); + recordSupplier.seekToLatest(partitions); + recordSupplier.start(); - Assert.assertEquals(insertDataResults.get(0).getSequenceNumber(), recordSupplier.getEarliestSequenceNumber(shard1)); - Assert.assertEquals(insertDataResults.get(8).getSequenceNumber(), recordSupplier.getEarliestSequenceNumber(shard0)); + while (recordSupplier.bufferSize() < 2) { + Thread.sleep(100); + } + Assert.assertEquals(Collections.emptyList(), cleanRecords(recordSupplier.poll(poll_timeout_millis))); - recordSupplier.seekToLatest(partitions); - Assert.assertEquals(Collections.emptyList(), recordSupplier.poll(poll_timeout_millis)); + verifyAll(); } @Test(expected = ISE.class) public void testSeekUnassigned() - throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException + throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException { - AmazonKinesis kinesis = getKinesisClientInstance(); - List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); - StreamPartition shard0 = StreamPartition.of(stream, shardId0); StreamPartition shard1 = StreamPartition.of(stream, shardId1); Set> partitions = ImmutableSet.of( @@ -466,14 +434,10 @@ public void testSeekUnassigned() ); recordSupplier = new KinesisRecordSupplier( - LocalstackTestRunner.getEndpointKinesis(), - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + kinesis, 1, 0, 2, - null, - null, false, 100, 5000, @@ -483,32 +447,50 @@ public void testSeekUnassigned() ); recordSupplier.assign(partitions); - - Assert.assertEquals(insertDataResults.get(0).getSequenceNumber(), recordSupplier.getEarliestSequenceNumber(shard1)); - recordSupplier.seekToEarliest(Collections.singleton(shard0)); } + @Test public void testPollAfterSeek() throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { - AmazonKinesis kinesis = getKinesisClientInstance(); - List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); + // tests that after doing a seek, the now invalid records in buffer is cleaned up properly + recordsPerFetch = 100; + + expect(kinesis.getShardIterator( + anyObject(), + eq(shardId1), + anyString(), + eq("5") + )).andReturn( + getShardIteratorResult1).once(); + + expect(kinesis.getShardIterator(anyObject(), eq(shardId1), anyString(), eq("7"))).andReturn(getShardIteratorResult0) + .once(); + + expect(getShardIteratorResult1.getShardIterator()).andReturn(shard1Iterator).once(); + expect(getShardIteratorResult0.getShardIterator()).andReturn(shard0Iterator).once(); + expect(kinesis.getRecords(generateGetRecordsReq(shard1Iterator, recordsPerFetch))).andReturn(getRecordsResult1) + .once(); + expect(kinesis.getRecords(generateGetRecordsReq(shard0Iterator, recordsPerFetch))).andReturn(getRecordsResult0) + .once(); + expect(getRecordsResult1.getRecords()).andReturn(shard1Records.subList(5, shard1Records.size())).once(); + expect(getRecordsResult0.getRecords()).andReturn(shard1Records.subList(7, shard1Records.size())).once(); + expect(getRecordsResult1.getNextShardIterator()).andReturn(null).anyTimes(); + expect(getRecordsResult0.getNextShardIterator()).andReturn(null).anyTimes(); + + replayAll(); Set> partitions = ImmutableSet.of( StreamPartition.of(stream, shardId1) ); recordSupplier = new KinesisRecordSupplier( - LocalstackTestRunner.getEndpointKinesis(), - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, - 10, + kinesis, + recordsPerFetch, 0, 2, - null, - null, false, 100, 5000, @@ -518,72 +500,80 @@ public void testPollAfterSeek() ); recordSupplier.assign(partitions); - recordSupplier.seek(StreamPartition.of(stream, shardId1), getSequenceNumber(insertDataResults, shardId1, 5)); + recordSupplier.seek(StreamPartition.of(stream, shardId1), "5"); + recordSupplier.start(); - for (int i = 0; recordSupplier.bufferSize() < 2 && i < pollRetry; i++) { - Thread.sleep(200); + while (recordSupplier.bufferSize() < 6) { + Thread.sleep(100); } + OrderedPartitionableRecord firstRecord = recordSupplier.poll(poll_timeout_millis).get(0); Assert.assertEquals( - getSequenceNumber(insertDataResults, shardId1, 5), - firstRecord.getSequenceNumber() + allRecords.get(7), + firstRecord ); - recordSupplier.seek(StreamPartition.of(stream, shardId1), getSequenceNumber(insertDataResults, shardId1, 7)); - for (int i = 0; recordSupplier.bufferSize() < 2 && i < pollRetry; i++) { - Thread.sleep(200); + recordSupplier.seek(StreamPartition.of(stream, shardId1), "7"); + recordSupplier.start(); + + while (recordSupplier.bufferSize() < 4) { + Thread.sleep(100); } OrderedPartitionableRecord record2 = recordSupplier.poll(poll_timeout_millis).get(0); - Assert.assertNotNull(record2); - Assert.assertEquals(stream, record2.getStream()); - Assert.assertEquals(shardId1, record2.getPartitionId()); - Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 7), record2.getSequenceNumber()); - - recordSupplier.seek(StreamPartition.of(stream, shardId1), getSequenceNumber(insertDataResults, shardId1, 2)); - for (int i = 0; recordSupplier.bufferSize() < 2 && i < pollRetry; i++) { - Thread.sleep(200); - } - OrderedPartitionableRecord record3 = recordSupplier.poll(poll_timeout_millis).get(0); - - Assert.assertNotNull(record3); - Assert.assertEquals(stream, record3.getStream()); - Assert.assertEquals(shardId1, record3.getPartitionId()); - Assert.assertEquals(getSequenceNumber(insertDataResults, shardId1, 2), record3.getSequenceNumber()); + Assert.assertEquals(allRecords.get(9), record2); + verifyAll(); } @Test - public void testDeaggregate() - throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException + public void testPollDeaggregate() + throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException, InterruptedException { - AmazonKinesis kinesis = getKinesisClientInstance(); - List insertDataResults = insertData(kinesis, generateRecordsRequests(stream)); - Set> initialRecords = insertDataResults.stream() - .map(r -> new OrderedPartitionableRecord<>( - stream, - r.getShardId(), - r.getSequenceNumber(), - null - )) - .collect(Collectors.toSet()); + recordsPerFetch = 100; + + expect(kinesis.getShardIterator( + anyObject(), + eq(shardId0), + anyString(), + anyString() + )).andReturn( + getShardIteratorResult0).anyTimes(); + + expect(kinesis.getShardIterator( + anyObject(), + eq(shardId1), + anyString(), + anyString() + )).andReturn( + getShardIteratorResult1).anyTimes(); + + expect(getShardIteratorResult0.getShardIterator()).andReturn(shard0Iterator).anyTimes(); + expect(getShardIteratorResult1.getShardIterator()).andReturn(shard1Iterator).anyTimes(); + expect(kinesis.getRecords(generateGetRecordsReq(shard0Iterator, recordsPerFetch))).andReturn(getRecordsResult0) + .anyTimes(); + expect(kinesis.getRecords(generateGetRecordsReq(shard1Iterator, recordsPerFetch))).andReturn(getRecordsResult1) + .anyTimes(); + expect(getRecordsResult0.getRecords()).andReturn(shard0Records).once(); + expect(getRecordsResult1.getRecords()).andReturn(shard1Records).once(); + expect(getRecordsResult0.getNextShardIterator()).andReturn(null).anyTimes(); + expect(getRecordsResult1.getNextShardIterator()).andReturn(null).anyTimes(); + + replayAll(); Set> partitions = ImmutableSet.of( StreamPartition.of(stream, shardId0), StreamPartition.of(stream, shardId1) ); + recordSupplier = new KinesisRecordSupplier( - LocalstackTestRunner.getEndpointKinesis(), - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, - 100, + kinesis, + recordsPerFetch, 0, 2, - null, - null, true, 100, 5000, @@ -594,25 +584,36 @@ public void testDeaggregate() recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); + recordSupplier.start(); - List> polledRecords = recordSupplier.poll(poll_timeout_millis); - for (int i = 0; polledRecords.size() != initialRecords.size() && i < pollRetry; i++) { - polledRecords.addAll(recordSupplier.poll(poll_timeout_millis)); - Thread.sleep(200); + while (recordSupplier.bufferSize() < 12) { + Thread.sleep(100); } + List> polledRecords = cleanRecords(recordSupplier.poll( + poll_timeout_millis)); + + verifyAll(); + Assert.assertEquals(partitions, recordSupplier.getAssignment()); - Assert.assertEquals(initialRecords.size(), polledRecords.size()); - Assert.assertTrue(polledRecords.containsAll(initialRecords)); + Assert.assertTrue(polledRecords.containsAll(allRecords)); } - - private static String getSequenceNumber(List entries, String shardId, int sequence) + /** + * Returns an array with the content between the position and limit of "buffer". This may be the buffer's backing + * array itself. Does not modify position or limit of the buffer. + */ + private static byte[] toByteArray(final ByteBuffer buffer) { - List sortedEntries = entries.stream() - .filter(e -> e.getShardId().equals(shardId)) - .sorted(Comparator.comparing(e -> KinesisSequenceNumber.of(e.getSequenceNumber()))) - .collect(Collectors.toList()); - return sortedEntries.get(sequence).getSequenceNumber(); + if (buffer.hasArray() + && buffer.arrayOffset() == 0 + && buffer.position() == 0 + && buffer.array().length == buffer.limit()) { + return buffer.array(); + } else { + final byte[] retVal = new byte[buffer.remaining()]; + buffer.duplicate().get(retVal); + return retVal; + } } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 9c8769e75b56..6e7b53d3f7f1 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -20,14 +20,6 @@ package org.apache.druid.indexing.kinesis.supervisor; import cloud.localstack.LocalstackTestRunner; -import cloud.localstack.docker.LocalstackDockerTestRunner; -import cloud.localstack.docker.annotation.LocalstackDockerProperties; -import com.amazonaws.http.SdkHttpMetadata; -import com.amazonaws.services.kinesis.AmazonKinesis; -import com.amazonaws.services.kinesis.model.PutRecordsRequest; -import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry; -import com.amazonaws.services.kinesis.model.PutRecordsResult; -import com.amazonaws.services.kinesis.model.PutRecordsResultEntry; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; @@ -55,7 +47,7 @@ import org.apache.druid.indexing.kinesis.KinesisIndexTask; import org.apache.druid.indexing.kinesis.KinesisIndexTaskClient; import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; -import org.apache.druid.indexing.kinesis.KinesisSequenceNumber; +import org.apache.druid.indexing.kinesis.KinesisRecordSupplier; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; @@ -65,8 +57,10 @@ import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -97,30 +91,29 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; -import org.junit.runner.RunWith; import java.io.File; -import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.TreeMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; +import static org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.anyString; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.expectLastCall; -@RunWith(LocalstackDockerTestRunner.class) -@LocalstackDockerProperties(services = {"kinesis"}) public class KinesisSupervisorTest extends EasyMockSupport { + + private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); private static final String TOPIC_PREFIX = "testTopic"; private static final String DATASOURCE = "testDS"; @@ -128,71 +121,39 @@ public class KinesisSupervisorTest extends EasyMockSupport private static final long TEST_CHAT_RETRIES = 9L; private static final Period TEST_HTTP_TIMEOUT = new Period("PT10S"); private static final Period TEST_SHUTDOWN_TIMEOUT = new Period("PT80S"); - private static final List records = ImmutableList.of( - generateRequestEntry( - "1", - JB("2008", "a", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2009", "b", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2010", "c", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2011", "d", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2011", "e", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( + private static final String stream = "stream"; + private static String shardId1 = "1"; + private static String shardId0 = "0"; + private static StreamPartition shard1Partition = StreamPartition.of(stream, shardId1); + private static StreamPartition shard0Partition = StreamPartition.of(stream, shardId0); + private static List> records = ImmutableList.of( + new OrderedPartitionableRecord<>(stream, "1", "0", JB("2008", "a", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "1", JB("2009", "b", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "2", JB("2010", "c", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "3", JB("2011", "d", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "4", JB("2011", "e", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>( + stream, "1", + "5", JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0") ), - generateRequestEntry("1", StringUtils.toUtf8("unparseable")), - generateRequestEntry( - "1", - StringUtils.toUtf8("unparseable2") - ), - generateRequestEntry("1", StringUtils.toUtf8("{}")), - generateRequestEntry( - "1", - JB("2013", "f", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2049", "f", "y", "notanumber", "20.0", "1.0") - ), - generateRequestEntry( - "1", - JB("2049", "f", "y", "10", "notanumber", "1.0") - ), - generateRequestEntry( - "1", - JB("2049", "f", "y", "10", "20.0", "notanumber") - ), - generateRequestEntry( - "123123", - JB("2012", "g", "y", "10", "20.0", "1.0") - ), - generateRequestEntry( - "123123", - JB("2011", "h", "y", "10", "20.0", "1.0") - ) + new OrderedPartitionableRecord<>(stream, "1", "6", Collections.singletonList(StringUtils.toUtf8("unparseable"))), + new OrderedPartitionableRecord<>(stream, "1", "7", Collections.singletonList(StringUtils.toUtf8("unparseable2"))), + new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(StringUtils.toUtf8("{}"))), + new OrderedPartitionableRecord<>(stream, "1", "9", JB("2013", "f", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "10", JB("2049", "f", "y", "notanumber", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "11", JB("2049", "f", "y", "10", "notanumber", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "12", JB("2049", "f", "y", "10", "20.0", "notanumber")), + new OrderedPartitionableRecord<>(stream, "0", "0", JB("2012", "g", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "0", "1", JB("2011", "h", "y", "10", "20.0", "1.0")) ); - private static String shardId1 = "shardId-000000000001"; - private static String shardId0 = "shardId-000000000000"; - private static DataSchema dataSchema; - private static int topicPostfix; + private KinesisRecordSupplier taskRecordSupplier; + private KinesisRecordSupplier supervisorRecordSupplier; private final int numThreads; - private KinesisSupervisor supervisor; private KinesisSupervisorTuningConfig tuningConfig; private TaskStorage taskStorage; @@ -201,15 +162,9 @@ public class KinesisSupervisorTest extends EasyMockSupport private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; private KinesisIndexTaskClient taskClient; private TaskQueue taskQueue; - private String stream; private RowIngestionMetersFactory rowIngestionMetersFactory; private ExceptionCapturingServiceEmitter serviceEmitter; - private static String getStream() - { - return TOPIC_PREFIX + topicPostfix++; - } - public KinesisSupervisorTest() { this.numThreads = 1; @@ -236,6 +191,8 @@ public void setupTest() indexerMetadataStorageCoordinator = createMock(IndexerMetadataStorageCoordinator.class); taskClient = createMock(KinesisIndexTaskClient.class); taskQueue = createMock(TaskQueue.class); + supervisorRecordSupplier = createMock(KinesisRecordSupplier.class); + taskRecordSupplier = mock(KinesisRecordSupplier.class); tuningConfig = new KinesisSupervisorTuningConfig( 1000, @@ -268,8 +225,6 @@ public void setupTest() null, null ); - - stream = getStream(); rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); serviceEmitter = new ExceptionCapturingServiceEmitter(); EmittingLogger.registerEmitter(serviceEmitter); @@ -286,20 +241,48 @@ public void tearDownTest() public void testNoInitialState() throws Exception { supervisor = getSupervisor(1, 1, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + //supervisorRecordSupplier.close(); + //expectLastCall().once(); + +// taskRecordSupplier.assign(anyObject()); +// expectLastCall().anyTimes(); +// +// expect(taskRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); +// +// taskRecordSupplier.seek(anyObject(), anyString()); +// expectLastCall().anyTimes(); +// +// expect(taskRecordSupplier.poll(anyLong())) +// .andReturn(records) +// .once() +// .andReturn(Collections.emptyList()) +// .anyTimes(); +// +// taskRecordSupplier.close(); +// expectLastCall().once(); Capture captured = Capture.newInstance(); - EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); - EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( + expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( null ) ).anyTimes(); - EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); - taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class)); replayAll(); supervisor.start(); @@ -318,11 +301,11 @@ public void testNoInitialState() throws Exception Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); Assert.assertEquals( - getSequenceNumber(res, shardId1, 0), + "0", taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( - getSequenceNumber(res, shardId0, 0), + "0", taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); @@ -342,8 +325,17 @@ public void testNoInitialState() throws Exception public void testMultiTask() throws Exception { supervisor = getSupervisor(1, 2, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -365,7 +357,7 @@ public void testMultiTask() throws Exception Assert.assertEquals(1, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); Assert.assertEquals(1, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); Assert.assertEquals( - getSequenceNumber(res, shardId1, 0), + "0", task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( @@ -377,7 +369,7 @@ public void testMultiTask() throws Exception Assert.assertEquals(1, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); Assert.assertEquals(1, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); Assert.assertEquals( - getSequenceNumber(res, shardId0, 0), + "0", task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); Assert.assertEquals( @@ -390,8 +382,17 @@ public void testMultiTask() throws Exception public void testReplicas() throws Exception { supervisor = getSupervisor(2, 1, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -413,7 +414,7 @@ public void testReplicas() throws Exception Assert.assertEquals(2, task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); Assert.assertEquals(2, task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); Assert.assertEquals( - getSequenceNumber(res, shardId0, 0), + "0", task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); Assert.assertEquals( @@ -421,7 +422,7 @@ public void testReplicas() throws Exception task1.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) ); Assert.assertEquals( - getSequenceNumber(res, shardId1, 0), + "0", task1.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( @@ -433,7 +434,7 @@ public void testReplicas() throws Exception Assert.assertEquals(2, task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().size()); Assert.assertEquals(2, task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().size()); Assert.assertEquals( - getSequenceNumber(res, shardId0, 0), + "0", task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); Assert.assertEquals( @@ -441,7 +442,7 @@ public void testReplicas() throws Exception task2.getIOConfig().getEndPartitions().getPartitionSequenceNumberMap().get(shardId0) ); Assert.assertEquals( - getSequenceNumber(res, shardId1, 0), + "0", task2.getIOConfig().getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( @@ -455,8 +456,17 @@ public void testReplicas() throws Exception public void testLateMessageRejectionPeriod() throws Exception { supervisor = getSupervisor(2, 1, true, "PT1H", new Period("PT1H"), null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -495,8 +505,17 @@ public void testLateMessageRejectionPeriod() throws Exception public void testEarlyMessageRejectionPeriod() throws Exception { supervisor = getSupervisor(2, 1, true, "PT1H", null, new Period("PT1H")); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -540,8 +559,19 @@ public void testEarlyMessageRejectionPeriod() throws Exception public void testDatasourceMetadata() throws Exception { supervisor = getSupervisor(1, 1, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(shard1Partition)).andReturn("2").anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(anyObject())).andReturn("100").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Capture captured = Capture.newInstance(); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -551,9 +581,9 @@ public void testDatasourceMetadata() throws Exception new KinesisDataSourceMetadata( new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )) ) ).anyTimes(); @@ -569,11 +599,11 @@ public void testDatasourceMetadata() throws Exception KinesisIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); Assert.assertEquals( - getSequenceNumber(res, shardId1, 2), + "2", taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( - getSequenceNumber(res, shardId0, 1), + "1", taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); } @@ -582,8 +612,18 @@ public void testDatasourceMetadata() throws Exception public void testBadMetadataOffsets() throws Exception { supervisor = getSupervisor(1, 1, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(anyObject())).andReturn("100").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); @@ -592,9 +632,9 @@ public void testBadMetadataOffsets() throws Exception new KinesisDataSourceMetadata( new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - "00000000000000000000000000000000000000000000000000000000", + "101", shardId0, - "00000000000000000000000000000000000000000000000000000000" + "-1" )) ) ).anyTimes(); @@ -608,8 +648,18 @@ public void testBadMetadataOffsets() throws Exception public void testKillIncompatibleTasks() throws Exception { supervisor = getSupervisor(2, 1, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(anyObject())).andReturn("100").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); // unexpected # of partitions (kill) Task id1 = createKinesisIndexTask( @@ -618,11 +668,11 @@ public void testKillIncompatibleTasks() throws Exception 1, new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )), null, null @@ -635,15 +685,15 @@ public void testKillIncompatibleTasks() throws Exception 0, new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 0), + "0", shardId1, - getSequenceNumber(res, shardId1, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 1), + "1", shardId1, - getSequenceNumber(res, shardId1, 12) + "12" )), null, null @@ -656,15 +706,15 @@ public void testKillIncompatibleTasks() throws Exception 1, new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 0), + "0", shardId1, - getSequenceNumber(res, shardId1, 1) + "1" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 1), + "1", shardId1, - getSequenceNumber(res, shardId1, 11) + "11" )), null, null @@ -677,15 +727,15 @@ public void testKillIncompatibleTasks() throws Exception 2, new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 0), + "0", shardId1, - getSequenceNumber(res, shardId1, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 1), + "1", shardId1, - getSequenceNumber(res, shardId1, 12) + "12" )), null, null @@ -736,9 +786,9 @@ public void testKillIncompatibleTasks() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 0), + "0", shardId1, - getSequenceNumber(res, shardId1, 0) + "0" )); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) @@ -755,8 +805,18 @@ public void testKillIncompatibleTasks() throws Exception public void testKillBadPartitionAssignment() throws Exception { supervisor = getSupervisor(1, 2, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(anyObject())).andReturn("100").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Task id1 = createKinesisIndexTask( "id1", @@ -764,11 +824,11 @@ public void testKillBadPartitionAssignment() throws Exception 0, new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 12) + "12" )), null, null @@ -779,11 +839,11 @@ public void testKillBadPartitionAssignment() throws Exception 1, new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )), null, null @@ -794,15 +854,15 @@ public void testKillBadPartitionAssignment() throws Exception 0, new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 0), + "0", shardId1, - getSequenceNumber(res, shardId1, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 1), + "1", shardId1, - getSequenceNumber(res, shardId1, 12) + "12" )), null, null @@ -813,11 +873,11 @@ public void testKillBadPartitionAssignment() throws Exception 0, new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )), null, null @@ -828,11 +888,11 @@ public void testKillBadPartitionAssignment() throws Exception 0, new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>(stream, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )), null, null @@ -870,9 +930,9 @@ public void testKillBadPartitionAssignment() throws Exception EasyMock.expect(taskClient.stopAsync("id5", false)).andReturn(Futures.immediateFuture((Boolean) null)); TreeMap> checkpoints1 = new TreeMap<>(); - checkpoints1.put(0, ImmutableMap.of(shardId1, getSequenceNumber(res, shardId1, 0))); + checkpoints1.put(0, ImmutableMap.of(shardId1, "0")); TreeMap> checkpoints2 = new TreeMap<>(); - checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); + checkpoints2.put(0, ImmutableMap.of(shardId0, "0")); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(1); @@ -895,8 +955,18 @@ public void testKillBadPartitionAssignment() throws Exception public void testRequeueTaskWhenFailed() throws Exception { supervisor = getSupervisor(2, 2, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(anyObject())).andReturn("100").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -921,13 +991,13 @@ public void testRequeueTaskWhenFailed() throws Exception 0, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0) + "0" ) ); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) @@ -994,8 +1064,17 @@ public void testRequeueTaskWhenFailed() throws Exception public void testRequeueAdoptedTaskWhenFailed() throws Exception { supervisor = getSupervisor(2, 1, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(anyObject())).andReturn("100").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); DateTime now = DateTimes.nowUtc(); DateTime maxi = now.plusMinutes(60); @@ -1005,9 +1084,9 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception 0, new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -1041,9 +1120,9 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) @@ -1117,8 +1196,17 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception public void testQueueNextTasksOnSuccess() throws Exception { supervisor = getSupervisor(2, 2, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(anyObject())).andReturn("100").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -1159,14 +1247,14 @@ public void testQueueNextTasksOnSuccess() throws Exception TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0) + "0" )); // there would be 4 tasks, 2 for each task group EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) @@ -1229,8 +1317,19 @@ public void testBeginPublishAndQueueNextTasks() throws Exception final TaskLocation location = new TaskLocation("testHost", 1234, -1); supervisor = getSupervisor(2, 2, true, "PT1M", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard1Partition)).andReturn("12").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -1278,24 +1377,24 @@ public void testBeginPublishAndQueueNextTasks() throws Exception EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFuture(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 1), + "1", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" ))) .andReturn(Futures.immediateFuture(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ))); EasyMock.expect( taskClient.setEndOffsetsAsync( EasyMock.contains("sequenceName-0"), EasyMock.eq(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )), EasyMock.eq(true) ) @@ -1305,12 +1404,12 @@ public void testBeginPublishAndQueueNextTasks() throws Exception TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0) + "0" )); TreeMap> checkpoints2 = new TreeMap<>(); checkpoints2.put(0, ImmutableMap.of( shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) @@ -1330,16 +1429,16 @@ public void testBeginPublishAndQueueNextTasks() throws Exception Assert.assertEquals(tuningConfig.copyOf(), KinesisIndexTask.getTuningConfig()); KinesisIOConfig taskConfig = KinesisIndexTask.getIOConfig(); - Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); + Assert.assertEquals("sequenceName-1", taskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); Assert.assertEquals(stream, taskConfig.getStartPartitions().getStream()); Assert.assertEquals( - getSequenceNumber(res, shardId1, 3), + "3", taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( - getSequenceNumber(res, shardId0, 1), + "1", taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); } @@ -1351,8 +1450,19 @@ public void testDiscoverExistingPublishingTask() throws Exception final TaskLocation location = new TaskLocation("testHost", 1234, -1); supervisor = getSupervisor(1, 1, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard1Partition)).andReturn("12").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Task task = createKinesisIndexTask( "id1", @@ -1360,9 +1470,9 @@ public void testDiscoverExistingPublishingTask() throws Exception 0, new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -1394,24 +1504,24 @@ public void testDiscoverExistingPublishingTask() throws Exception EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ))); EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )); EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )); EasyMock.expect(taskClient.getCheckpoints(EasyMock.anyString(), EasyMock.anyBoolean())) .andReturn(checkpoints) @@ -1443,15 +1553,15 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertEquals("id1", publishingReport.getId()); Assert.assertEquals(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" ), publishingReport.getStartingOffsets()); Assert.assertEquals(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ), publishingReport.getCurrentOffsets()); KinesisIndexTask capturedTask = captured.getValue(); @@ -1468,11 +1578,11 @@ public void testDiscoverExistingPublishingTask() throws Exception // check that the new task was created with starting sequences matching where the publishing task finished Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); Assert.assertEquals( - getSequenceNumber(res, shardId1, 2), + "2", capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( - getSequenceNumber(res, shardId0, 1), + "1", capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); @@ -1493,8 +1603,18 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() final TaskLocation location = new TaskLocation("testHost", 1234, -1); supervisor = getSupervisor(1, 1, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard1Partition)).andReturn("12").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Task task = createKinesisIndexTask( "id1", @@ -1502,9 +1622,9 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() 0, new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -1536,15 +1656,15 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ))); EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )); EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); @@ -1574,15 +1694,15 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertEquals("id1", publishingReport.getId()); Assert.assertEquals(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" ), publishingReport.getStartingOffsets()); Assert.assertEquals(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ), publishingReport.getCurrentOffsets()); KinesisIndexTask capturedTask = captured.getValue(); @@ -1599,11 +1719,11 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() // check that the new task was created with starting sequences matching where the publishing task finished Assert.assertEquals(stream, capturedTaskConfig.getStartPartitions().getStream()); Assert.assertEquals( - getSequenceNumber(res, shardId1, 2), + "2", capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertEquals( - getSequenceNumber(res, shardId0, 1), + "1", capturedTaskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId0) ); @@ -1626,9 +1746,19 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception final DateTime startTime = DateTimes.nowUtc(); supervisor = getSupervisor(1, 1, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard1Partition)).andReturn("12").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Task id1 = createKinesisIndexTask( "id1", DATASOURCE, @@ -1637,9 +1767,9 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception "stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" ) ), new SeekableStreamPartitions<>("stream", ImmutableMap.of( @@ -1658,9 +1788,9 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception 0, new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -1697,22 +1827,22 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception EasyMock.expect(taskClient.getCurrentOffsetsAsync("id1", false)) .andReturn(Futures.immediateFuture(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ))); EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )); EasyMock.expect(taskClient.getCurrentOffsetsAsync("id2", false)) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 12), + "12", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ))); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); @@ -1721,9 +1851,9 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) @@ -1756,29 +1886,29 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception Assert.assertEquals(startTime, activeReport.getStartTime()); Assert.assertEquals(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ), activeReport.getStartingOffsets()); Assert.assertEquals(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 12), + "12", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ), activeReport.getCurrentOffsets()); Assert.assertEquals("id1", publishingReport.getId()); Assert.assertEquals(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" ), publishingReport.getStartingOffsets()); Assert.assertEquals(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 2), + "2", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ), publishingReport.getCurrentOffsets()); } @@ -1786,8 +1916,18 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception { supervisor = getSupervisor(2, 2, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard1Partition)).andReturn("12").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -1815,14 +1955,14 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception checkpoints1.put(0, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )); TreeMap> checkpoints2 = new TreeMap<>(); - checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); + checkpoints2.put(0, ImmutableMap.of(shardId0, "0")); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); @@ -1855,8 +1995,18 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception final TaskLocation location = new TaskLocation("testHost", 1234, -1); supervisor = getSupervisor(2, 2, true, "PT1M", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard1Partition)).andReturn("12").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -1887,10 +2037,10 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0) + "0" )); TreeMap> checkpoints2 = new TreeMap<>(); - checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); + checkpoints2.put(0, ImmutableMap.of(shardId0, "0")); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); @@ -1934,7 +2084,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception for (Task task : captured.getValues()) { KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); Assert.assertEquals( - getSequenceNumber(res, shardId1, 0), + "0", taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); Assert.assertNull( @@ -1949,8 +2099,18 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception final TaskLocation location = new TaskLocation("testHost", 1234, -1); supervisor = getSupervisor(2, 2, true, "PT1M", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard1Partition)).andReturn("12").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -1981,10 +2141,10 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception TreeMap> checkpoints1 = new TreeMap<>(); checkpoints1.put(0, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0) + "0" )); TreeMap> checkpoints2 = new TreeMap<>(); - checkpoints2.put(0, ImmutableMap.of(shardId0, getSequenceNumber(res, shardId0, 0))); + checkpoints2.put(0, ImmutableMap.of(shardId0, "0")); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints1)) .times(2); @@ -2013,18 +2173,18 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception EasyMock.expect(taskClient.pauseAsync(EasyMock.contains("sequenceName-0"))) .andReturn(Futures.immediateFuture(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 1) + "1" ))) .andReturn(Futures.immediateFuture(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3) + "3" ))); EasyMock.expect( taskClient.setEndOffsetsAsync( EasyMock.contains("sequenceName-0"), EasyMock.eq(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3) + "3" )), EasyMock.eq(true) ) @@ -2045,7 +2205,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception for (Task task : captured.getValues()) { KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); Assert.assertEquals( - getSequenceNumber(res, shardId1, 0), + "0", taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) ); } @@ -2061,6 +2221,9 @@ public void testStopNotStarted() @Test public void testStop() { + supervisorRecordSupplier.close(); + expectLastCall().anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); taskClient.close(); taskRunner.unregisterListener(StringUtils.format("KinesisSupervisor-%s", DATASOURCE)); @@ -2081,8 +2244,18 @@ public void testStopGracefully() throws Exception final DateTime startTime = DateTimes.nowUtc(); supervisor = getSupervisor(2, 1, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard1Partition)).andReturn("12").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Task id1 = createKinesisIndexTask( "id1", @@ -2090,9 +2263,9 @@ public void testStopGracefully() throws Exception 0, new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -2110,9 +2283,9 @@ public void testStopGracefully() throws Exception 0, new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -2130,9 +2303,9 @@ public void testStopGracefully() throws Exception 0, new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -2173,18 +2346,18 @@ public void testStopGracefully() throws Exception EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )); // getCheckpoints will not be called for id1 as it is in publishing state TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) @@ -2205,15 +2378,15 @@ public void testStopGracefully() throws Exception EasyMock.expect(taskClient.pauseAsync("id2")) .andReturn(Futures.immediateFuture((Map) ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 12), + "12", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ))); EasyMock.expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 12), + "12", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ), true)) .andReturn(Futures.immediateFuture(true)); taskQueue.shutdown("id3", "Killing task for graceful shutdown"); @@ -2230,6 +2403,8 @@ public void testStopGracefully() throws Exception @Test public void testResetNoTasks() throws Exception { + expect(supervisorRecordSupplier.getPartitionIds(anyObject())).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); @@ -2239,6 +2414,7 @@ public void testResetNoTasks() throws Exception supervisor = getSupervisor(1, 1, true, "PT1H", null, null); + supervisor.start(); supervisor.runInternal(); verifyAll(); @@ -2255,6 +2431,7 @@ public void testResetNoTasks() throws Exception @Test public void testResetDataSourceMetadata() throws Exception { + expect(supervisorRecordSupplier.getPartitionIds(anyObject())).andReturn(Collections.emptySet()).anyTimes(); supervisor = getSupervisor(1, 1, true, "PT1H", null, null); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); @@ -2322,6 +2499,7 @@ public void testResetDataSourceMetadata() throws Exception @Test public void testResetNoDataSourceMetadata() throws Exception { + expect(supervisorRecordSupplier.getPartitionIds(anyObject())).andReturn(Collections.emptySet()).anyTimes(); supervisor = getSupervisor(1, 1, true, "PT1H", null, null); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); @@ -2359,8 +2537,18 @@ public void testResetRunningTasks() throws Exception final DateTime startTime = DateTimes.nowUtc(); supervisor = getSupervisor(2, 1, true, "PT1H", null, null); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard1Partition)).andReturn("12").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); Task id1 = createKinesisIndexTask( "id1", @@ -2369,11 +2557,11 @@ public void testResetRunningTasks() throws Exception new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -2392,11 +2580,11 @@ public void testResetRunningTasks() throws Exception new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -2415,11 +2603,11 @@ public void testResetRunningTasks() throws Exception new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -2461,22 +2649,22 @@ public void testResetRunningTasks() throws Exception EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )); TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) @@ -2569,6 +2757,18 @@ public void testNoDataIngestionTasks() throws Exception null ); + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard1Partition)).andReturn("12").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes(); @@ -2632,11 +2832,13 @@ public void testNoDataIngestionTasks() throws Exception @Test(timeout = 60_000L) public void testCheckpointForInactiveTaskGroup() throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException, ExecutionException, - TimeoutException, JsonProcessingException + TimeoutException, JsonProcessingException, NoSuchMethodException, IllegalAccessException, + ClassNotFoundException { supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); //not adding any events - final Task id1 = createKinesisIndexTask( + final Task id1; + id1 = createKinesisIndexTask( "id1", DATASOURCE, 0, @@ -2703,6 +2905,19 @@ public void testCheckpointForInactiveTaskGroup() workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard1Partition)).andReturn("12").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes(); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); @@ -2770,9 +2985,25 @@ public void testCheckpointForInactiveTaskGroup() } @Test(timeout = 60_000L) - public void testCheckpointForUnknownTaskGroup() throws InterruptedException + public void testCheckpointForUnknownTaskGroup() + throws InterruptedException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException { supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); + + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard1Partition)).andReturn("12").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + //not adding any events final Task id1 = createKinesisIndexTask( "id1", @@ -2880,7 +3111,8 @@ public void testCheckpointForUnknownTaskGroup() throws InterruptedException @Test(timeout = 60_000L) public void testCheckpointWithNullTaskGroupId() - throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException + throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException, NoSuchMethodException, + IllegalAccessException, ClassNotFoundException { supervisor = getSupervisor(1, 3, true, "PT1S", null, null, false); //not adding any events @@ -2932,6 +3164,7 @@ public void testCheckpointWithNullTaskGroupId() null ); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(Collections.emptySet()).anyTimes(); 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(); @@ -2996,6 +3229,7 @@ public void testSuspendedNoRunningTasks() throws Exception { supervisor = getSupervisor(1, 1, true, "PT1H", null, null, true); + expect(supervisorRecordSupplier.getPartitionIds(anyObject())).andReturn(Collections.emptySet()).anyTimes(); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); @@ -3027,8 +3261,20 @@ public void testSuspendedRunningTasks() throws Exception final DateTime startTime = DateTimes.nowUtc(); supervisor = getSupervisor(2, 1, true, "PT1H", null, null, true); - AmazonKinesis kinesis = getKinesisClientInstance(); - List res = insertData(kinesis, generateRecordsRequests(stream)); + + supervisorRecordSupplier.assign(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getPartitionIds(stream)).andReturn(ImmutableSet.of(shardId1, shardId0)).anyTimes(); + expect(supervisorRecordSupplier.getAssignment()).andReturn(ImmutableSet.of(shard1Partition, shard0Partition)) + .anyTimes(); + supervisorRecordSupplier.seekToLatest(anyObject()); + expectLastCall().anyTimes(); + expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard1Partition)).andReturn("12").anyTimes(); + expect(supervisorRecordSupplier.getLatestSequenceNumber(shard0Partition)).andReturn("1").anyTimes(); + supervisorRecordSupplier.seek(anyObject(), anyString()); + expectLastCall().anyTimes(); + Task id1 = createKinesisIndexTask( "id1", @@ -3036,9 +3282,9 @@ public void testSuspendedRunningTasks() throws Exception 0, new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 0), + "0", shardId0, - getSequenceNumber(res, shardId0, 0) + "0" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -3056,9 +3302,9 @@ public void testSuspendedRunningTasks() throws Exception 0, new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -3076,9 +3322,9 @@ public void testSuspendedRunningTasks() throws Exception 0, new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )), new SeekableStreamPartitions<>("stream", ImmutableMap.of( shardId1, @@ -3119,18 +3365,18 @@ public void testSuspendedRunningTasks() throws Exception EasyMock.expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime)); EasyMock.expect(taskClient.getEndOffsets("id1")).andReturn(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )); // getCheckpoints will not be called for id1 as it is in publishing state TreeMap> checkpoints = new TreeMap<>(); checkpoints.put(0, ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 3), + "3", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" )); EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), EasyMock.anyBoolean())) .andReturn(Futures.immediateFuture(checkpoints)) @@ -3144,15 +3390,15 @@ public void testSuspendedRunningTasks() throws Exception EasyMock.expect(taskClient.pauseAsync("id2")) .andReturn(Futures.immediateFuture(ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 12), + "12", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ))); EasyMock.expect(taskClient.setEndOffsetsAsync("id2", ImmutableMap.of( shardId1, - getSequenceNumber(res, shardId1, 12), + "12", shardId0, - getSequenceNumber(res, shardId0, 1) + "1" ), true)) .andReturn(Futures.immediateFuture(true)); taskQueue.shutdown("id3", "Killing task for graceful shutdown"); @@ -3169,6 +3415,7 @@ public void testSuspendedRunningTasks() throws Exception @Test public void testResetSuspended() throws Exception { + expect(supervisorRecordSupplier.getPartitionIds(anyObject())).andReturn(Collections.emptySet()).anyTimes(); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); @@ -3234,33 +3481,6 @@ public void testGetCurrentTotalStats() Assert.assertEquals(ImmutableMap.of("task2", ImmutableMap.of("prop2", "val2")), stats.get("1")); } - - private static List insertData( - AmazonKinesis kinesis, - PutRecordsRequest req - ) - { - PutRecordsResult res = kinesis.putRecords(req); - Assert.assertTrue(isResponseOk(res.getSdkHttpMetadata())); - Assert.assertEquals((int) res.getFailedRecordCount(), 0); - return res.getRecords(); - } - - private AmazonKinesis getKinesisClientInstance() throws InterruptedException - { - AmazonKinesis kinesis = cloud.localstack.TestUtils.getClientKinesis(); - SdkHttpMetadata createRes = kinesis.createStream(stream, 2).getSdkHttpMetadata(); - // sleep required because of kinesalite - Thread.sleep(500); - Assert.assertTrue(isResponseOk(createRes)); - return kinesis; - } - - private static boolean isResponseOk(SdkHttpMetadata sdkHttpMetadata) - { - return sdkHttpMetadata.getHttpStatusCode() == 200; - } - private KinesisSupervisor getSupervisor( int replicas, int taskCount, @@ -3399,30 +3619,17 @@ private static DataSchema getDataSchema(String dataSource) ); } - private static PutRecordsRequestEntry generateRequestEntry(String partition, byte[] data) - { - return new PutRecordsRequestEntry().withPartitionKey(partition) - .withData(ByteBuffer.wrap(data)); - } - - private static PutRecordsRequest generateRecordsRequests(String stream, int first, int last) - { - return new PutRecordsRequest() - .withStreamName(stream) - .withRecords(records.subList(first, last)); - } - - private static PutRecordsRequest generateRecordsRequests(String stream) - { - return new PutRecordsRequest() - .withStreamName(stream) - .withRecords(records); - } - - private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) + private static List JB( + String timestamp, + String dim1, + String dim2, + String dimLong, + String dimFloat, + String met1 + ) { try { - return new ObjectMapper().writeValueAsBytes( + return Collections.singletonList(new ObjectMapper().writeValueAsBytes( ImmutableMap.builder() .put("timestamp", timestamp) .put("dim1", dim1) @@ -3431,22 +3638,13 @@ private static byte[] JB(String timestamp, String dim1, String dim2, String dimL .put("dimFloat", dimFloat) .put("met1", met1) .build() - ); + )); } catch (Exception e) { throw Throwables.propagate(e); } } - private static String getSequenceNumber(List entries, String shardId, int stream) - { - List sortedEntries = entries.stream() - .filter(e -> e.getShardId().equals(shardId)) - .sorted(Comparator.comparing(e -> KinesisSequenceNumber.of(e.getSequenceNumber()))) - .collect(Collectors.toList()); - return sortedEntries.get(stream).getSequenceNumber(); - } - private KinesisIndexTask createKinesisIndexTask( String id, String dataSource, @@ -3455,7 +3653,7 @@ private KinesisIndexTask createKinesisIndexTask( SeekableStreamPartitions endPartitions, DateTime minimumMessageTime, DateTime maximumMessageTime - ) + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { return new KinesisIndexTask( id, @@ -3600,8 +3798,10 @@ public String getDataSource() } - private static class TestableKinesisSupervisor extends KinesisSupervisor + private class TestableKinesisSupervisor extends KinesisSupervisor { + private final KinesisSupervisorSpec spec; + public TestableKinesisSupervisor( TaskStorage taskStorage, TaskMaster taskMaster, @@ -3621,6 +3821,7 @@ public TestableKinesisSupervisor( spec, rowIngestionMetersFactory ); + this.spec = spec; } @Override @@ -3633,6 +3834,13 @@ protected String generateSequenceName( final int groupId = getTaskGroupIdForPartition(startPartitions.keySet().iterator().next()); return StringUtils.format("sequenceName-%d", groupId); } - } + @Override + protected RecordSupplier setupRecordSupplier() + { + return supervisorRecordSupplier; + } + + + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index ffea28a32862..d159e28dcdaa 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -38,7 +38,6 @@ import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; -import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; @@ -81,7 +80,6 @@ public abstract class SeekableStreamIndexTask exten protected final RowIngestionMetersFactory rowIngestionMetersFactory; protected CircularBuffer savedParseExceptions; - @JsonCreator public SeekableStreamIndexTask( @JsonProperty("id") String id, @@ -94,7 +92,7 @@ public SeekableStreamIndexTask( @JacksonInject AuthorizerMapper authorizerMapper, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, String type - ) + ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { super( id == null ? makeTaskId(dataSchema.getDataSource(), RANDOM.nextInt(), type) : id, @@ -245,9 +243,6 @@ public StreamAppenderatorDriver newDriver( ); } - protected abstract RecordSupplier newRecordSupplier() - throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException; - public boolean withinMinMaxRecordTime(final InputRow row) { final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent() @@ -283,7 +278,8 @@ public boolean withinMinMaxRecordTime(final InputRow row) return !beforeMinimumMessageTime && !afterMaximumMessageTime; } - protected abstract SeekableStreamIndexTaskRunner createTaskRunner(); + protected abstract SeekableStreamIndexTaskRunner createTaskRunner() + throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException; @VisibleForTesting public Appenderator getAppenderator() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index ae214c184704..e798a1f1ea17 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -169,7 +169,7 @@ public enum Status private final Condition hasPaused = pauseLock.newCondition(); private final Condition shouldResume = pauseLock.newCondition(); - protected final AtomicBoolean stopRequested = new AtomicBoolean(false); + public final AtomicBoolean stopRequested = new AtomicBoolean(false); private final AtomicBoolean publishOnStop = new AtomicBoolean(false); // [statusLock] is used to synchronize the Jetty thread calling stopGracefully() with the main run thread. It prevents @@ -185,6 +185,7 @@ public enum Status protected final Condition isAwaitingRetry = pollRetryLock.newCondition(); private final SeekableStreamIndexTask task; + private final RecordSupplier recordSupplier; private final SeekableStreamIOConfig ioConfig; private final SeekableStreamTuningConfig tuningConfig; private final InputRowParser parser; @@ -219,6 +220,7 @@ public enum Status public SeekableStreamIndexTaskRunner( final SeekableStreamIndexTask task, + final RecordSupplier recordSupplier, final InputRowParser parser, final AuthorizerMapper authorizerMapper, final Optional chatHandlerProvider, @@ -227,7 +229,9 @@ public SeekableStreamIndexTaskRunner( final boolean isSkipSegmentLineageCheck ) { + Preconditions.checkNotNull(task); this.task = task; + this.recordSupplier = recordSupplier; this.ioConfig = task.getIOConfig(); this.tuningConfig = task.getTuningConfig(); this.parser = parser; @@ -346,7 +350,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception ); Throwable caughtExceptionOuter = null; - try (final RecordSupplier recordSupplier = task.newRecordSupplier()) { + try { toolbox.getDataSegmentServerAnnouncer().announce(); toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java index c1900300e22c..35ab93af2b0a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java @@ -89,6 +89,7 @@ public boolean equals(Object o) return Objects.equals(stream, that.stream) && Objects.equals(partitionId, that.partitionId) && Objects.equals(sequenceNumber, that.sequenceNumber); + } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index dfe004db48c8..2c9860932323 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -220,7 +220,8 @@ public String toString() */ private interface Notice { - void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException; + void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException, + NoSuchMethodException, IllegalAccessException, ClassNotFoundException; } private static class StatsFromTaskResult @@ -260,7 +261,8 @@ public Map getStats() private class RunNotice implements Notice { @Override - public void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException + public void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException, + NoSuchMethodException, IllegalAccessException, ClassNotFoundException { long nowTime = System.currentTimeMillis(); if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { @@ -1015,7 +1017,8 @@ protected void addTaskGroupToPendingCompletionTaskGroup( @VisibleForTesting protected void runInternal() - throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException + throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException, NoSuchMethodException, + IllegalAccessException, ClassNotFoundException { possiblyRegisterListener(); updatePartitionDataFromStream(); @@ -2223,7 +2226,8 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); } - private void createNewTasks() throws JsonProcessingException + private void createNewTasks() + throws JsonProcessingException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException { // update the checkpoints in the taskGroup to latest ones so that new tasks do not read what is already published verifyAndMergeCheckpoints( @@ -2451,7 +2455,8 @@ private SequenceType getOffsetFromStreamForPartition(PartitionType partition, bo } } - private void createTasksForGroup(int groupId, int replicas) throws JsonProcessingException + private void createTasksForGroup(int groupId, int replicas) + throws JsonProcessingException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException { TaskGroup group = activelyReadingTaskGroups.get(groupId); Map startPartitions = group.startingSequences; @@ -2682,7 +2687,7 @@ protected abstract List> cr SeekableStreamIOConfig taskIoConfig, SeekableStreamTuningConfig taskTuningConfig, RowIngestionMetersFactory rowIngestionMetersFactory - ) throws JsonProcessingException; + ) throws JsonProcessingException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException; /** * calculates the taskgroup id that the given partition belongs to. From 6eeac457283aee7892d9e74db11970c09d86eaf7 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Fri, 30 Nov 2018 14:50:59 -0800 Subject: [PATCH 56/87] PR comments --- ...ementalPublishingKafkaIndexTaskRunner.java | 13 +-- .../kafka/LegacyKafkaIndexTaskRunner.java | 7 +- .../kinesis/KinesisIndexTaskRunner.java | 16 +-- .../kinesis/KinesisRecordSupplier.java | 7 +- .../indexing/kinesis/KinesisTuningConfig.java | 2 +- .../kinesis/supervisor/KinesisSupervisor.java | 2 +- .../kinesis/KinesisIndexTaskTest.java | 2 +- .../kinesis/KinesisRecordSupplierTest.java | 4 +- .../kinesis/KinesisTuningConfigTest.java | 2 +- .../supervisor/KinesisSupervisorTest.java | 19 ---- .../SeekableStreamIndexTask.java | 8 +- .../SeekableStreamIndexTaskRunner.java | 103 +++++++----------- .../SeekableStreamPartitions.java | 18 +-- .../seekablestream/common/RecordSupplier.java | 2 +- pom.xml | 2 +- 15 files changed, 66 insertions(+), 141 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 02cea4f1a195..0215718a459f 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; -import com.google.common.collect.ImmutableSet; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -82,17 +81,14 @@ public IncrementalPublishingKafkaIndexTaskRunner( authorizerMapper, chatHandlerProvider, savedParseExceptions, - rowIngestionMetersFactory, - true + rowIngestionMetersFactory ); this.task = task; this.tuningConfig = task.getTuningConfig(); } @Override - protected Long getNextSequenceNumber( - @NotNull Long sequenceNumber - ) + protected Long getSequenceNumberToStoreAfterRead(@NotNull Long sequenceNumber) { return sequenceNumber + 1; } @@ -150,7 +146,6 @@ private void possiblyResetOffsetsOrWait( topicPartition.topic(), topicPartition.partition() ); - recordSupplier.seekToEarliest(ImmutableSet.of(streamPartition)); final Long leastAvailableOffset = recordSupplier.getEarliestSequenceNumber(streamPartition); if (leastAvailableOffset == null) { throw new ISE( @@ -213,7 +208,7 @@ protected Type getRunnerType() } @Override - protected SequenceMetadata createSequenceMetaData( + protected SequenceMetadata createSequenceMetadata( int sequenceId, String sequenceName, Map startOffsets, @@ -256,7 +251,7 @@ protected TreeMap> getCheckPointsFromContext( private class KafkaSequenceMetaData extends SequenceMetadata { - public KafkaSequenceMetaData( + KafkaSequenceMetaData( int sequenceId, String sequenceName, Map startOffsets, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index ed100828d675..99b14d01d6de 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -196,8 +196,7 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner> getCheckPointsFromContext( } @Override - protected Long getNextSequenceNumber(Long sequenceNumber) + protected Long getSequenceNumberToStoreAfterRead(Long sequenceNumber) { throw new UnsupportedOperationException(); } @@ -1227,7 +1226,7 @@ public DateTime getStartTime(@Context final HttpServletRequest req) } @Override - protected SequenceMetadata createSequenceMetaData( + protected SequenceMetadata createSequenceMetadata( int sequenceId, String sequenceName, Map startOffsets, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index 195edd066fef..fc6b13687188 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -45,16 +45,13 @@ import java.util.Map; import java.util.Set; import java.util.TreeMap; -import java.util.concurrent.CopyOnWriteArrayList; public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner { private static final EmittingLogger log = new EmittingLogger(KinesisIndexTaskRunner.class); private static final long POLL_TIMEOUT = 100; - private volatile CopyOnWriteArrayList sequences; - - public KinesisIndexTaskRunner( + KinesisIndexTaskRunner( KinesisIndexTask task, KinesisRecordSupplier recordSupplier, InputRowParser parser, @@ -71,16 +68,13 @@ public KinesisIndexTaskRunner( authorizerMapper, chatHandlerProvider, savedParseExceptions, - rowIngestionMetersFactory, - false + rowIngestionMetersFactory ); } @Override - protected String getNextSequenceNumber( - String sequenceNumber - ) + protected String getSequenceNumberToStoreAfterRead(String sequenceNumber) { return sequenceNumber; } @@ -129,7 +123,7 @@ protected Type getRunnerType() } @Override - protected SequenceMetadata createSequenceMetaData( + protected SequenceMetadata createSequenceMetadata( int sequenceId, String sequenceName, Map startOffsets, @@ -172,7 +166,7 @@ protected TreeMap> getCheckPointsFromContext( private class KinesisSequenceMetaData extends SequenceMetadata { - public KinesisSequenceMetaData( + KinesisSequenceMetaData( int sequenceId, String sequenceName, Map startOffsets, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 6135953e59b9..0d48700adf3e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -309,11 +309,11 @@ public KinesisRecordSupplier( if (deaggregate) { try { - Class KCLUserRecordclass = Class.forName("com.amazonaws.services.kinesis.clientlibrary.types.UserRecord"); + Class kclUserRecordclass = Class.forName("com.amazonaws.services.kinesis.clientlibrary.types.UserRecord"); MethodHandles.Lookup lookup = MethodHandles.publicLookup(); - Method deaggregateMethod = KCLUserRecordclass.getMethod("deaggregate", List.class); - Method getDataMethod = KCLUserRecordclass.getMethod("getData"); + Method deaggregateMethod = kclUserRecordclass.getMethod("deaggregate", List.class); + Method getDataMethod = kclUserRecordclass.getMethod("getData"); deaggregateHandle = lookup.unreflect(deaggregateMethod); getDataHandle = lookup.unreflect(getDataMethod); @@ -329,7 +329,6 @@ public KinesisRecordSupplier( getDataHandle = null; } - log.info( "Creating fetch thread pool of size [%d] (Runtime.availableProcessors=%d)", fetchThreads, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java index 51288732b3ff..753e8c8809c7 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java @@ -34,7 +34,7 @@ public class KinesisTuningConfig extends SeekableStreamTuningConfig { - private static final int DEFAULT_RECORD_BUFFER_SIZE = 100000; + private static final int DEFAULT_RECORD_BUFFER_SIZE = 10000; private static final int DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT = 5000; private static final int DEFAULT_RECORD_BUFFER_FULL_WAIT = 5000; private static final int DEFAULT_FETCH_SEQUENCE_NUMBER_TIMEOUT = 20000; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 005bf490e0a9..7ba29be4d035 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -71,7 +71,7 @@ * tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of * Kinesis sequences. *

- * the Kinesis supervisor does not yet support incremental handoff and emitLag + * the Kinesis supervisor does not yet support lag calculations */ public class KinesisSupervisor extends SeekableStreamSupervisor { diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index c35b4e92121d..cdac31a16f8c 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -3001,7 +3001,7 @@ private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOExcep } @JsonTypeName("index_kinesis") - private class TestableKinesisIndexTask extends KinesisIndexTask + private static class TestableKinesisIndexTask extends KinesisIndexTask { @JsonCreator public TestableKinesisIndexTask( diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index c454d49432b7..dd696ca89262 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -72,7 +72,8 @@ public class KinesisRecordSupplierTest extends EasyMockSupport private static GetRecordsResult getRecordsResult0; private static GetRecordsResult getRecordsResult1; private static StreamDescription streamDescription; - private static Shard shard0, shard1; + private static Shard shard0; + private static Shard shard1; private static KinesisRecordSupplier recordSupplier; private static List shard1Records = ImmutableList.of( new Record().withData(JB("2011", "d", "y", "10", "20.0", "1.0")).withSequenceNumber("0"), @@ -118,7 +119,6 @@ public class KinesisRecordSupplierTest extends EasyMockSupport Collectors .toList())) .build(); - ; private static ByteBuffer JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) { diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java index a2629d2edaf1..67a7f3d51666 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java @@ -71,7 +71,7 @@ public void testSerdeWithDefaults() throws Exception Assert.assertTrue(config.getBuildV9Directly()); Assert.assertFalse(config.isReportParseExceptions()); Assert.assertEquals(0, config.getHandoffConditionTimeout()); - Assert.assertEquals(100000, config.getRecordBufferSize()); + Assert.assertEquals(10000, config.getRecordBufferSize()); Assert.assertEquals(5000, config.getRecordBufferOfferTimeout()); Assert.assertEquals(5000, config.getRecordBufferFullWait()); Assert.assertEquals(20000, config.getFetchSequenceNumberTimeout()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 6e7b53d3f7f1..a632c7af6253 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -252,25 +252,6 @@ public void testNoInitialState() throws Exception expect(supervisorRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); supervisorRecordSupplier.seek(anyObject(), anyString()); expectLastCall().anyTimes(); - //supervisorRecordSupplier.close(); - //expectLastCall().once(); - -// taskRecordSupplier.assign(anyObject()); -// expectLastCall().anyTimes(); -// -// expect(taskRecordSupplier.getEarliestSequenceNumber(anyObject())).andReturn("0").anyTimes(); -// -// taskRecordSupplier.seek(anyObject(), anyString()); -// expectLastCall().anyTimes(); -// -// expect(taskRecordSupplier.poll(anyLong())) -// .andReturn(records) -// .once() -// .andReturn(Collections.emptyList()) -// .anyTimes(); -// -// taskRecordSupplier.close(); -// expectLastCall().once(); Capture captured = Capture.newInstance(); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index d159e28dcdaa..18f80d4131e4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -64,21 +64,21 @@ public abstract class SeekableStreamIndexTask extends AbstractTask implements ChatHandler { - private final EmittingLogger log = new EmittingLogger(this.getClass()); public static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15; - private static final Random RANDOM = ThreadLocalRandom.current(); + + private final EmittingLogger log = new EmittingLogger(this.getClass()); + private final SeekableStreamIndexTaskRunner runner; protected final DataSchema dataSchema; protected final InputRowParser parser; protected final SeekableStreamTuningConfig tuningConfig; protected final SeekableStreamIOConfig ioConfig; protected final Optional chatHandlerProvider; - private final SeekableStreamIndexTaskRunner runner; protected final String type; protected final Map context; protected final AuthorizerMapper authorizerMapper; protected final RowIngestionMetersFactory rowIngestionMetersFactory; - protected CircularBuffer savedParseExceptions; + protected final CircularBuffer savedParseExceptions; @JsonCreator public SeekableStreamIndexTask( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index e798a1f1ea17..a5e1677dbca8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -125,8 +125,7 @@ import java.util.stream.Collectors; /** - * Interface for abstracting the indexing task run logic. Only used by Kafka indexing tasks, - * but will also be used by Kinesis indexing tasks once implemented + * Interface for abstracting the indexing task run logic. * * @param Partition Number Type * @param Sequence Number Type @@ -142,6 +141,13 @@ public enum Status PUBLISHING } + protected enum Type + { + KAFKA, + KINESIS + } + + private final EmittingLogger log = new EmittingLogger(this.getClass()); private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; private static final String METADATA_PUBLISH_PARTITIONS = "publishPartitions"; @@ -201,9 +207,6 @@ public enum Status private final Map initialOffsetsSnapshot = new HashMap<>(); private final Set exclusiveStartingPartitions = new HashSet<>(); - // true for kafka, falsse for kinesis - private final boolean isSkipSegmentLineageCheck; - private volatile DateTime startTime; private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread) private volatile TaskToolbox toolbox; @@ -225,8 +228,7 @@ public SeekableStreamIndexTaskRunner( final AuthorizerMapper authorizerMapper, final Optional chatHandlerProvider, final CircularBuffer savedParseExceptions, - final RowIngestionMetersFactory rowIngestionMetersFactory, - final boolean isSkipSegmentLineageCheck + final RowIngestionMetersFactory rowIngestionMetersFactory ) { Preconditions.checkNotNull(task); @@ -240,8 +242,6 @@ public SeekableStreamIndexTaskRunner( this.savedParseExceptions = savedParseExceptions; this.stream = ioConfig.getStartPartitions().getStream(); this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters(); - this.isSkipSegmentLineageCheck = isSkipSegmentLineageCheck; - this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndPartitions().getPartitionSequenceNumberMap()); this.sequences = new CopyOnWriteArrayList<>(); this.ingestionState = IngestionState.NOT_STARTED; @@ -283,7 +283,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception Map.Entry> previous = sequenceOffsets.next(); while (sequenceOffsets.hasNext()) { Map.Entry> current = sequenceOffsets.next(); - sequences.add(createSequenceMetaData( + sequences.add(createSequenceMetadata( previous.getKey(), StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), previous.getValue(), @@ -294,7 +294,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception previous = current; exclusive = true; } - sequences.add(createSequenceMetaData( + sequences.add(createSequenceMetadata( previous.getKey(), StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), previous.getValue(), @@ -303,7 +303,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception exclusive ? previous.getValue().keySet() : null )); } else { - sequences.add(createSequenceMetaData( + sequences.add(createSequenceMetadata( 0, StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0), ioConfig.getStartPartitions().getPartitionSequenceNumberMap(), @@ -498,7 +498,7 @@ public void run() maybePersistAndPublishSequences(committerSupplier); - // calling getReocrd() ensures that excpetions specific to kafka/kinesis like OffsetOutOfRangeException + // calling getRecord() ensures that exceptions specific to kafka/kinesis like OffsetOutOfRangeException // are handled in the subclasses List> records = getRecords(recordSupplier, toolbox); @@ -515,7 +515,7 @@ public void run() // for the first message we receive, check that we were given a message with a sequenceNumber that matches our // expected starting sequenceNumber - if (!verifyInitialRecordAndSkipExclusiveParition(record, initialOffsetsSnapshot)) { + if (!verifyInitialRecordAndSkipExclusivePartition(record, initialOffsetsSnapshot)) { continue; } @@ -535,15 +535,14 @@ public void run() createSequenceNumber(endOffsets.get(record.getPartitionId()))) <= 0) { - if (!record.getSequenceNumber().equals(currOffsets.get(record.getPartitionId()))) { - if (!ioConfig.isSkipOffsetGaps()) { - throw new ISE( - "WTF?! Got sequence[%s] after sequence[%s] in partition[%s].", - record.getSequenceNumber(), - currOffsets.get(record.getPartitionId()), - record.getPartitionId() - ); - } + if (!record.getSequenceNumber().equals(currOffsets.get(record.getPartitionId())) + && !ioConfig.isSkipOffsetGaps()) { + throw new ISE( + "WTF?! Got sequence[%s] after sequence[%s] in partition[%s].", + record.getSequenceNumber(), + currOffsets.get(record.getPartitionId()), + record.getPartitionId() + ); } try { @@ -580,8 +579,7 @@ public void run() row, sequenceToUse.getSequenceName(), committerSupplier, - // true for kafka, false for kinesis - isSkipSegmentLineageCheck, + true, // do not allow incremental persists to happen until all the rows from this batch // of rows are indexed false @@ -637,8 +635,8 @@ public void onFailure(@ParametersAreNonnullByDefault Throwable t) // in kafka, we can easily get the next offset by adding 1, but for kinesis, there's no way // to get the next sequence number without having to make an expensive api call. So the behavior - // here for kafka is to +1 while for kinesis we simply save the current sequence numberkkkkk - currOffsets.put(record.getPartitionId(), getNextSequenceNumber(record.getSequenceNumber())); + // here for kafka is to +1 while for kinesis we simply save the current sequence number + currOffsets.put(record.getPartitionId(), getSequenceNumberToStoreAfterRead(record.getSequenceNumber())); } if ((currOffsets.get(record.getPartitionId()).equals(endOffsets.get(record.getPartitionId())) @@ -1495,7 +1493,7 @@ public Response setEndOffsets( exclusiveStartingPartitions.addAll(exclusivePartitions); // create new sequence - final SequenceMetadata newSequence = createSequenceMetaData( + final SequenceMetadata newSequence = createSequenceMetadata( latestSequence.getSequenceId() + 1, StringUtils.format("%s_%d", ioConfig.getBaseSequenceName(), latestSequence.getSequenceId() + 1), sequenceNumbers, @@ -1656,7 +1654,7 @@ public DateTime getStartTime(@Context final HttpServletRequest req) return startTime; } - protected abstract SequenceMetadata createSequenceMetaData( + protected abstract SequenceMetadata createSequenceMetadata( int sequenceId, String sequenceName, Map startOffsets, @@ -1667,20 +1665,20 @@ protected abstract SequenceMetadata createSequenceMetaData( protected abstract class SequenceMetadata { + private final int sequenceId; + private final String sequenceName; + private final Set exclusiveStartPartitions; + private final Set assignments; + private final boolean sentinel; + private boolean checkpointed; + /** * Lock for accessing {@link #endOffsets} and {@link #checkpointed}. This lock is required because * {@link #setEndOffsets)} can be called by both the main thread and the HTTP thread. */ protected final ReentrantLock lock = new ReentrantLock(); - - private final int sequenceId; - private final String sequenceName; protected final Map startOffsets; protected final Map endOffsets; - private final Set exclusiveStartPartitions; - private final Set assignments; - private final boolean sentinel; - private boolean checkpointed; @JsonCreator @@ -1690,7 +1688,7 @@ public SequenceMetadata( @JsonProperty("startOffsets") Map startOffsets, @JsonProperty("endOffsets") Map endOffsets, @JsonProperty("checkpointed") boolean checkpointed, - @JsonProperty("exclusiveStartParittions") Set exclusiveStartPartitions + @JsonProperty("exclusiveStartPartitions") Set exclusiveStartPartitions ) { Preconditions.checkNotNull(sequenceName); @@ -1799,25 +1797,6 @@ protected boolean isOpen() protected abstract boolean canHandle(OrderedPartitionableRecord record); - boolean acanHandle(OrderedPartitionableRecord record) - { - lock.lock(); - try { - final OrderedSequenceNumber partitionEndOffset = createSequenceNumber(endOffsets.get(record.getPartitionId())); - final OrderedSequenceNumber partitionStartOffset = createSequenceNumber(startOffsets.get(record.getPartitionId())); - final OrderedSequenceNumber recordOffset = createSequenceNumber(record.getSequenceNumber()); - return isOpen() - && recordOffset != null - && partitionEndOffset != null - && partitionStartOffset != null - && recordOffset.compareTo(partitionStartOffset) >= 0 - && recordOffset.compareTo(partitionEndOffset) < 0; - } - finally { - lock.unlock(); - } - } - @Override public String toString() { @@ -1937,7 +1916,7 @@ TransactionalSegmentPublisher createPublisher(TaskToolbox toolbox, boolean useTr } - private boolean verifyInitialRecordAndSkipExclusiveParition( + private boolean verifyInitialRecordAndSkipExclusivePartition( final OrderedPartitionableRecord record, final Map intialSequenceSnapshot ) @@ -1979,9 +1958,7 @@ protected abstract TreeMap> getCheckPo SeekableStreamIndexTask task ) throws IOException; - protected abstract SequenceType getNextSequenceNumber( - SequenceType sequenceNumber - ); + protected abstract SequenceType getSequenceNumberToStoreAfterRead(SequenceType sequenceNumber); protected abstract SeekableStreamPartitions createSeekableStreamPartitions( ObjectMapper mapper, @@ -2001,10 +1978,4 @@ protected abstract SeekableStreamDataSourceMetadata protected abstract OrderedSequenceNumber createSequenceNumber(SequenceType sequenceNumber); protected abstract Type getRunnerType(); - - protected enum Type - { - KAFKA, - KINESIS - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index f270198cb2ba..7dcbaa8001cb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; -import com.google.errorprone.annotations.DoNotCall; import javax.validation.constraints.NotNull; import java.util.Map; @@ -47,7 +46,6 @@ public class SeekableStreamPartitions { public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER"; - public static final String BEGINNING_SEQUENCE_NUMBER = "BEGINNING_SEQUENCE_NUMBER"; // stream/topic private final String stream; @@ -57,8 +55,10 @@ public class SeekableStreamPartitions @JsonCreator public SeekableStreamPartitions( @JsonProperty("stream") final String stream, + // kept for backward compatibility @JsonProperty("topic") final String topic, @JsonProperty("partitionSequenceNumberMap") final Map partitionSequenceNumberMap, + // kept for backward compatibility @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap ) { @@ -85,26 +85,12 @@ public String getStream() return stream; } - @DoNotCall - @JsonProperty - public final String getTopic() - { - return stream; - } - @JsonProperty public Map getPartitionSequenceNumberMap() { return map; } - @DoNotCall - @JsonProperty - public final Map getPartitionOffsetMap() - { - return map; - } - @Override public boolean equals(Object o) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java index 21e6e0318324..f063260322b6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java @@ -39,7 +39,7 @@ public interface RecordSupplier extends Closeable { /** - * assigns the given parittions to this RecordSupplier + * assigns the given partitions to this RecordSupplier * and seek to the earliest sequence number. Previously * assigned partitions will be replaced. * diff --git a/pom.xml b/pom.xml index c917892c73c9..df9fcab146aa 100644 --- a/pom.xml +++ b/pom.xml @@ -1000,7 +1000,7 @@ sun.nio.ch.DirectBuffer sun.misc.Cleaner sun.misc.Unsafe - + java.lang.invoke.MethodHandle From c4553eec6a11d05d267aa8e47a52b0d980832837 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Fri, 30 Nov 2018 16:07:01 -0800 Subject: [PATCH 57/87] avg lag in kafkasupervisor #6587 --- .../kafka/supervisor/KafkaSupervisor.java | 25 +++++++++++++------ 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 98709cf8dd98..dac877b5b226 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -305,6 +305,7 @@ private Runnable emitLag() return () -> { try { Map highestCurrentOffsets = getHighestCurrentOffsets(); + String dataSource = spec.getDataSchema().getDataSource(); if (latestSequenceFromStream == null) { throw new ISE("Latest offsets from Kafka have not been fetched"); @@ -318,16 +319,24 @@ private Runnable emitLag() ); } - long lag = getLagPerPartition(highestCurrentOffsets) - .values() - .stream() - .mapToLong(x -> Math.max(x, 0)) - .sum(); + Map partitionLags = getLagPerPartition(highestCurrentOffsets); + long maxLag = 0, totalLag = 0, avgLag; + for (long lag : partitionLags.values()) { + if (lag > maxLag) { + maxLag = lag; + } + totalLag += lag; + } + avgLag = partitionLags.size() == 0 ? 0 : totalLag / partitionLags.size(); emitter.emit( - ServiceMetricEvent.builder() - .setDimension("dataSource", spec.getDataSchema().getDataSource()) - .build("ingest/kafka/lag", lag) + ServiceMetricEvent.builder().setDimension("dataSource", dataSource).build("ingest/kafka/lag", totalLag) + ); + emitter.emit( + ServiceMetricEvent.builder().setDimension("dataSource", dataSource).build("ingest/kafka/maxLag", maxLag) + ); + emitter.emit( + ServiceMetricEvent.builder().setDimension("dataSource", dataSource).build("ingest/kafka/avgLag", avgLag) ); } catch (Exception e) { From a511c0cb9b464072b9d6eb0fe3851dacd3cbbbbe Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Fri, 30 Nov 2018 16:41:38 -0800 Subject: [PATCH 58/87] refacotred SequenceMetadata in taskRunners --- ...ementalPublishingKafkaIndexTaskRunner.java | 58 +----------------- .../kafka/LegacyKafkaIndexTaskRunner.java | 12 ---- .../kinesis/KinesisIndexTaskRunner.java | 60 ------------------- .../SeekableStreamIndexTaskRunner.java | 46 +++++++++----- 4 files changed, 31 insertions(+), 145 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 0215718a459f..826d90a9fa67 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -50,7 +50,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -207,26 +206,6 @@ protected Type getRunnerType() return Type.KAFKA; } - @Override - protected SequenceMetadata createSequenceMetadata( - int sequenceId, - String sequenceName, - Map startOffsets, - Map endOffsets, - boolean checkpointed, - Set exclusiveStartPartition - ) - { - return new KafkaSequenceMetaData( - sequenceId, - sequenceName, - startOffsets, - endOffsets, - checkpointed, - null - ); - } - @Nullable @Override protected TreeMap> getCheckPointsFromContext( @@ -247,40 +226,5 @@ protected TreeMap> getCheckPointsFromContext( return null; } } - - private class KafkaSequenceMetaData extends SequenceMetadata - { - - KafkaSequenceMetaData( - int sequenceId, - String sequenceName, - Map startOffsets, - Map endOffsets, - boolean checkpointed, - Set exclusiveStartPartitions - ) - { - super(sequenceId, sequenceName, startOffsets, endOffsets, checkpointed, null); - } - - @Override - protected boolean canHandle(OrderedPartitionableRecord record) - { - lock.lock(); - try { - final OrderedSequenceNumber partitionEndOffset = createSequenceNumber(endOffsets.get(record.getPartitionId())); - final OrderedSequenceNumber partitionStartOffset = createSequenceNumber(startOffsets.get(record.getPartitionId())); - final OrderedSequenceNumber recordOffset = createSequenceNumber(record.getSequenceNumber()); - return isOpen() - && recordOffset != null - && partitionEndOffset != null - && partitionStartOffset != null - && recordOffset.compareTo(partitionStartOffset) >= 0 - && recordOffset.compareTo(partitionEndOffset) < 0; - } - finally { - lock.unlock(); - } - } - } } + diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 99b14d01d6de..eb0c85c635db 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -1225,16 +1225,4 @@ public DateTime getStartTime(@Context final HttpServletRequest req) return startTime; } - @Override - protected SequenceMetadata createSequenceMetadata( - int sequenceId, - String sequenceName, - Map startOffsets, - Map endOffsets, - boolean checkpointed, - Set exclusiveStartPartitions - ) - { - throw new UnsupportedOperationException(); - } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index fc6b13687188..8999222c0cdf 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -43,7 +43,6 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.TreeMap; public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner @@ -122,26 +121,6 @@ protected Type getRunnerType() return Type.KINESIS; } - @Override - protected SequenceMetadata createSequenceMetadata( - int sequenceId, - String sequenceName, - Map startOffsets, - Map endOffsets, - boolean checkpointed, - Set exclusiveStartPartitions - ) - { - return new KinesisSequenceMetaData( - sequenceId, - sequenceName, - startOffsets, - endOffsets, - checkpointed, - exclusiveStartPartitions - ); - } - @Nullable @Override protected TreeMap> getCheckPointsFromContext( @@ -163,43 +142,4 @@ protected TreeMap> getCheckPointsFromContext( } } - private class KinesisSequenceMetaData extends SequenceMetadata - { - - KinesisSequenceMetaData( - int sequenceId, - String sequenceName, - Map startOffsets, - Map endOffsets, - boolean checkpointed, - Set exclusiveStartPartitions - ) - { - super(sequenceId, sequenceName, startOffsets, endOffsets, checkpointed, exclusiveStartPartitions); - } - - @Override - protected boolean canHandle(OrderedPartitionableRecord record) - { - lock.lock(); - try { - final OrderedSequenceNumber partitionEndOffset = createSequenceNumber(endOffsets.get(record.getPartitionId())); - final OrderedSequenceNumber partitionStartOffset = createSequenceNumber(startOffsets.get(record.getPartitionId())); - final OrderedSequenceNumber recordOffset = createSequenceNumber(record.getSequenceNumber()); - return isOpen() - && recordOffset != null - && partitionEndOffset != null - && partitionStartOffset != null - && recordOffset.compareTo(partitionStartOffset) - >= (getExclusiveStartPartitions().contains(record.getPartitionId()) ? 1 : 0) - && recordOffset.compareTo(partitionEndOffset) <= 0; - } - finally { - lock.unlock(); - } - } - - } - - } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index a5e1677dbca8..96d4222f1b55 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -283,7 +283,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception Map.Entry> previous = sequenceOffsets.next(); while (sequenceOffsets.hasNext()) { Map.Entry> current = sequenceOffsets.next(); - sequences.add(createSequenceMetadata( + sequences.add(new SequenceMetadata( previous.getKey(), StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), previous.getValue(), @@ -294,7 +294,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception previous = current; exclusive = true; } - sequences.add(createSequenceMetadata( + sequences.add(new SequenceMetadata( previous.getKey(), StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), previous.getValue(), @@ -303,7 +303,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception exclusive ? previous.getValue().keySet() : null )); } else { - sequences.add(createSequenceMetadata( + sequences.add(new SequenceMetadata( 0, StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0), ioConfig.getStartPartitions().getPartitionSequenceNumberMap(), @@ -1493,7 +1493,7 @@ public Response setEndOffsets( exclusiveStartingPartitions.addAll(exclusivePartitions); // create new sequence - final SequenceMetadata newSequence = createSequenceMetadata( + final SequenceMetadata newSequence = new SequenceMetadata( latestSequence.getSequenceId() + 1, StringUtils.format("%s_%d", ioConfig.getBaseSequenceName(), latestSequence.getSequenceId() + 1), sequenceNumbers, @@ -1654,16 +1654,7 @@ public DateTime getStartTime(@Context final HttpServletRequest req) return startTime; } - protected abstract SequenceMetadata createSequenceMetadata( - int sequenceId, - String sequenceName, - Map startOffsets, - Map endOffsets, - boolean checkpointed, - Set exclusiveStartingPartitions - ); - - protected abstract class SequenceMetadata + private class SequenceMetadata { private final int sequenceId; private final String sequenceName; @@ -1790,12 +1781,35 @@ void updateAssignments(Map nextPartitionOffset) } } - protected boolean isOpen() + boolean isOpen() { return !assignments.isEmpty(); } - protected abstract boolean canHandle(OrderedPartitionableRecord record); + boolean canHandle(OrderedPartitionableRecord record) + { + lock.lock(); + try { + final OrderedSequenceNumber partitionEndOffset = createSequenceNumber(endOffsets.get(record.getPartitionId())); + final OrderedSequenceNumber partitionStartOffset = createSequenceNumber(startOffsets.get(record.getPartitionId())); + final OrderedSequenceNumber recordOffset = createSequenceNumber(record.getSequenceNumber()); + if (!isOpen() || recordOffset == null || partitionEndOffset == null || partitionStartOffset == null) { + return false; + } + if (getRunnerType() == Type.KINESIS) { + return recordOffset.compareTo(partitionStartOffset) + >= (getExclusiveStartPartitions().contains(record.getPartitionId()) ? 1 : 0) + && recordOffset.compareTo(partitionEndOffset) <= 0; + } else { + return recordOffset.compareTo(partitionStartOffset) >= 0 + && recordOffset.compareTo(partitionEndOffset) < 0; + + } + } + finally { + lock.unlock(); + } + } @Override public String toString() From c5d455403a50af9ec880ad89bab8486cd0196f15 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 3 Dec 2018 10:52:53 -0800 Subject: [PATCH 59/87] small fix --- .../indexing/kinesis/supervisor/KinesisSupervisorTest.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index a632c7af6253..dff277c50677 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -2812,9 +2812,7 @@ public void testNoDataIngestionTasks() throws Exception @Test(timeout = 60_000L) public void testCheckpointForInactiveTaskGroup() - throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException, ExecutionException, - TimeoutException, JsonProcessingException, NoSuchMethodException, IllegalAccessException, - ClassNotFoundException + throws InterruptedException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException { supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); //not adding any events From 751571978100274c4d6c4181fa7c126efdbf8cc9 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 3 Dec 2018 10:57:14 -0800 Subject: [PATCH 60/87] more small fix --- .../indexing/kinesis/supervisor/KinesisSupervisorTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index dff277c50677..944a2a118b3c 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -2812,7 +2812,8 @@ public void testNoDataIngestionTasks() throws Exception @Test(timeout = 60_000L) public void testCheckpointForInactiveTaskGroup() - throws InterruptedException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException + throws InterruptedException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException, + ExecutionException, TimeoutException, JsonProcessingException { supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); //not adding any events From 81b5087e75a596e19daebcb1570c404710ad83b1 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 4 Dec 2018 12:01:17 -0800 Subject: [PATCH 61/87] recordsupplier resource leak --- .../IncrementalPublishingKafkaIndexTaskRunner.java | 2 -- .../apache/druid/indexing/kafka/KafkaIndexTask.java | 4 +--- .../indexing/kafka/LegacyKafkaIndexTaskRunner.java | 2 -- .../druid/indexing/kinesis/KinesisIndexTask.java | 5 +---- .../indexing/kinesis/KinesisIndexTaskRunner.java | 2 -- .../seekablestream/SeekableStreamIndexTask.java | 5 +++++ .../seekablestream/SeekableStreamIndexTaskRunner.java | 11 +++++------ 7 files changed, 12 insertions(+), 19 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 826d90a9fa67..46886914cea1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -65,7 +65,6 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd public IncrementalPublishingKafkaIndexTaskRunner( KafkaIndexTask task, - KafkaRecordSupplier recordSupplier, InputRowParser parser, AuthorizerMapper authorizerMapper, Optional chatHandlerProvider, @@ -75,7 +74,6 @@ public IncrementalPublishingKafkaIndexTaskRunner( { super( task, - recordSupplier, parser, authorizerMapper, chatHandlerProvider, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 2ec6db18b7ad..707f71a2d252 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -141,7 +141,6 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() && ((boolean) context.get(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED))) { return new IncrementalPublishingKafkaIndexTaskRunner( this, - newTaskRecordSupplier(), parser, authorizerMapper, chatHandlerProvider, @@ -151,7 +150,6 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() } else { return new LegacyKafkaIndexTaskRunner( this, - newTaskRecordSupplier(), parser, authorizerMapper, chatHandlerProvider, @@ -161,7 +159,7 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() } } - @VisibleForTesting + @Override protected KafkaRecordSupplier newTaskRecordSupplier() { ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 8b794ee7c51f..5428a63edfc0 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -181,7 +181,6 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner parser, AuthorizerMapper authorizerMapper, Optional chatHandlerProvider, @@ -191,7 +190,6 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner createTaskRunner() - throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { return new KinesisIndexTaskRunner( this, - newTaskRecordSupplier(), parser, authorizerMapper, chatHandlerProvider, @@ -78,7 +75,7 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() ); } - @VisibleForTesting + @Override protected KinesisRecordSupplier newTaskRecordSupplier() throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index 8999222c0cdf..9c53e15a6420 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -52,7 +52,6 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner parser, AuthorizerMapper authorizerMapper, Optional chatHandlerProvider, @@ -62,7 +61,6 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner createTaskRunner() throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException; + protected abstract RecordSupplier newTaskRecordSupplier() throws ClassNotFoundException, + NoSuchMethodException, + IllegalAccessException; + @VisibleForTesting public Appenderator getAppenderator() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 96d4222f1b55..ecab0dafef2b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -191,7 +191,6 @@ protected enum Type protected final Condition isAwaitingRetry = pollRetryLock.newCondition(); private final SeekableStreamIndexTask task; - private final RecordSupplier recordSupplier; private final SeekableStreamIOConfig ioConfig; private final SeekableStreamTuningConfig tuningConfig; private final InputRowParser parser; @@ -223,7 +222,6 @@ protected enum Type public SeekableStreamIndexTaskRunner( final SeekableStreamIndexTask task, - final RecordSupplier recordSupplier, final InputRowParser parser, final AuthorizerMapper authorizerMapper, final Optional chatHandlerProvider, @@ -233,7 +231,6 @@ public SeekableStreamIndexTaskRunner( { Preconditions.checkNotNull(task); this.task = task; - this.recordSupplier = recordSupplier; this.ioConfig = task.getIOConfig(); this.tuningConfig = task.getTuningConfig(); this.parser = parser; @@ -350,7 +347,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception ); Throwable caughtExceptionOuter = null; - try { + try (final RecordSupplier recordSupplier = task.newTaskRecordSupplier()) { toolbox.getDataSegmentServerAnnouncer().announce(); toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); @@ -500,7 +497,10 @@ public void run() // calling getRecord() ensures that exceptions specific to kafka/kinesis like OffsetOutOfRangeException // are handled in the subclasses - List> records = getRecords(recordSupplier, toolbox); + List> records = getRecords( + recordSupplier, + toolbox + ); stillReading = !assignment.isEmpty(); @@ -758,7 +758,6 @@ public void onFailure(@ParametersAreNonnullByDefault Throwable t) } appenderator.close(); - recordSupplier.close(); } catch (InterruptedException | RejectedExecutionException e) { // (2) catch InterruptedException and RejectedExecutionException thrown for the whole ingestion steps including From 09676e885f647b26c842346202f40c762cf124b3 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 4 Dec 2018 13:27:35 -0800 Subject: [PATCH 62/87] revert .travis.yml formatting --- .travis.yml | 222 ++++++++++++++++++++++++++-------------------------- 1 file changed, 111 insertions(+), 111 deletions(-) diff --git a/.travis.yml b/.travis.yml index c7d3d7da4544..1e17711f7060 100644 --- a/.travis.yml +++ b/.travis.yml @@ -23,130 +23,130 @@ dist: trusty group: deprecated-2017Q4 jdk: -- oraclejdk8 + - oraclejdk8 cache: directories: - - $HOME/.m2 + - $HOME/.m2 matrix: include: - # license checks - - env: - - NAME="license checks" - install: true - script: MAVEN_OPTS='-Xmx3000m' mvn clean verify -Prat -DskipTests -B -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn + # license checks + - env: + - NAME="license checks" + install: true + script: MAVEN_OPTS='-Xmx3000m' mvn clean verify -Prat -DskipTests -B -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn - # strict compilation - - env: - - NAME="strict compilation" - install: true - # Strict compilation requires more than 2 GB - script: MAVEN_OPTS='-Xmx3000m' mvn clean -Pstrict -pl '!benchmarks' compile test-compile -B --fail-at-end + # strict compilation + - env: + - NAME="strict compilation" + install: true + # Strict compilation requires more than 2 GB + script: MAVEN_OPTS='-Xmx3000m' mvn clean -Pstrict -pl '!benchmarks' compile test-compile -B --fail-at-end - # processing module test - - env: - - NAME="processing module test" - install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B - before_script: unset _JAVA_OPTIONS - script: - # Set MAVEN_OPTS for Surefire launcher - - MAVEN_OPTS='-Xmx512m' mvn test -B -pl processing - - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" - - free -m + # processing module test + - env: + - NAME="processing module test" + install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B + before_script: unset _JAVA_OPTIONS + script: + # Set MAVEN_OPTS for Surefire launcher + - MAVEN_OPTS='-Xmx512m' mvn test -B -pl processing + - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" + - free -m - # processing module tests with SQL Compatibility enabled - - env: - - NAME="processing module test with SQL Compatibility" - install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B - before_script: unset _JAVA_OPTIONS - script: - # Set MAVEN_OPTS for Surefire launcher - - MAVEN_OPTS='-Xmx512m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl processing - - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" - - free -m + # processing module tests with SQL Compatibility enabled + - env: + - NAME="processing module test with SQL Compatibility" + install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B + before_script: unset _JAVA_OPTIONS + script: + # Set MAVEN_OPTS for Surefire launcher + - MAVEN_OPTS='-Xmx512m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl processing + - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" + - free -m - # server module test - - env: - - NAME="server module test" - install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B - before_script: unset _JAVA_OPTIONS - script: - # Set MAVEN_OPTS for Surefire launcher - - MAVEN_OPTS='-Xmx512m' mvn test -B -pl server + # server module test + - env: + - NAME="server module test" + install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B + before_script: unset _JAVA_OPTIONS + script: + # Set MAVEN_OPTS for Surefire launcher + - MAVEN_OPTS='-Xmx512m' mvn test -B -pl server - # server module test with SQL Compatibility enabled - - env: - - NAME="server module test with SQL Compatibility enabled" - install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B - before_script: unset _JAVA_OPTIONS - script: - # Set MAVEN_OPTS for Surefire launcher - - MAVEN_OPTS='-Xmx512m' mvn test -B -pl server -Ddruid.generic.useDefaultValueForNull=false + # server module test with SQL Compatibility enabled + - env: + - NAME="server module test with SQL Compatibility enabled" + install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B + before_script: unset _JAVA_OPTIONS + script: + # Set MAVEN_OPTS for Surefire launcher + - MAVEN_OPTS='-Xmx512m' mvn test -B -pl server -Ddruid.generic.useDefaultValueForNull=false - # other modules test - - env: - - NAME="other modules test" - - AWS_REGION=us-east-1 # set a aws region for unit tests - install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B - before_script: unset _JAVA_OPTIONS - script: - # Set MAVEN_OPTS for Surefire launcher - - MAVEN_OPTS='-Xmx512m' mvn test -B -pl '!processing,!server' - - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" - - free -m + # other modules test + - env: + - NAME="other modules test" + - AWS_REGION=us-east-1 # set a aws region for unit tests + install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B + before_script: unset _JAVA_OPTIONS + script: + # Set MAVEN_OPTS for Surefire launcher + - MAVEN_OPTS='-Xmx512m' mvn test -B -pl '!processing,!server' + - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" + - free -m - # other modules test with SQL Compatibility enabled - - env: - - NAME="other modules test with SQL Compatibility" - - AWS_REGION=us-east-1 # set a aws region for unit tests - install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B - before_script: unset _JAVA_OPTIONS - script: - # Set MAVEN_OPTS for Surefire launcher - - MAVEN_OPTS='-Xmx512m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl '!processing,!server' - - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" - - free -m + # other modules test with SQL Compatibility enabled + - env: + - NAME="other modules test with SQL Compatibility" + - AWS_REGION=us-east-1 # set a aws region for unit tests + install: MAVEN_OPTS='-Xmx3000m' mvn install -q -ff -DskipTests -B + before_script: unset _JAVA_OPTIONS + script: + # Set MAVEN_OPTS for Surefire launcher + - MAVEN_OPTS='-Xmx512m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl '!processing,!server' + - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" + - free -m - # run integration tests - - sudo: required - services: - - docker - env: - - NAME="integration test part 1" - - DOCKER_IP=127.0.0.1 - install: - # Only errors will be shown with the -q option. This is to avoid generating too many logs which make travis build failed. - - mvn install -q -ff -DskipTests -B - script: - - $TRAVIS_BUILD_DIR/ci/travis_script_integration.sh - after_failure: - - for v in ~/shared/logs/*.log ; do - echo $v logtail ======================== ; tail -100 $v ; - done - - for v in broker middlemanager overlord router coordinator historical ; do - echo $v dmesg ======================== ; - docker exec -it druid-$v sh -c 'dmesg | tail -3' ; - done + # run integration tests + - sudo: required + services: + - docker + env: + - NAME="integration test part 1" + - DOCKER_IP=127.0.0.1 + install: + # Only errors will be shown with the -q option. This is to avoid generating too many logs which make travis build failed. + - mvn install -q -ff -DskipTests -B + script: + - $TRAVIS_BUILD_DIR/ci/travis_script_integration.sh + after_failure: + - for v in ~/shared/logs/*.log ; do + echo $v logtail ======================== ; tail -100 $v ; + done + - for v in broker middlemanager overlord router coordinator historical ; do + echo $v dmesg ======================== ; + docker exec -it druid-$v sh -c 'dmesg | tail -3' ; + done - # run integration tests - - sudo: required - services: - - docker - env: - - NAME="integration test part 2" - - DOCKER_IP=127.0.0.1 - install: - # Only errors will be shown with the -q option. This is to avoid generating too many logs which make travis build failed. - - mvn install -q -ff -DskipTests -B - script: - - $TRAVIS_BUILD_DIR/ci/travis_script_integration_part2.sh - after_failure: - - for v in ~/shared/logs/*.log ; do - echo $v logtail ======================== ; tail -100 $v ; - done - - for v in broker middlemanager overlord router coordinator historical ; do - echo $v dmesg ======================== ; - docker exec -it druid-$v sh -c 'dmesg | tail -3' ; - done + # run integration tests + - sudo: required + services: + - docker + env: + - NAME="integration test part 2" + - DOCKER_IP=127.0.0.1 + install: + # Only errors will be shown with the -q option. This is to avoid generating too many logs which make travis build failed. + - mvn install -q -ff -DskipTests -B + script: + - $TRAVIS_BUILD_DIR/ci/travis_script_integration_part2.sh + after_failure: + - for v in ~/shared/logs/*.log ; do + echo $v logtail ======================== ; tail -100 $v ; + done + - for v in broker middlemanager overlord router coordinator historical ; do + echo $v dmesg ======================== ; + docker exec -it druid-$v sh -c 'dmesg | tail -3' ; + done From aaf1419042d29d40867cbf22da271a19639d40e4 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 4 Dec 2018 16:05:45 -0800 Subject: [PATCH 63/87] fix style --- .../druid/indexing/seekablestream/SeekableStreamIndexTask.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 25560d3cd02d..4b30f4e6dbec 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -279,8 +279,7 @@ public boolean withinMinMaxRecordTime(final InputRow row) return !beforeMinimumMessageTime && !afterMaximumMessageTime; } - protected abstract SeekableStreamIndexTaskRunner createTaskRunner() - throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException; + protected abstract SeekableStreamIndexTaskRunner createTaskRunner(); protected abstract RecordSupplier newTaskRecordSupplier() throws ClassNotFoundException, NoSuchMethodException, From 98de4cc07ba353a3b392f201811d0e2e11fd68cf Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 5 Dec 2018 16:24:36 -0800 Subject: [PATCH 64/87] kinesis docs --- .../extensions-core/kinesis-ingestion.md | 176 ++++++++++++++++++ 1 file changed, 176 insertions(+) create mode 100644 docs/content/development/extensions-core/kinesis-ingestion.md diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md new file mode 100644 index 000000000000..7bde4a6f733c --- /dev/null +++ b/docs/content/development/extensions-core/kinesis-ingestion.md @@ -0,0 +1,176 @@ +# Kinesis Indexing Service + +Pull request [Link](https://github.com/apache/incubator-druid/pull/6431) + +Similar to the [Kafka indexing service](http://druid.io/docs/0.10.0/development/extensions-core/kafka-ingestion.html), +the Kinesis indexing service uses supervisors which run on the overlord and manage the creation and lifetime of Kinesis +indexing tasks. This indexing service can handle non-recent events and provides exactly-once ingestion semantics. + +The Kinesis indexing service is provided as the `druid-kinesis-indexing-service` core extension (see +[Including Extensions](http://druid.io/docs/0.10.0/operations/including-extensions.html)). Please note that this is +currently designated as an *experimental feature* and is subject to the usual +[experimental caveats](http://druid.io/docs/0.10.0/development/experimental.html). + +## Submitting a Supervisor Spec + +The Kinesis indexing service requires that the `druid-kinesis-indexing-service` extension be loaded on both the overlord +and the middle managers. A supervisor for a dataSource is started by submitting a supervisor spec via HTTP POST to +`http://:/druid/indexer/v1/supervisor`, for example: + +``` +curl -X POST -H 'Content-Type: application/json' -d @supervisor-spec.json http://localhost:8090/druid/indexer/v1/supervisor +``` + +A sample supervisor spec is shown below: + +```json +{ + "type": "kinesis", + "dataSchema": { + "dataSource": "metrics-kinesis", + "parser": { + "type": "string", + "parseSpec": { + "format": "json", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [], + "dimensionExclusions": [ + "timestamp", + "value" + ] + } + } + }, + "metricsSpec": [ + { + "name": "count", + "type": "count" + }, + { + "name": "value_sum", + "fieldName": "value", + "type": "doubleSum" + }, + { + "name": "value_min", + "fieldName": "value", + "type": "doubleMin" + }, + { + "name": "value_max", + "fieldName": "value", + "type": "doubleMax" + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "HOUR", + "queryGranularity": "NONE" + } + }, + "tuningConfig": { + "type": "kinesis", + "maxRowsPerSegment": 5000000 + }, + "ioConfig": { + "stream": "metrics", + "endpoint": "kinesis.us-east-1.amazonaws.com", + "taskCount": 1, + "replicas": 1, + "taskDuration": "PT1H", + "recordsPerFetch": 2000, + "fetchDelayMillis": 1000 + } +} +``` + +## Supervisor Configuration + +|Field|Description|Required| +|--------|-----------|---------| +|`type`|The supervisor type, this should always be `kinesis`.|yes| +|`dataSchema`|The schema that will be used by the Kinesis indexing task during ingestion, see [Ingestion Spec](http://druid.io/docs/0.10.0/ingestion/index.html).|yes| +|`tuningConfig`|A KinesisSupervisorTuningConfig to configure the supervisor and indexing tasks, see below.|no| +|`ioConfig`|A KinesisSupervisorIOConfig to configure the supervisor and indexing tasks, see below.|yes| + +### KinesisSupervisorTuningConfig + +The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`type`|String|The indexing task type, this should always be `kinesis`.|yes| +|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 75000)| +|`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number is post-aggregation rows.|no (default == 5000000)| +|`intermediatePersistPeriod`|ISO8601 Period|The period that determines the rate at which intermediate persists occur.|no (default == PT10M)| +|`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)| +|`indexSpec`|Object|Tune how data is indexed, see 'IndexSpec' [here](http://druid.io/docs/0.10.0/development/extensions-core/kafka-ingestion.html#indexspec).|no| +|`buildV9Directly`|Boolean|Whether to build a v9 index directly instead of first building a v8 index and then converting it to v9 format.|no (default == true)| +|`reportParseExceptions`|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|no (default == false)| +|`workerThreads`|Integer|The number of threads that will be used by the supervisor for asynchronous operations.|no (default == min(10, taskCount))| +|`chatThreads`|Integer|The number of threads that will be used for communicating with indexing tasks.|no (default == min(10, taskCount * replicas))| +|`chatRetries`|Integer|The number of times HTTP requests to indexing tasks will be retried before considering tasks unresponsive.|no (default == 8)| +|`httpTimeout`|ISO8601 Period|How long to wait for a HTTP response from an indexing task.|no (default == PT10S)| +|`shutdownTimeout`|ISO8601 Period|How long to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|no (default == PT80S)| +|`recordBufferSize`|Integer|Size of the buffer (number of events) used between the Kinesis fetch threads and the main ingestion thread.|no (default == 10000)| +|`recordBufferOfferTimeout`|Integer|Length of time in milliseconds to wait for space to become available in the buffer before timing out.|no (default == 10000)| +|`recordBufferFullWait`|Integer|Length of time in milliseconds to wait for the buffer to drain before attempting to fetch records from Kinesis again.|no (default == 10000)| +|`fetchSequenceNumberTimeout`|Integer|Length of time in milliseconds to wait for Kinesis to return the earliest or latest sequence number for a partition. Kinesis will not return the latest sequence number if no data is actively being written to that partition. In this case, this fetch call will repeatedly timeout and retry until fresh data is written to the stream.|no (default == 60000)| +|`fetchThreads`|Integer|Size of the pool of threads fetching data from Kinesis. There is no benefit in having more threads than Kinesis partitions.|no (default == max(1, {numProcessors} - 1))| + +### KinesisSupervisorIOConfig + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`stream`|String|The Kinesis stream to read.|yes| +|`endpoint`|String|The AWS Kinesis stream endpoint for a region. You can find a list of endpoints [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region).|no (default == kinesis.us-east-1.amazonaws.com)| +|`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against node failure.|no (default == 1)| +|`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. See 'Capacity Planning' below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKinesisPartitions}`.|no (default == 1)| +|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment. Note that segments are only pushed to deep storage and loadable by historical nodes when the indexing task completes.|no (default == PT1H)| +|`startDelay`|ISO8601 Period|The period to wait before the supervisor starts managing tasks.|no (default == PT5S)| +|`period`|ISO8601 Period|How often the supervisor will execute its management logic. Note that the supervisor will also run in response to certain events (such as tasks succeeding, failing, and reaching their taskDuration) so this value specifies the maximum time between iterations.|no (default == PT30S)| +|`useEarliestSequenceNumber`|Boolean|If a supervisor is managing a dataSource for the first time, it will obtain a set of starting sequence numbers from Kinesis. This flag determines whether it retrieves the earliest or latest offsets in Kinesis. Under normal circumstances, subsequent tasks will start from where the previous segments ended so this flag will only be used on first run.|no (default == false)| +|`completionTimeout`|ISO8601 Period|The length of time to wait before declaring a publishing task as failed and terminating it. If this is set too low, your tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|no (default == PT6H)| +|`lateMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps earlier than this period before the task was created; for example if this is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps earlier than *2016-01-01T11:00Z* will be dropped. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments (e.g. a realtime and a nightly batch ingestion pipeline).|no (default == none)| +|`recordsPerFetch`|Integer|The number of records to request per GetRecords call to Kinesis. See 'Determining Fetch Settings' below.|no (default == 2000)| +|`fetchDelayMillis`|Integer|Time in milliseconds to wait between subsequent GetRecords calls to Kinesis. See 'Determining Fetch Settings' below.|no (default == 1000)| +|`awsAccessKeyId`|String|The AWS access key ID to use for Kinesis API requests. If this is not provided, the service will look for credentials set in the environment variables, system properties, in the default profile configuration file, and from the EC2 instance profile provider (in this order).|no| +|`awsSecretAccessKey`|String|The AWS secret access key to use for Kinesis API requests. Only used if `awsAccessKeyId` is also provided.|no| + +## Determining Fetch Settings + +Kinesis places the following restrictions on calls to fetch records: + +- Each data record can be up to 1 MB in size. +- Each shard can support up to 5 transactions per second for reads. +- Each shard can read up to 2 MB per second. +- The maximum size of data that GetRecords can return is 10 MB. + +Values for `recordsPerFetch` and `fetchDelayMillis` should be chosen to maximize throughput under the above constraints. +The values that you choose will depend on the average size of a record and the number of consumers you have reading from +a given shard (which will be `replicas` unless you have other consumers also reading from this Kinesis stream). + +If the above limits are violated, AWS will throw ProvisionedThroughputExceededException errors on subsequent calls to +read data. When this happens, the Kinesis indexing service will pause by `fetchDelayMillis` and then attempt the call +again. + +## Supervisor API, Capacity Planning, Persistence, and Schema Changes + +The Kinesis indexing service uses the same supervisor API and has the same considerations for capacity planning, +persistence, and schema changes as the Kafka indexing service. For documentation on these topics, see the relevant +sections of the Kafka indexing service [documentation](http://druid.io/docs/0.10.0/development/extensions-core/kafka-ingestion.html#supervisor-api). + +## Deaggregation +See [issue](https://github.com/apache/incubator-druid/issues/6714) + +The Kinesis indexing service supports de-aggregation of multiple rows packed into a single record by the Kinesis +Producer Library's aggregate method for more efficient data transfer. Currently, enabling the de-aggregate functionality +requires the user to manually provide the Kinesis Client Library on the classpath, since this library has a license not +compatible with Apache projects. + +To enable this feature, add the `amazon-kinesis-client` (tested on version `1.9.2`) jar file ([link](https://mvnrepository.com/artifact/com.amazonaws/amazon-kinesis-client/1.9.2)) under `dist/druid/extensions/druid-kinesis-indexing-service/`. +Then when submitting a supervisor-spec, set `deaggregate` to true. \ No newline at end of file From f5d88d91fb3d5d261088148119dbfa7c303fadf1 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 6 Dec 2018 13:05:03 -0800 Subject: [PATCH 65/87] doc part2 --- .../extensions-core/kinesis-ingestion.md | 256 ++++++++++++++++-- 1 file changed, 232 insertions(+), 24 deletions(-) diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md index 7bde4a6f733c..d577109bae13 100644 --- a/docs/content/development/extensions-core/kinesis-ingestion.md +++ b/docs/content/development/extensions-core/kinesis-ingestion.md @@ -1,15 +1,39 @@ -# Kinesis Indexing Service + + +--- +layout: doc_page +--- -Pull request [Link](https://github.com/apache/incubator-druid/pull/6431) +# Kinesis Indexing Service -Similar to the [Kafka indexing service](http://druid.io/docs/0.10.0/development/extensions-core/kafka-ingestion.html), -the Kinesis indexing service uses supervisors which run on the overlord and manage the creation and lifetime of Kinesis -indexing tasks. This indexing service can handle non-recent events and provides exactly-once ingestion semantics. +Similar to the [Kafka indexing service](./kafka-ingestion.html), The Kinesis indexing service enables the configuration of *supervisors* on the Overlord, which facilitate ingestion from +Kinesis by managing the creation and lifetime of Kinesis indexing tasks. These indexing tasks read events using Kinesis's own +Shards and Sequence Number mechanism and are therefore able to provide guarantees of exactly-once ingestion. They are also +able to read non-recent events from Kinesis and are not subject to the window period considerations imposed on other +ingestion mechanisms. The supervisor oversees the state of the indexing tasks to coordinate handoffs, manage failures, +and ensure that the scalability and replication requirements are maintained. The Kinesis indexing service is provided as the `druid-kinesis-indexing-service` core extension (see -[Including Extensions](http://druid.io/docs/0.10.0/operations/including-extensions.html)). Please note that this is +[Including Extensions](../../operations/including-extensions.html)). Please note that this is currently designated as an *experimental feature* and is subject to the usual -[experimental caveats](http://druid.io/docs/0.10.0/development/experimental.html). +[experimental caveats](../experimental.html). ## Submitting a Supervisor Spec @@ -93,7 +117,7 @@ A sample supervisor spec is shown below: |Field|Description|Required| |--------|-----------|---------| |`type`|The supervisor type, this should always be `kinesis`.|yes| -|`dataSchema`|The schema that will be used by the Kinesis indexing task during ingestion, see [Ingestion Spec](http://druid.io/docs/0.10.0/ingestion/index.html).|yes| +|`dataSchema`|The schema that will be used by the Kinesis indexing task during ingestion, see [Ingestion Spec DataSchema](../../ingestion/ingestion-spec.html#dataschema).|yes| |`tuningConfig`|A KinesisSupervisorTuningConfig to configure the supervisor and indexing tasks, see below.|no| |`ioConfig`|A KinesisSupervisorIOConfig to configure the supervisor and indexing tasks, see below.|yes| @@ -104,23 +128,63 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |Field|Type|Description|Required| |-----|----|-----------|--------| |`type`|String|The indexing task type, this should always be `kinesis`.|yes| -|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 75000)| -|`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number is post-aggregation rows.|no (default == 5000000)| +|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 100000)| +|`maxBytesInMemory`|Long|The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. Normally this is computed internally and user does not need to set it. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists). |no (default == One-sixth of max JVM memory)| +|`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number is post-aggregation rows. Handoff will happen either if `maxRowsPerSegment` or `maxTotalRows` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.|no (default == 5000000)| +|`maxTotalRows`|Long|The number of rows to aggregate across all segments; this number is post-aggregation rows. Handoff will happen either if `maxRowsPerSegment` or `maxTotalRows` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.|no (default == unlimited)| |`intermediatePersistPeriod`|ISO8601 Period|The period that determines the rate at which intermediate persists occur.|no (default == PT10M)| |`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)| -|`indexSpec`|Object|Tune how data is indexed, see 'IndexSpec' [here](http://druid.io/docs/0.10.0/development/extensions-core/kafka-ingestion.html#indexspec).|no| -|`buildV9Directly`|Boolean|Whether to build a v9 index directly instead of first building a v8 index and then converting it to v9 format.|no (default == true)| +|`indexSpec`|Object|Tune how data is indexed, see 'IndexSpec' below for more details.|no| |`reportParseExceptions`|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|no (default == false)| +|`handoffConditionTimeout`|Long|Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever.|no (default == 0)| +|`resetOffsetAutomatically`|Boolean|Whether to reset the consumer sequence numbers if the next sequence number that it is trying to fetch is less than the earliest available sequence number for that particular shard. The sequence number will be reset to either the earliest or latest sequence number depending on `useEarliestOffset` property of `KinesisSupervisorIOConfig` (see below). This situation typically occurs when messages in Kinesis are no longer available for consumption and therefore won't be ingested into Druid. If set to false then ingestion for that particular shard will halt and manual intervention is required to correct the situation, please see `Reset Supervisor` API below.|no (default == false)| +|`skipSequenceNumberAvailabilityCheck`|Boolean|Whether to enable checking if the current sequence number is still available in a particular Kinesis shard. If set to false, the indexing task will attempt to reset the current sequence number (or not), depending on the value of `resetOffsetAutomatically`. |no (default == false)| |`workerThreads`|Integer|The number of threads that will be used by the supervisor for asynchronous operations.|no (default == min(10, taskCount))| |`chatThreads`|Integer|The number of threads that will be used for communicating with indexing tasks.|no (default == min(10, taskCount * replicas))| |`chatRetries`|Integer|The number of times HTTP requests to indexing tasks will be retried before considering tasks unresponsive.|no (default == 8)| |`httpTimeout`|ISO8601 Period|How long to wait for a HTTP response from an indexing task.|no (default == PT10S)| |`shutdownTimeout`|ISO8601 Period|How long to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|no (default == PT80S)| |`recordBufferSize`|Integer|Size of the buffer (number of events) used between the Kinesis fetch threads and the main ingestion thread.|no (default == 10000)| -|`recordBufferOfferTimeout`|Integer|Length of time in milliseconds to wait for space to become available in the buffer before timing out.|no (default == 10000)| -|`recordBufferFullWait`|Integer|Length of time in milliseconds to wait for the buffer to drain before attempting to fetch records from Kinesis again.|no (default == 10000)| -|`fetchSequenceNumberTimeout`|Integer|Length of time in milliseconds to wait for Kinesis to return the earliest or latest sequence number for a partition. Kinesis will not return the latest sequence number if no data is actively being written to that partition. In this case, this fetch call will repeatedly timeout and retry until fresh data is written to the stream.|no (default == 60000)| -|`fetchThreads`|Integer|Size of the pool of threads fetching data from Kinesis. There is no benefit in having more threads than Kinesis partitions.|no (default == max(1, {numProcessors} - 1))| +|`recordBufferOfferTimeout`|Integer|Length of time in milliseconds to wait for space to become available in the buffer before timing out.|no (default == 5000)| +|`recordBufferFullWait`|Integer|Length of time in milliseconds to wait for the buffer to drain before attempting to fetch records from Kinesis again.|no (default == 5000)| +|`fetchSequenceNumberTimeout`|Integer|Length of time in milliseconds to wait for Kinesis to return the earliest or latest sequence number for a shard. Kinesis will not return the latest sequence number if no data is actively being written to that shard. In this case, this fetch call will repeatedly timeout and retry until fresh data is written to the stream.|no (default == 60000)| +|`fetchThreads`|Integer|Size of the pool of threads fetching data from Kinesis. There is no benefit in having more threads than Kinesis shards.|no (default == max(1, {numProcessors} - 1))| +|`segmentWriteOutMediumFactory`|Object|Segment write-out medium to use when creating segments. See below for more information.|no (not specified by default, the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type` is used)| +|`intermediateHandoffPeriod`|ISO8601 Period|How often the tasks should hand off segments. Handoff will happen either if `maxRowsPerSegment` or `maxTotalRows` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.|no (default == P2147483647D)| +|`logParseExceptions`|Boolean|If true, log an error message when a parsing exception occurs, containing information about the row where the error occurred.|no, default == false| +|`maxParseExceptions`|Integer|The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overridden if `reportParseExceptions` is set.|no, unlimited default| +|`maxSavedParseExceptions`|Integer|When a parse exception occurs, Druid can keep track of the most recent parse exceptions. "maxSavedParseExceptions" limits how many exception instances will be saved. These saved exceptions will be made available after the task finishes in the [task completion report](../../ingestion/reports.html). Overridden if `reportParseExceptions` is set.|no, default == 0| +|`maxRecordsPerPoll`|Integer| The maximum number of records/events to be fetched from buffer per poll. The actual maximum will be `Max(maxRecordsPerPoll, Max(bufferSize, 1)) |no, default == 100| + +#### IndexSpec + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)| +|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| +|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)| +|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using sequence number or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)| + +##### Bitmap types + +For Concise bitmaps: + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`type`|String|Must be `concise`.|yes| + +For Roaring bitmaps: + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`type`|String|Must be `roaring`.|yes| +|`compressRunOnSerialization`|Boolean|Use a run-length encoding where it is estimated as more space efficient.|no (default == `true`)| + +#### SegmentWriteOutMediumFactory + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`type`|String|See [Additional Peon Configuration: SegmentWriteOutMediumFactory](../../configuration/index.html#segmentwriteoutmediumfactory) for explanation and available options.|yes| ### KinesisSupervisorIOConfig @@ -129,19 +193,165 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |`stream`|String|The Kinesis stream to read.|yes| |`endpoint`|String|The AWS Kinesis stream endpoint for a region. You can find a list of endpoints [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region).|no (default == kinesis.us-east-1.amazonaws.com)| |`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against node failure.|no (default == 1)| -|`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. See 'Capacity Planning' below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKinesisPartitions}`.|no (default == 1)| +|`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. See 'Capacity Planning' below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKinesisshards}`.|no (default == 1)| |`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment. Note that segments are only pushed to deep storage and loadable by historical nodes when the indexing task completes.|no (default == PT1H)| |`startDelay`|ISO8601 Period|The period to wait before the supervisor starts managing tasks.|no (default == PT5S)| |`period`|ISO8601 Period|How often the supervisor will execute its management logic. Note that the supervisor will also run in response to certain events (such as tasks succeeding, failing, and reaching their taskDuration) so this value specifies the maximum time between iterations.|no (default == PT30S)| -|`useEarliestSequenceNumber`|Boolean|If a supervisor is managing a dataSource for the first time, it will obtain a set of starting sequence numbers from Kinesis. This flag determines whether it retrieves the earliest or latest offsets in Kinesis. Under normal circumstances, subsequent tasks will start from where the previous segments ended so this flag will only be used on first run.|no (default == false)| +|`useEarliestSequenceNumber`|Boolean|If a supervisor is managing a dataSource for the first time, it will obtain a set of starting sequence numbers from Kinesis. This flag determines whether it retrieves the earliest or latest sequence numbers in Kinesis. Under normal circumstances, subsequent tasks will start from where the previous segments ended so this flag will only be used on first run.|no (default == false)| |`completionTimeout`|ISO8601 Period|The length of time to wait before declaring a publishing task as failed and terminating it. If this is set too low, your tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|no (default == PT6H)| |`lateMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps earlier than this period before the task was created; for example if this is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps earlier than *2016-01-01T11:00Z* will be dropped. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments (e.g. a realtime and a nightly batch ingestion pipeline).|no (default == none)| +|`earlyMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps later than this period after the task reached its taskDuration; for example if this is set to `PT1H`, the taskDuration is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps later than *2016-01-01T14:00Z* will be dropped. **Note:** Tasks sometimes run past their task duration, for example, in cases of supervisor failover. Setting earlyMessageRejectionPeriod too low may cause messages to be dropped unexpectedly whenever a task runs past its originally configured task duration.|no (default == none)| |`recordsPerFetch`|Integer|The number of records to request per GetRecords call to Kinesis. See 'Determining Fetch Settings' below.|no (default == 2000)| |`fetchDelayMillis`|Integer|Time in milliseconds to wait between subsequent GetRecords calls to Kinesis. See 'Determining Fetch Settings' below.|no (default == 1000)| |`awsAccessKeyId`|String|The AWS access key ID to use for Kinesis API requests. If this is not provided, the service will look for credentials set in the environment variables, system properties, in the default profile configuration file, and from the EC2 instance profile provider (in this order).|no| |`awsSecretAccessKey`|String|The AWS secret access key to use for Kinesis API requests. Only used if `awsAccessKeyId` is also provided.|no| +|`awsAssumedRoleArn`|String|The AWS assumed role to use for additional permissions.|no| +|`awsExternalId`|String|The AWS external id to use for additional permissions.|no| +|`deaggregate`|Boolean|Whether to use the de-aggregate function of the KCL. See below for details.|no| + +## Operations + +This section gives descriptions of how some supervisor APIs work specifically in Kinesis Indexing Service. +For all supervisor APIs, please check [Supervisor APIs](../../operations/api-reference.html#supervisors). + +### Getting Supervisor Status Report + +`GET /druid/indexer/v1/supervisor//status` returns a snapshot report of the current state of the tasks managed by the given supervisor. This includes the latest +sequence numbers as reported by Kinesis. Unlike the Kafka Indexing Service, stats about lag is not yet supported. + +### Getting Supervisor Ingestion Stats Report + +`GET /druid/indexer/v1/supervisor//stats` returns a snapshot of the current ingestion row counters for each task being managed by the supervisor, along with moving averages for the row counters. + +See [Task Reports: Row Stats](../../ingestion/reports.html#row-stats) for more information. + +### Updating Existing Supervisors + +`POST /druid/indexer/v1/supervisor` can be used to update existing supervisor spec. +Calling this endpoint when there is already an existing supervisor for the same dataSource will cause: + +- The running supervisor to signal its managed tasks to stop reading and begin publishing. +- The running supervisor to exit. +- A new supervisor to be created using the configuration provided in the request body. This supervisor will retain the +existing publishing tasks and will create new tasks starting at the sequence numbers the publishing tasks ended on. + +Seamless schema migrations can thus be achieved by simply submitting the new schema using this endpoint. -## Determining Fetch Settings +### Suspending and Resuming Supervisors + +You can suspend and resume a supervisor using `POST /druid/indexer/v1/supervisor//suspend` and `POST /druid/indexer/v1/supervisor//resume`, respectively. + +Note that the supervisor itself will still be operating and emitting logs and metrics, +it will just ensure that no indexing tasks are running until the supervisor is resumed. + +### Resetting Supervisors + +To reset a running supervisor, you can use `POST /druid/indexer/v1/supervisor//reset`. + +The indexing service keeps track of the latest persisted Kinesis sequence number in order to provide exactly-once ingestion +guarantees across tasks. Subsequent tasks must start reading from where the previous task completed in order for the +generated segments to be accepted. If the messages at the expected starting sequence numbers are no longer available in Kinesis +(typically because the message retention period has elapsed or the topic was removed and re-created) the supervisor will +refuse to start and in-flight tasks will fail. + +This endpoint can be used to clear the stored sequence numbers which will cause the supervisor to start reading from +either the earliest or latest sequence numbers in Kinesis (depending on the value of `useEarliestOffset`). The supervisor must be +running for this endpoint to be available. After the stored sequence numbers are cleared, the supervisor will automatically kill +and re-create any active tasks so that tasks begin reading from valid sequence numbers. + +Note that since the stored sequence numbers are necessary to guarantee exactly-once ingestion, resetting them with this endpoint +may cause some Kinesis messages to be skipped or to be read twice. + +### Terminating Supervisors + +`POST /druid/indexer/v1/supervisor//terminate` terminates a supervisor and causes all associated indexing +tasks managed by this supervisor to immediately stop and begin +publishing their segments. This supervisor will still exist in the metadata store and it's history may be retrieved +with the supervisor history api, but will not be listed in the 'get supervisors' api response nor can it's configuration +or status report be retrieved. The only way this supervisor can start again is by submitting a functioning supervisor +spec to the create api. + +### Capacity Planning + +Kinesis indexing tasks run on middle managers and are thus limited by the resources available in the middle manager +cluster. In particular, you should make sure that you have sufficient worker capacity (configured using the +`druid.worker.capacity` property) to handle the configuration in the supervisor spec. Note that worker capacity is +shared across all types of indexing tasks, so you should plan your worker capacity to handle your total indexing load +(e.g. batch processing, realtime tasks, merging tasks, etc.). If your workers run out of capacity, Kinesis indexing tasks +will queue and wait for the next available worker. This may cause queries to return partial results but will not result +in data loss (assuming the tasks run before Kinesis purges those sequence numbers). + +A running task will normally be in one of two states: *reading* or *publishing*. A task will remain in reading state for +`taskDuration`, at which point it will transition to publishing state. A task will remain in publishing state for as long +as it takes to generate segments, push segments to deep storage, and have them be loaded and served by a historical node +(or until `completionTimeout` elapses). + +The number of reading tasks is controlled by `replicas` and `taskCount`. In general, there will be `replicas * taskCount` +reading tasks, the exception being if taskCount > {numKinesisShards} in which case {numKinesisShards} tasks will +be used instead. When `taskDuration` elapses, these tasks will transition to publishing state and `replicas * taskCount` +new reading tasks will be created. Therefore to allow for reading tasks and publishing tasks to run concurrently, there +should be a minimum capacity of: + +``` +workerCapacity = 2 * replicas * taskCount +``` + +This value is for the ideal situation in which there is at most one set of tasks publishing while another set is reading. +In some circumstances, it is possible to have multiple sets of tasks publishing simultaneously. This would happen if the +time-to-publish (generate segment, push to deep storage, loaded on historical) > `taskDuration`. This is a valid +scenario (correctness-wise) but requires additional worker capacity to support. In general, it is a good idea to have +`taskDuration` be large enough that the previous set of tasks finishes publishing before the current set begins. + +### Supervisor Persistence + +When a supervisor spec is submitted via the `POST /druid/indexer/v1/supervisor` endpoint, it is persisted in the +configured metadata database. There can only be a single supervisor per dataSource, and submitting a second spec for +the same dataSource will overwrite the previous one. + +When an overlord gains leadership, either by being started or as a result of another overlord failing, it will spawn +a supervisor for each supervisor spec in the metadata database. The supervisor will then discover running Kinesis indexing +tasks and will attempt to adopt them if they are compatible with the supervisor's configuration. If they are not +compatible because they have a different ingestion spec or shard allocation, the tasks will be killed and the +supervisor will create a new set of tasks. In this way, the supervisors are persistent across overlord restarts and +fail-overs. + +A supervisor is stopped via the `POST /druid/indexer/v1/supervisor//shutdown` endpoint. This places a +tombstone marker in the database (to prevent the supervisor from being reloaded on a restart) and then gracefully +shuts down the currently running supervisor. When a supervisor is shut down in this way, it will instruct its +managed tasks to stop reading and begin publishing their segments immediately. The call to the shutdown endpoint will +return after all tasks have been signalled to stop but before the tasks finish publishing their segments. + +### Schema/Configuration Changes + +Schema and configuration changes are handled by submitting the new supervisor spec via the same +`POST /druid/indexer/v1/supervisor` endpoint used to initially create the supervisor. The overlord will initiate a +graceful shutdown of the existing supervisor which will cause the tasks being managed by that supervisor to stop reading +and begin publishing their segments. A new supervisor will then be started which will create a new set of tasks that +will start reading from the sequence numbers where the previous now-publishing tasks left off, but using the updated schema. +In this way, configuration changes can be applied without requiring any pause in ingestion. + +### Deployment Notes + +#### On the Subject of Segments + +Each Kinesis Indexing Task puts events consumed from Kinesis Shards assigned to it in a single segment for each segment +granular interval until maxRowsPerSegment, maxTotalRows or intermediateHandoffPeriod limit is reached, at this point a new shard +for this segment granularity is created for further events. Kinesis Indexing Task also does incremental hand-offs which +means that all the segments created by a task will not be held up till the task duration is over. As soon as maxRowsPerSegment, +maxTotalRows or intermediateHandoffPeriod limit is hit, all the segments held by the task at that point in time will be handed-off +and new set of segments will be created for further events. This means that the task can run for longer durations of time +without accumulating old segments locally on Middle Manager nodes and it is encouraged to do so. + +Kinesis Indexing Service may still produce some small segments. Lets say the task duration is 4 hours, segment granularity +is set to an HOUR and Supervisor was started at 9:10 then after 4 hours at 13:10, new set of tasks will be started and +events for the interval 13:00 - 14:00 may be split across previous and new set of tasks. If you see it becoming a problem then +one can schedule re-indexing tasks be run to merge segments together into new segments of an ideal size (in the range of ~500-700 MB per segment). +Details on how to optimize the segment size can be found on [Segment size optimization](../../operations/segment-optimization.html). +There is also ongoing work to support automatic segment compaction of sharded segments as well as compaction not requiring +Hadoop (see [here](https://github.com/apache/incubator-druid/pull/5102)). + + +### Determining Fetch Settings Kinesis places the following restrictions on calls to fetch records: @@ -158,11 +368,9 @@ If the above limits are violated, AWS will throw ProvisionedThroughputExceededEx read data. When this happens, the Kinesis indexing service will pause by `fetchDelayMillis` and then attempt the call again. -## Supervisor API, Capacity Planning, Persistence, and Schema Changes - -The Kinesis indexing service uses the same supervisor API and has the same considerations for capacity planning, -persistence, and schema changes as the Kafka indexing service. For documentation on these topics, see the relevant -sections of the Kafka indexing service [documentation](http://druid.io/docs/0.10.0/development/extensions-core/kafka-ingestion.html#supervisor-api). +Internally, each indexing task maintains a buffer that stores the fetched but not yet processed recordd. `recordsPerFetch` and `fetchDelayMillis` +control this behavior. The number of records that the indexing task fetch from the buffer is controlled by `maxRecordsPerPoll`, which +determines the number of records to be processed per each ingestion loop in the task. ## Deaggregation See [issue](https://github.com/apache/incubator-druid/issues/6714) From 2a08eee10f13e3d84764dfed14f52ba9e0f95470 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 13 Dec 2018 15:00:12 -0800 Subject: [PATCH 66/87] more docs --- docs/content/development/extensions.md | 1 + .../druid/indexing/kinesis/KinesisRecordSupplier.java | 9 +++++++++ .../supervisor/SeekableStreamSupervisor.java | 4 ++-- 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md index 998f3dcd778a..951a577b5128 100644 --- a/docs/content/development/extensions.md +++ b/docs/content/development/extensions.md @@ -50,6 +50,7 @@ Core extensions are maintained by Druid committers. |druid-kafka-eight|Kafka ingest firehose (high level consumer) for realtime nodes.|[link](../development/extensions-core/kafka-eight-firehose.html)| |druid-kafka-extraction-namespace|Kafka-based namespaced lookup. Requires namespace lookup extension.|[link](../development/extensions-core/kafka-extraction-namespace.html)| |druid-kafka-indexing-service|Supervised exactly-once Kafka ingestion for the indexing service.|[link](../development/extensions-core/kafka-ingestion.html)| +|druid-kinesis-indexing-service|Supervised exactly-once Kinesis ingestion for the indexing service.|[link](../development/extensions-core/kinesis-ingestion.html)| |druid-kerberos|Kerberos authentication for druid nodes.|[link](../development/extensions-core/druid-kerberos.html)| |druid-lookups-cached-global|A module for [lookups](../querying/lookups.html) providing a jvm-global eager caching for lookups. It provides JDBC and URI implementations for fetching lookup data.|[link](../development/extensions-core/lookups-cached-global.html)| |druid-lookups-cached-single| Per lookup caching module to support the use cases where a lookup need to be isolated from the global pool of lookups |[link](../development/extensions-core/druid-lookups.html)| diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 0d48700adf3e..bbd85d2bc3cf 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -71,6 +71,10 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +/** + * This class implements a local buffer for storing fetched Kinesis records. Fetching is done + * in background threads. + */ public class KinesisRecordSupplier implements RecordSupplier { private static final EmittingLogger log = new EmittingLogger(KinesisRecordSupplier.class); @@ -307,6 +311,11 @@ public KinesisRecordSupplier( this.fetchThreads = fetchThreads; this.recordBufferSize = recordBufferSize; + /** + * the deaggregate function is implemented by the amazon-kinesis-client, whose license is not compatible with Apache. + * The work around here is to use reflection to find the deaggregate function in the classpath. See details on the + * docs page for more information on how to use deaggregation + */ if (deaggregate) { try { Class kclUserRecordclass = Class.forName("com.amazonaws.services.kinesis.clientlibrary.types.UserRecord"); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 2c9860932323..f6e96a728e13 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -122,8 +122,8 @@ * stream sequences. *

* - * @param partition id type - * @param sequence number type + * @param the type of the partition id, for example, partitions in Kafka are int type while partitions in Kinesis are String type + * @param the type of the sequence number or offsets, for example, Kafka uses long offsets while Kinesis uses String sequence numbers */ public abstract class SeekableStreamSupervisor implements Supervisor From b39f9c385d873d2cee12d0fd88e62e26e1f9b561 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Fri, 14 Dec 2018 14:47:52 -0800 Subject: [PATCH 67/87] comments --- .../extensions-core/kafka-ingestion.md | 4 +- .../extensions-core/kinesis-ingestion.md | 19 +- ...ementalPublishingKafkaIndexTaskRunner.java | 10 +- .../druid/indexing/kafka/KafkaIndexTask.java | 20 +- .../indexing/kafka/KafkaIndexTaskClient.java | 18 +- ...onfig.java => KafkaIndexTaskIOConfig.java} | 8 +- .../indexing/kafka/KafkaIndexTaskModule.java | 2 +- ...g.java => KafkaIndexTaskTuningConfig.java} | 40 +- .../indexing/kafka/KafkaRecordSupplier.java | 11 +- .../indexing/kafka/KafkaSequenceNumber.java | 20 +- .../kafka/LegacyKafkaIndexTaskRunner.java | 28 +- .../kafka/supervisor/KafkaSupervisor.java | 116 +----- .../KafkaSupervisorTuningConfig.java | 30 +- .../indexing/kafka/KafkaIOConfigTest.java | 4 +- .../indexing/kafka/KafkaIndexTaskTest.java | 72 ++-- ...va => KafkaIndexTaskTuningConfigTest.java} | 44 ++- .../kafka/supervisor/KafkaSupervisorTest.java | 36 +- .../KafkaSupervisorTuningConfigTest.java | 1 + .../indexing/kinesis/KinesisIndexTask.java | 16 +- .../kinesis/KinesisIndexTaskClient.java | 16 +- ...fig.java => KinesisIndexTaskIOConfig.java} | 8 +- .../kinesis/KinesisIndexTaskRunner.java | 10 +- ...java => KinesisIndexTaskTuningConfig.java} | 47 +-- .../kinesis/KinesisIndexingServiceModule.java | 2 +- .../kinesis/KinesisRecordSupplier.java | 7 +- .../kinesis/supervisor/KinesisSupervisor.java | 232 +---------- .../supervisor/KinesisSupervisorIOConfig.java | 6 +- .../KinesisSupervisorTuningConfig.java | 37 +- .../indexing/kinesis/KinesisIOConfigTest.java | 4 +- .../kinesis/KinesisIndexTaskTest.java | 68 ++-- ... => KinesisIndexTaskTuningConfigTest.java} | 22 +- .../supervisor/KinesisSupervisorTest.java | 30 +- .../indexing/common/IndexTaskClient.java | 37 +- .../indexing/common/TaskInfoProvider.java | 4 +- .../druid/indexing/common/task/Task.java | 12 +- .../indexing/common/task/TaskResource.java | 6 +- .../parallel/ParallelIndexTaskRunner.java | 6 +- .../overlord/HeapMemoryTaskStorage.java | 4 +- .../overlord/MetadataTaskStorage.java | 2 +- .../overlord/SingleTaskBackgroundRunner.java | 2 +- .../druid/indexing/overlord/TaskQueue.java | 2 +- .../druid/indexing/overlord/TaskRunner.java | 2 +- .../druid/indexing/overlord/TaskStorage.java | 18 +- .../overlord/TaskStorageQueryAdapter.java | 2 +- .../overlord/http/TaskPayloadResponse.java | 2 +- .../overlord/http/TaskStatusResponse.java | 2 +- .../SeekableStreamDataSourceMetadata.java | 31 +- .../SeekableStreamIndexTask.java | 29 +- .../SeekableStreamIndexTaskClient.java | 54 ++- ...a => SeekableStreamIndexTaskIOConfig.java} | 22 +- .../SeekableStreamIndexTaskRunner.java | 250 +++++++----- ... SeekableStreamIndexTaskTuningConfig.java} | 27 +- .../SeekableStreamPartitions.java | 35 +- .../common/OrderedPartitionableRecord.java | 22 +- .../common/OrderedSequenceNumber.java | 33 +- .../seekablestream/common/RecordSupplier.java | 26 +- .../common/StreamPartition.java | 10 +- .../supervisor/SeekableStreamSupervisor.java | 363 +++++++++--------- ...SeekableStreamSupervisorReportPayload.java | 14 +- .../SeekableStreamSupervisorTuningConfig.java | 4 +- .../supervisor/TaskReportData.java | 22 +- .../seekablestream/utils/RandomId.java | 37 ++ 62 files changed, 953 insertions(+), 1115 deletions(-) rename extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/{KafkaIOConfig.java => KafkaIndexTaskIOConfig.java} (93%) rename extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/{KafkaTuningConfig.java => KafkaIndexTaskTuningConfig.java} (82%) rename extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/{KafkaTuningConfigTest.java => KafkaIndexTaskTuningConfigTest.java} (76%) rename extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/{KinesisIOConfig.java => KinesisIndexTaskIOConfig.java} (95%) rename extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/{KinesisTuningConfig.java => KinesisIndexTaskTuningConfig.java} (89%) rename extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/{KinesisTuningConfigTest.java => KinesisIndexTaskTuningConfigTest.java} (92%) rename indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/{SeekableStreamIOConfig.java => SeekableStreamIndexTaskIOConfig.java} (84%) rename indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/{SeekableStreamTuningConfig.java => SeekableStreamIndexTaskTuningConfig.java} (95%) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/utils/RandomId.java diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index ebecacf15e99..43ae819c1212 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -27,7 +27,7 @@ The Kafka indexing service enables the configuration of *supervisors* on the Ove Kafka by managing the creation and lifetime of Kafka indexing tasks. These indexing tasks read events using Kafka's own partition and offset mechanism and are therefore able to provide guarantees of exactly-once ingestion. They are also able to read non-recent events from Kafka and are not subject to the window period considerations imposed on other -ingestion mechanisms. The supervisor oversees the state of the indexing tasks to coordinate handoffs, manage failures, +ingestion mechanisms using Tranquility. The supervisor oversees the state of the indexing tasks to coordinate handoffs, manage failures, and ensure that the scalability and replication requirements are maintained. This service is provided in the `druid-kafka-indexing-service` core extension (see @@ -124,8 +124,8 @@ A sample supervisor spec is shown below: |--------|-----------|---------| |`type`|The supervisor type, this should always be `kafka`.|yes| |`dataSchema`|The schema that will be used by the Kafka indexing task during ingestion, see [Ingestion Spec DataSchema](../../ingestion/ingestion-spec.html#dataschema).|yes| -|`tuningConfig`|A KafkaSupervisorTuningConfig to configure the supervisor and indexing tasks, see below.|no| |`ioConfig`|A KafkaSupervisorIOConfig to configure the supervisor and indexing tasks, see below.|yes| +|`tuningConfig`|A KafkaSupervisorTuningConfig to configure the supervisor and indexing tasks, see below.|no| ### KafkaSupervisorTuningConfig diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md index d577109bae13..a8d596f08f38 100644 --- a/docs/content/development/extensions-core/kinesis-ingestion.md +++ b/docs/content/development/extensions-core/kinesis-ingestion.md @@ -23,11 +23,11 @@ layout: doc_page # Kinesis Indexing Service -Similar to the [Kafka indexing service](./kafka-ingestion.html), The Kinesis indexing service enables the configuration of *supervisors* on the Overlord, which facilitate ingestion from +Similar to the [Kafka indexing service](./kafka-ingestion.html), the Kinesis indexing service enables the configuration of *supervisors* on the Overlord, which facilitate ingestion from Kinesis by managing the creation and lifetime of Kinesis indexing tasks. These indexing tasks read events using Kinesis's own Shards and Sequence Number mechanism and are therefore able to provide guarantees of exactly-once ingestion. They are also able to read non-recent events from Kinesis and are not subject to the window period considerations imposed on other -ingestion mechanisms. The supervisor oversees the state of the indexing tasks to coordinate handoffs, manage failures, +ingestion mechanisms using Tranquility. The supervisor oversees the state of the indexing tasks to coordinate handoffs, manage failures, and ensure that the scalability and replication requirements are maintained. The Kinesis indexing service is provided as the `druid-kinesis-indexing-service` core extension (see @@ -118,8 +118,8 @@ A sample supervisor spec is shown below: |--------|-----------|---------| |`type`|The supervisor type, this should always be `kinesis`.|yes| |`dataSchema`|The schema that will be used by the Kinesis indexing task during ingestion, see [Ingestion Spec DataSchema](../../ingestion/ingestion-spec.html#dataschema).|yes| -|`tuningConfig`|A KinesisSupervisorTuningConfig to configure the supervisor and indexing tasks, see below.|no| |`ioConfig`|A KinesisSupervisorIOConfig to configure the supervisor and indexing tasks, see below.|yes| +|`tuningConfig`|A KinesisSupervisorTuningConfig to configure the supervisor and indexing tasks, see below.|no| ### KinesisSupervisorTuningConfig @@ -350,8 +350,17 @@ Details on how to optimize the segment size can be found on [Segment size optimi There is also ongoing work to support automatic segment compaction of sharded segments as well as compaction not requiring Hadoop (see [here](https://github.com/apache/incubator-druid/pull/5102)). - ### Determining Fetch Settings +Internally, the Kinesis Indexing Service uses the Kinesis Record Supplier abstraction for fetching Kinesis data records and storing the records +locally. The way the Kinesis Record Supplier fetches records is to have a separate thread run the fetching operation per each Kinesis Shard, the +max number of threads is determined by `fetchThreads`. For example, a Kinesis stream with 3 shards will have 3 threads, each fetching from a shard separately. +There is a delay between each fetching operation, which is controlled by `fetchDelayMillis`. The maximum number of records to be fetched per thread per +operation is controlled by `recordsPerFetch`. Note that this is not the same as `maxRecordsPerPoll`. + +The records fetched by each thread will be pushed to a queue in the order that they are fetched. The records are stored in this queue until `poll()` is called +by either the supervisor or the indexing task. `poll()` will attempt to drain the internal buffer queue up to a limit of `max(maxRecordsPerPoll, q.size())`. +Here `maxRecordsPerPoll` controls the theoretical maximum records to drain out of the buffer queue, so setting this parameter to a reasonable value is essential +in preventing the queue from overflowing or memory exceeding heap size. Kinesis places the following restrictions on calls to fetch records: @@ -368,7 +377,7 @@ If the above limits are violated, AWS will throw ProvisionedThroughputExceededEx read data. When this happens, the Kinesis indexing service will pause by `fetchDelayMillis` and then attempt the call again. -Internally, each indexing task maintains a buffer that stores the fetched but not yet processed recordd. `recordsPerFetch` and `fetchDelayMillis` +Internally, each indexing task maintains a buffer that stores the fetched but not yet processed record. `recordsPerFetch` and `fetchDelayMillis` control this behavior. The number of records that the indexing task fetch from the buffer is controlled by `maxRecordsPerPoll`, which determines the number of records to be processed per each ingestion loop in the task. diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 46886914cea1..f70ac02cdd19 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -26,7 +26,6 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; @@ -60,7 +59,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner { private static final EmittingLogger log = new EmittingLogger(IncrementalPublishingKafkaIndexTaskRunner.class); - private final KafkaTuningConfig tuningConfig; + private final KafkaIndexTaskTuningConfig tuningConfig; private final KafkaIndexTask task; public IncrementalPublishingKafkaIndexTaskRunner( @@ -97,7 +96,7 @@ protected List> getRecords( TaskToolbox toolbox ) throws Exception { - // The retrying business is because the KafkaConsumer throws OffsetOutOfRangeException if the seeked-to + // Handles OffsetOutOfRangeException, which is thrown if the seeked-to // offset is not present in the topic-partition. This can happen if we're asking a task to read from data // that has not been written yet (which is totally legitimate). So let's wait for it to show up. List> records = new ArrayList<>(); @@ -113,7 +112,7 @@ protected List> getRecords( } @Override - protected SeekableStreamPartitions createSeekableStreamPartitions( + protected SeekableStreamPartitions deserializeSeekableStreamPartitionsFromMetadata( ObjectMapper mapper, Object object ) @@ -208,10 +207,9 @@ protected Type getRunnerType() @Override protected TreeMap> getCheckPointsFromContext( TaskToolbox toolbox, - SeekableStreamIndexTask task + String checkpointsString ) throws IOException { - final String checkpointsString = task.getContextValue("checkpoints"); if (checkpointsString != null) { log.info("Checkpoints [%s]", checkpointsString); return toolbox.getObjectMapper().readValue( diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 707f71a2d252..81b576e3de1f 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -51,7 +51,7 @@ public class KafkaIndexTask extends SeekableStreamIndexTask private static final EmittingLogger log = new EmittingLogger(KafkaIndexTask.class); static final long POLL_TIMEOUT_MILLIS = TimeUnit.MILLISECONDS.toMillis(100); - private final KafkaIOConfig ioConfig; + private final KafkaIndexTaskIOConfig ioConfig; private final ObjectMapper configMapper; // This value can be tuned in some tests @@ -62,14 +62,14 @@ public KafkaIndexTask( @JsonProperty("id") String id, @JsonProperty("resource") TaskResource taskResource, @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("tuningConfig") KafkaTuningConfig tuningConfig, - @JsonProperty("ioConfig") KafkaIOConfig ioConfig, + @JsonProperty("tuningConfig") KafkaIndexTaskTuningConfig tuningConfig, + @JsonProperty("ioConfig") KafkaIndexTaskIOConfig ioConfig, @JsonProperty("context") Map context, @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, @JacksonInject ObjectMapper configMapper - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { super( id, @@ -105,7 +105,7 @@ KafkaConsumer newConsumer() KafkaRecordSupplier.addConsumerPropertiesFromConfig( props, configMapper, - ((KafkaIOConfig) ioConfig).getConsumerProperties() + ((KafkaIndexTaskIOConfig) ioConfig).getConsumerProperties() ); props.setProperty("enable.auto.commit", "false"); @@ -166,7 +166,7 @@ protected KafkaRecordSupplier newTaskRecordSupplier() try { Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); - final Map props = new HashMap<>(((KafkaIOConfig) super.ioConfig).getConsumerProperties()); + final Map props = new HashMap<>(((KafkaIndexTaskIOConfig) super.ioConfig).getConsumerProperties()); props.put("auto.offset.reset", "none"); props.put("key.deserializer", ByteArrayDeserializer.class.getName()); @@ -181,9 +181,9 @@ protected KafkaRecordSupplier newTaskRecordSupplier() @Override @JsonProperty - public KafkaTuningConfig getTuningConfig() + public KafkaIndexTaskTuningConfig getTuningConfig() { - return (KafkaTuningConfig) super.getTuningConfig(); + return (KafkaIndexTaskTuningConfig) super.getTuningConfig(); } @VisibleForTesting @@ -194,8 +194,8 @@ void setPollRetryMs(long retryMs) @Override @JsonProperty("ioConfig") - public KafkaIOConfig getIOConfig() + public KafkaIndexTaskIOConfig getIOConfig() { - return (KafkaIOConfig) super.getIOConfig(); + return (KafkaIndexTaskIOConfig) super.getIOConfig(); } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java index 1162a251968c..a915f44e09d4 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -19,21 +19,15 @@ package org.apache.druid.indexing.kafka; -import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; import org.apache.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; -import java.util.Map; - -//Left here for backward compatibility public class KafkaIndexTaskClient extends SeekableStreamIndexTaskClient { - private static ObjectMapper mapper = new ObjectMapper(); - - public KafkaIndexTaskClient( + KafkaIndexTaskClient( HttpClient httpClient, ObjectMapper jsonMapper, TaskInfoProvider taskInfoProvider, @@ -55,8 +49,14 @@ public KafkaIndexTaskClient( } @Override - protected JavaType constructPartitionOffsetMapType(Class mapType) + protected Class getPartitionType() + { + return Integer.class; + } + + @Override + protected Class getSequenceType() { - return mapper.getTypeFactory().constructMapType(mapType, Integer.class, Long.class); + return Long.class; } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java similarity index 93% rename from extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java rename to extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java index 34e38e91f9c1..fc5c28751bc9 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java @@ -22,19 +22,19 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.Map; -public class KafkaIOConfig extends SeekableStreamIOConfig +public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig { private final Map consumerProperties; @JsonCreator - public KafkaIOConfig( + public KafkaIndexTaskIOConfig( @JsonProperty("taskGroupId") @Nullable Integer taskGroupId, // can be null for backward compabitility @JsonProperty("baseSequenceName") String baseSequenceName, @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, @@ -80,7 +80,7 @@ public Map getConsumerProperties() @Override public String toString() { - return "KafkaIOConfig{" + + return "KafkaIndexTaskIOConfig{" + "taskGroupId=" + getTaskGroupId() + ", baseSequenceName='" + getBaseSequenceName() + '\'' + ", startPartitions=" + getStartPartitions() + diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java index 751421057ab0..c74359fff1e9 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java @@ -43,7 +43,7 @@ public List getJacksonModules() .registerSubtypes( new NamedType(KafkaIndexTask.class, "index_kafka"), new NamedType(KafkaDataSourceMetadata.class, "kafka"), - new NamedType(KafkaIOConfig.class, "kafka"), + new NamedType(KafkaIndexTaskIOConfig.class, "kafka"), new NamedType(KafkaSupervisorTuningConfig.class, "kafka"), new NamedType(KafkaSupervisorSpec.class, "kafka") ) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java similarity index 82% rename from extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java rename to extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java index e275322e9218..8b6747a1a1f1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; @@ -29,14 +29,14 @@ import javax.annotation.Nullable; import java.io.File; -public class KafkaTuningConfig extends SeekableStreamTuningConfig +public class KafkaIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningConfig { @JsonCreator - public KafkaTuningConfig( + public KafkaIndexTaskTuningConfig( @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, - @JsonProperty("maxTotalRows") Long maxTotalRows, + @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows, @JsonProperty("intermediatePersistPeriod") @Nullable Period intermediatePersistPeriod, @JsonProperty("basePersistDirectory") @Nullable File basePersistDirectory, @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, @@ -60,7 +60,7 @@ public KafkaTuningConfig( maxTotalRows, intermediatePersistPeriod, basePersistDirectory, - maxPendingPersists, + 0, indexSpec, true, reportParseExceptions, @@ -76,33 +76,9 @@ public KafkaTuningConfig( } @Override - public KafkaTuningConfig copyOf() - { - return new KafkaTuningConfig( - getMaxRowsInMemory(), - getMaxBytesInMemory(), - getMaxRowsPerSegment(), - getMaxTotalRows(), - getIntermediatePersistPeriod(), - getBasePersistDirectory(), - 0, - getIndexSpec(), - true, - isReportParseExceptions(), - getHandoffConditionTimeout(), - isResetOffsetAutomatically(), - getSegmentWriteOutMediumFactory(), - getIntermediateHandoffPeriod(), - isLogParseExceptions(), - getMaxParseExceptions(), - getMaxSavedParseExceptions() - ); - } - - @Override - public KafkaTuningConfig withBasePersistDirectory(File dir) + public KafkaIndexTaskTuningConfig withBasePersistDirectory(File dir) { - return new KafkaTuningConfig( + return new KafkaIndexTaskTuningConfig( getMaxRowsInMemory(), getMaxBytesInMemory(), getMaxRowsPerSegment(), @@ -127,7 +103,7 @@ public KafkaTuningConfig withBasePersistDirectory(File dir) @Override public String toString() { - return "KafkaTuningConfig{" + + return "KafkaIndexTaskTuningConfig{" + "maxRowsInMemory=" + getMaxRowsInMemory() + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + ", maxBytesInMemory=" + getMaxBytesInMemory() + diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 9220cb1b02d5..4a49351f47bb 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -25,6 +25,7 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.utils.RandomId; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.metadata.PasswordProvider; @@ -198,7 +199,7 @@ private KafkaConsumer getKafkaConsumer() final Properties props = new Properties(); props.setProperty("metadata.max.age.ms", "10000"); - props.setProperty("group.id", StringUtils.format("kafka-supervisor-%s", getRandomId())); + props.setProperty("group.id", StringUtils.format("kafka-supervisor-%s", RandomId.getRandomId())); addConsumerPropertiesFromConfig(props, sortingMapper, consumerProperties); @@ -214,12 +215,4 @@ private KafkaConsumer getKafkaConsumer() } } - private static String getRandomId() - { - final StringBuilder suffix = new StringBuilder(8); - for (int i = 0; i < Integer.BYTES * 2; ++i) { - suffix.append((char) ('a' + ((RANDOM.nextInt() >>> (i * 4)) & 0x0F))); - } - return suffix.toString(); - } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java index efb953be0888..e9033521b07e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSequenceNumber.java @@ -25,14 +25,14 @@ public class KafkaSequenceNumber extends OrderedSequenceNumber { - private KafkaSequenceNumber(Long sequenceNumber, boolean isExclusive) + private KafkaSequenceNumber(Long sequenceNumber) { super(sequenceNumber, false); } public static KafkaSequenceNumber of(Long sequenceNumber) { - return new KafkaSequenceNumber(sequenceNumber, false); + return new KafkaSequenceNumber(sequenceNumber); } @Override @@ -43,20 +43,4 @@ public int compareTo( return this.get().compareTo(o.get()); } - @Override - public boolean equals(Object o) - { - if (!(o instanceof KafkaSequenceNumber)) { - return false; - } - return this.compareTo((KafkaSequenceNumber) o) == 0; - } - - @Override - public int hashCode() - { - return super.hashCode(); - } - - } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 5428a63edfc0..7f104eea3258 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -160,8 +160,8 @@ public class LegacyKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner parser; private final AuthorizerMapper authorizerMapper; private final Optional chatHandlerProvider; @@ -697,9 +697,9 @@ private boolean possiblyPause() throws InterruptedException } @Override - protected SeekableStreamPartitions createSeekableStreamPartitions( + protected SeekableStreamPartitions deserializeSeekableStreamPartitionsFromMetadata( ObjectMapper mapper, - Object obeject + Object object ) { throw new UnsupportedOperationException(); @@ -782,16 +782,6 @@ private void requestPause() pauseRequested = true; } - @Nullable - @Override - protected TreeMap> getCheckPointsFromContext( - TaskToolbox toolbox, - SeekableStreamIndexTask task - ) - { - throw new UnsupportedOperationException(); - } - @Override protected Long getSequenceNumberToStoreAfterRead(Long sequenceNumber) { @@ -1223,4 +1213,14 @@ public DateTime getStartTime(@Context final HttpServletRequest req) return startTime; } + @Nullable + @Override + protected TreeMap> getCheckPointsFromContext( + TaskToolbox toolbox, + String checkpointsString + ) + { + throw new UnsupportedOperationException(); + } + } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index dac877b5b226..d4a465c80e04 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -24,32 +24,32 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; -import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.kafka.KafkaDataSourceMetadata; -import org.apache.druid.indexing.kafka.KafkaIOConfig; import org.apache.druid.indexing.kafka.KafkaIndexTask; import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; +import org.apache.druid.indexing.kafka.KafkaIndexTaskIOConfig; +import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig; import org.apache.druid.indexing.kafka.KafkaRecordSupplier; import org.apache.druid.indexing.kafka.KafkaSequenceNumber; -import org.apache.druid.indexing.kafka.KafkaTuningConfig; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; -import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; -import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; +import org.apache.druid.indexing.seekablestream.utils.RandomId; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -64,10 +64,8 @@ import java.util.Map.Entry; import java.util.Set; import java.util.TreeMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; /** @@ -193,7 +191,7 @@ protected SeekableStreamSupervisorReportPayload createReportPaylo @Override - protected SeekableStreamIOConfig createIoConfig( + protected SeekableStreamIndexTaskIOConfig createIoConfig( int groupId, Map startPartitions, Map endPartitions, @@ -205,7 +203,7 @@ protected SeekableStreamIOConfig createIoConfig( ) { KafkaSupervisorIOConfig kafkaIoConfig = (KafkaSupervisorIOConfig) ioConfig; - return new KafkaIOConfig( + return new KafkaIndexTaskIOConfig( groupId, baseSequenceName, new SeekableStreamPartitions<>(kafkaIoConfig.getTopic(), startPartitions), @@ -224,10 +222,10 @@ protected List> createIndexTasks( String baseSequenceName, ObjectMapper sortingMapper, TreeMap> sequenceOffsets, - SeekableStreamIOConfig taskIoConfig, - SeekableStreamTuningConfig taskTuningConfig, + SeekableStreamIndexTaskIOConfig taskIoConfig, + SeekableStreamIndexTaskTuningConfig taskTuningConfig, RowIngestionMetersFactory rowIngestionMetersFactory - ) throws JsonProcessingException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) throws JsonProcessingException { final String checkpoints = sortingMapper.writerWithType(new TypeReference>>() { @@ -246,13 +244,13 @@ protected List> createIndexTasks( List> taskList = new ArrayList<>(); for (int i = 0; i < replicas; i++) { - String taskId = Joiner.on("_").join(baseSequenceName, getRandomId()); + String taskId = Joiner.on("_").join(baseSequenceName, RandomId.getRandomId()); taskList.add(new KafkaIndexTask( taskId, new TaskResource(baseSequenceName, 1), spec.getDataSchema(), - (KafkaTuningConfig) taskTuningConfig, - (KafkaIOConfig) taskIoConfig, + (KafkaIndexTaskTuningConfig) taskTuningConfig, + (KafkaIndexTaskIOConfig) taskIoConfig, context, null, null, @@ -357,23 +355,6 @@ protected Long getEndOfPartitionMarker() return END_OF_PARTITION; } - // the following are for unit testing purposes only - @Override - @VisibleForTesting - protected void runInternal() - throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException, NoSuchMethodException, - IllegalAccessException, ClassNotFoundException - { - super.runInternal(); - } - - @Override - @VisibleForTesting - protected Runnable updateCurrentAndLatestOffsets() - { - return super.updateCurrentAndLatestOffsets(); - } - @Override protected void updateLatestSequenceFromStream( RecordSupplier recordSupplier, @@ -387,41 +368,12 @@ protected void updateLatestSequenceFromStream( )); } - @Override - @VisibleForTesting - protected void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException - { - super.gracefulShutdownInternal(); - } - - @Override - @VisibleForTesting - protected void resetInternal(DataSourceMetadata dataSourceMetadata) - { - super.resetInternal(dataSourceMetadata); - } - @Override protected String baseTaskName() { return "index_kafka"; } - @Override - @VisibleForTesting - protected void moveTaskGroupToPendingCompletion(int taskGroupId) - { - super.moveTaskGroupToPendingCompletion(taskGroupId); - } - - @Override - @VisibleForTesting - protected int getNoticesQueueSize() - { - return super.getNoticesQueueSize(); - } - - @Override @VisibleForTesting public KafkaSupervisorIOConfig getIoConfig() @@ -435,46 +387,4 @@ protected void tryInit() { super.tryInit(); } - - @Override - @VisibleForTesting - protected void addTaskGroupToActivelyReadingTaskGroup( - int taskGroupId, - ImmutableMap partitionOffsets, - Optional minMsgTime, - Optional maxMsgTime, - Set tasks, - Set exclusiveStartingSequencePartitions - ) - { - super.addTaskGroupToActivelyReadingTaskGroup( - taskGroupId, - partitionOffsets, - minMsgTime, - maxMsgTime, - tasks, - exclusiveStartingSequencePartitions - ); - } - - @Override - @VisibleForTesting - protected void addTaskGroupToPendingCompletionTaskGroup( - int taskGroupId, - ImmutableMap partitionOffsets, - Optional minMsgTime, - Optional maxMsgTime, - Set tasks, - Set exclusiveStartingSequencePartitions - ) - { - super.addTaskGroupToPendingCompletionTaskGroup( - taskGroupId, - partitionOffsets, - minMsgTime, - maxMsgTime, - tasks, - exclusiveStartingSequencePartitions - ); - } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index ee1c9d1748fb..230f9ad3e710 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -20,7 +20,8 @@ package org.apache.druid.indexing.kafka.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.kafka.KafkaTuningConfig; +import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.TuningConfigs; @@ -31,7 +32,8 @@ import javax.annotation.Nullable; import java.io.File; -public class KafkaSupervisorTuningConfig extends KafkaTuningConfig implements SeekableStreamSupervisorTuningConfig +public class KafkaSupervisorTuningConfig extends KafkaIndexTaskTuningConfig + implements SeekableStreamSupervisorTuningConfig { private static final String DEFAULT_OFFSET_FETCH_PERIOD = "PT30S"; @@ -172,5 +174,27 @@ public String toString() '}'; } - + @Override + public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig() + { + return new KafkaIndexTaskTuningConfig( + getMaxRowsInMemory(), + getMaxBytesInMemory(), + getMaxRowsPerSegment(), + getMaxTotalRows(), + getIntermediatePersistPeriod(), + getBasePersistDirectory(), + 0, + getIndexSpec(), + true, + isReportParseExceptions(), + getHandoffConditionTimeout(), + isResetOffsetAutomatically(), + getSegmentWriteOutMediumFactory(), + getIntermediateHandoffPeriod(), + isLogParseExceptions(), + getMaxParseExceptions(), + getMaxSavedParseExceptions() + ); + } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java index c40a3e73bff6..7ce8df024443 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java @@ -59,7 +59,7 @@ public void testSerdeWithDefaults() throws Exception + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"}\n" + "}"; - KafkaIOConfig config = (KafkaIOConfig) mapper.readValue( + KafkaIndexTaskIOConfig config = (KafkaIndexTaskIOConfig) mapper.readValue( mapper.writeValueAsString( mapper.readValue( jsonStr, @@ -97,7 +97,7 @@ public void testSerdeWithNonDefaults() throws Exception + " \"skipOffsetGaps\": true\n" + "}"; - KafkaIOConfig config = (KafkaIOConfig) mapper.readValue( + KafkaIndexTaskIOConfig config = (KafkaIndexTaskIOConfig) mapper.readValue( mapper.writeValueAsString( mapper.readValue( jsonStr, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index d51f116674f9..91b54a13dba9 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -380,7 +380,7 @@ public void testRunAfterDataInserted() throws Exception final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -423,7 +423,7 @@ public void testRunBeforeDataInserted() throws Exception Map consumerProps = kafkaServer.consumerProperties(); final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -531,7 +531,7 @@ public void testIncrementalHandOff() throws Exception ); final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, baseSequenceName, startPartitions, @@ -658,7 +658,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception ); final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, baseSequenceName, startPartitions, @@ -798,7 +798,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception ); final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, baseSequenceName, startPartitions, @@ -900,7 +900,7 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, baseSequenceName, startPartitions, @@ -936,7 +936,7 @@ public void testRunWithMinimumMessageTime() throws Exception { final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)), @@ -990,7 +990,7 @@ public void testRunWithMaximumMessageTime() throws Exception { final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)), @@ -1054,7 +1054,7 @@ public void testRunWithTransformSpec() throws Exception ) ) ), - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 0L)), @@ -1114,7 +1114,7 @@ public void testRunOnNothing() throws Exception final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1155,7 +1155,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1207,7 +1207,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1262,7 +1262,7 @@ public void testReportParseExceptions() throws Exception final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1306,7 +1306,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1388,7 +1388,7 @@ public void testMultipleParseExceptionsFailure() throws Exception final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1448,7 +1448,7 @@ public void testRunReplicas() throws Exception { final KafkaIndexTask task1 = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1462,7 +1462,7 @@ public void testRunReplicas() throws Exception ); final KafkaIndexTask task2 = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1516,7 +1516,7 @@ public void testRunConflicting() throws Exception { final KafkaIndexTask task1 = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1530,7 +1530,7 @@ public void testRunConflicting() throws Exception ); final KafkaIndexTask task2 = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 1, "sequence1", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 3L)), @@ -1585,7 +1585,7 @@ public void testRunConflictingWithoutTransactions() throws Exception { final KafkaIndexTask task1 = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1599,7 +1599,7 @@ public void testRunConflictingWithoutTransactions() throws Exception ); final KafkaIndexTask task2 = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 1, "sequence1", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 3L)), @@ -1659,7 +1659,7 @@ public void testRunOneTaskTwoPartitions() throws Exception { final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L, 1, 0L)), @@ -1724,7 +1724,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception { final KafkaIndexTask task1 = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1738,7 +1738,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception ); final KafkaIndexTask task2 = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 1, "sequence1", new SeekableStreamPartitions<>(topic, ImmutableMap.of(1, 0L)), @@ -1794,7 +1794,7 @@ public void testRestore() throws Exception { final KafkaIndexTask task1 = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1831,7 +1831,7 @@ public void testRestore() throws Exception // Start a new task final KafkaIndexTask task2 = createTask( task1.getId(), - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1883,7 +1883,7 @@ public void testRunWithPauseAndResume() throws Exception { final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -1967,7 +1967,7 @@ public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception { final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 2L)), @@ -2006,7 +2006,7 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", new SeekableStreamPartitions<>(topic, ImmutableMap.of(0, 200L)), @@ -2059,7 +2059,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception final KafkaIndexTask task = createTask( null, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( 0, "sequence0", // task should ignore these and use sequence info sent in the context @@ -2145,7 +2145,7 @@ public boolean apply(TaskLock lock) private KafkaIndexTask createTask( final String taskId, - final KafkaIOConfig ioConfig + final KafkaIndexTaskIOConfig ioConfig ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { return createTask(taskId, DATA_SCHEMA, ioConfig); @@ -2153,7 +2153,7 @@ private KafkaIndexTask createTask( private KafkaIndexTask createTask( final String taskId, - final KafkaIOConfig ioConfig, + final KafkaIndexTaskIOConfig ioConfig, final Map context ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { @@ -2163,10 +2163,10 @@ private KafkaIndexTask createTask( private KafkaIndexTask createTask( final String taskId, final DataSchema dataSchema, - final KafkaIOConfig ioConfig + final KafkaIndexTaskIOConfig ioConfig ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { - final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( + final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( 1000, null, maxRowsPerSegment, @@ -2211,11 +2211,11 @@ private KafkaIndexTask createTask( private KafkaIndexTask createTask( final String taskId, final DataSchema dataSchema, - final KafkaIOConfig ioConfig, + final KafkaIndexTaskIOConfig ioConfig, final Map context ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { - final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( + final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( 1000, null, maxRowsPerSegment, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java similarity index 76% rename from extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaTuningConfigTest.java rename to extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java index 0acea38b1747..8ee3db67450e 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.TuningConfig; @@ -30,11 +31,11 @@ import java.io.File; -public class KafkaTuningConfigTest +public class KafkaIndexTaskTuningConfigTest { private final ObjectMapper mapper; - public KafkaTuningConfigTest() + public KafkaIndexTaskTuningConfigTest() { mapper = new DefaultObjectMapper(); mapper.registerModules((Iterable) new KafkaIndexTaskModule().getJacksonModules()); @@ -45,7 +46,7 @@ public void testSerdeWithDefaults() throws Exception { String jsonStr = "{\"type\": \"kafka\"}"; - KafkaTuningConfig config = (KafkaTuningConfig) mapper.readValue( + KafkaIndexTaskTuningConfig config = (KafkaIndexTaskTuningConfig) mapper.readValue( mapper.writeValueAsString( mapper.readValue( jsonStr, @@ -81,7 +82,7 @@ public void testSerdeWithNonDefaults() throws Exception + " \"handoffConditionTimeout\": 100\n" + "}"; - KafkaTuningConfig config = (KafkaTuningConfig) mapper.readValue( + KafkaIndexTaskTuningConfig config = (KafkaIndexTaskTuningConfig) mapper.readValue( mapper.writeValueAsString( mapper.readValue( jsonStr, @@ -103,9 +104,9 @@ public void testSerdeWithNonDefaults() throws Exception } @Test - public void testCopyOf() + public void testConvert() { - KafkaTuningConfig original = new KafkaTuningConfig( + KafkaSupervisorTuningConfig original = new KafkaSupervisorTuningConfig( 1, null, 2, @@ -122,9 +123,15 @@ public void testCopyOf() null, null, null, + null, + null, + null, + null, + null, + null, null ); - KafkaTuningConfig copy = original.copyOf(); + KafkaIndexTaskTuningConfig copy = (KafkaIndexTaskTuningConfig) original.convertToTaskTuningConfig(); Assert.assertEquals(1, copy.getMaxRowsInMemory()); Assert.assertEquals(2, copy.getMaxRowsPerSegment()); @@ -137,4 +144,27 @@ public void testCopyOf() Assert.assertEquals(true, copy.isReportParseExceptions()); Assert.assertEquals(5L, copy.getHandoffConditionTimeout()); } + + private static KafkaIndexTaskTuningConfig copy(KafkaIndexTaskTuningConfig config) + { + return new KafkaIndexTaskTuningConfig( + config.getMaxRowsInMemory(), + config.getMaxBytesInMemory(), + config.getMaxRowsPerSegment(), + config.getMaxTotalRows(), + config.getIntermediatePersistPeriod(), + config.getBasePersistDirectory(), + 0, + config.getIndexSpec(), + true, + config.isReportParseExceptions(), + config.getHandoffConditionTimeout(), + config.isResetOffsetAutomatically(), + config.getSegmentWriteOutMediumFactory(), + config.getIntermediateHandoffPeriod(), + config.isLogParseExceptions(), + config.getMaxParseExceptions(), + config.getMaxSavedParseExceptions() + ); + } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index b4833e6c501d..19b1aa7b79fd 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -45,10 +45,10 @@ import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.kafka.KafkaDataSourceMetadata; -import org.apache.druid.indexing.kafka.KafkaIOConfig; import org.apache.druid.indexing.kafka.KafkaIndexTask; import org.apache.druid.indexing.kafka.KafkaIndexTaskClient; import org.apache.druid.indexing.kafka.KafkaIndexTaskClientFactory; +import org.apache.druid.indexing.kafka.KafkaIndexTaskIOConfig; import org.apache.druid.indexing.kafka.test.TestBroker; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -280,9 +280,9 @@ public void testNoInitialState() throws Exception KafkaIndexTask task = captured.getValue(); Assert.assertEquals(dataSchema, task.getDataSchema()); - Assert.assertEquals(tuningConfig.copyOf(), task.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), task.getTuningConfig()); - KafkaIOConfig taskConfig = task.getIOConfig(); + KafkaIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); Assert.assertEquals("myCustomValue", taskConfig.getConsumerProperties().get("myCustomKey")); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); @@ -326,7 +326,7 @@ public void testSkipOffsetGaps() throws Exception verifyAll(); KafkaIndexTask task = captured.getValue(); - KafkaIOConfig taskConfig = task.getIOConfig(); + KafkaIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertTrue("skipOffsetGaps", taskConfig.isSkipOffsetGaps()); } @@ -550,7 +550,7 @@ public void testDatasourceMetadata() throws Exception verifyAll(); KafkaIndexTask task = captured.getValue(); - KafkaIOConfig taskConfig = task.getIOConfig(); + KafkaIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); Assert.assertEquals(10L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); Assert.assertEquals(20L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(1)); @@ -1160,9 +1160,9 @@ public void testBeginPublishAndQueueNextTasks() throws Exception for (Task task : captured.getValues()) { KafkaIndexTask kafkaIndexTask = (KafkaIndexTask) task; Assert.assertEquals(dataSchema, kafkaIndexTask.getDataSchema()); - Assert.assertEquals(tuningConfig.copyOf(), kafkaIndexTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), kafkaIndexTask.getTuningConfig()); - KafkaIOConfig taskConfig = kafkaIndexTask.getIOConfig(); + KafkaIndexTaskIOConfig taskConfig = kafkaIndexTask.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); @@ -1250,9 +1250,9 @@ public void testDiscoverExistingPublishingTask() throws Exception KafkaIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig.copyOf(), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); - KafkaIOConfig capturedTaskConfig = capturedTask.getIOConfig(); + KafkaIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); Assert.assertEquals("myCustomValue", capturedTaskConfig.getConsumerProperties().get("myCustomKey")); Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); @@ -1347,9 +1347,9 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() KafkaIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig.copyOf(), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); - KafkaIOConfig capturedTaskConfig = capturedTask.getIOConfig(); + KafkaIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); Assert.assertEquals("myCustomValue", capturedTaskConfig.getConsumerProperties().get("myCustomKey")); Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); @@ -1623,7 +1623,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception verifyAll(); for (Task task : captured.getValues()) { - KafkaIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig(); + KafkaIndexTaskIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig(); Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); } @@ -1716,7 +1716,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception verifyAll(); for (Task task : captured.getValues()) { - KafkaIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig(); + KafkaIndexTaskIOConfig taskConfig = ((KafkaIndexTask) task).getIOConfig(); Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(0)); Assert.assertEquals(0L, (long) taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(2)); } @@ -2662,9 +2662,9 @@ public void testFailedInitializationAndRecovery() throws Exception KafkaIndexTask task = captured.getValue(); Assert.assertEquals(dataSchema, task.getDataSchema()); - Assert.assertEquals(tuningConfig.copyOf(), task.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), task.getTuningConfig()); - KafkaIOConfig taskConfig = task.getIOConfig(); + KafkaIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); Assert.assertEquals("myCustomValue", taskConfig.getConsumerProperties().get("myCustomKey")); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); @@ -2892,14 +2892,14 @@ private KafkaIndexTask createKafkaIndexTask( SeekableStreamPartitions endPartitions, DateTime minimumMessageTime, DateTime maximumMessageTime - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { return new KafkaIndexTask( id, null, getDataSchema(dataSource), tuningConfig, - new KafkaIOConfig( + new KafkaIndexTaskIOConfig( taskGroupId, "sequenceName-" + taskGroupId, startPartitions, @@ -2986,4 +2986,6 @@ protected String generateSequenceName( return StringUtils.format("sequenceName-%d", groupId); } } + + } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java index 0a3ea95fea54..4cddabac70d0 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java @@ -117,4 +117,5 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(Duration.standardSeconds(95), config.getShutdownTimeout()); Assert.assertEquals(Duration.standardSeconds(20), config.getOffsetFetchPeriod()); } + } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 997a1ba2ee8b..522224c37b1c 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -39,13 +39,13 @@ public KinesisIndexTask( @JsonProperty("id") String id, @JsonProperty("resource") TaskResource taskResource, @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("tuningConfig") KinesisTuningConfig tuningConfig, - @JsonProperty("ioConfig") KinesisIOConfig ioConfig, + @JsonProperty("tuningConfig") KinesisIndexTaskTuningConfig tuningConfig, + @JsonProperty("ioConfig") KinesisIndexTaskIOConfig ioConfig, @JsonProperty("context") Map context, @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { super( id, @@ -77,10 +77,10 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() @Override protected KinesisRecordSupplier newTaskRecordSupplier() - throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException + throws RuntimeException { - KinesisIOConfig ioConfig = ((KinesisIOConfig) super.ioConfig); - KinesisTuningConfig tuningConfig = ((KinesisTuningConfig) super.tuningConfig); + KinesisIndexTaskIOConfig ioConfig = ((KinesisIndexTaskIOConfig) super.ioConfig); + KinesisIndexTaskTuningConfig tuningConfig = ((KinesisIndexTaskTuningConfig) super.tuningConfig); int fetchThreads = tuningConfig.getFetchThreads() != null ? tuningConfig.getFetchThreads() : Math.max(1, ioConfig.getStartPartitions().getPartitionSequenceNumberMap().size()); @@ -107,8 +107,8 @@ protected KinesisRecordSupplier newTaskRecordSupplier() @Override @JsonProperty("ioConfig") - public KinesisIOConfig getIOConfig() + public KinesisIndexTaskIOConfig getIOConfig() { - return (KinesisIOConfig) super.getIOConfig(); + return (KinesisIndexTaskIOConfig) super.getIOConfig(); } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java index 4812fc1c1c14..db9c21ef7eef 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java @@ -19,20 +19,17 @@ package org.apache.druid.indexing.kinesis; -import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; import org.apache.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; -import java.util.Map; - public class KinesisIndexTaskClient extends SeekableStreamIndexTaskClient { private static ObjectMapper mapper = new ObjectMapper(); - public KinesisIndexTaskClient( + KinesisIndexTaskClient( HttpClient httpClient, ObjectMapper jsonMapper, TaskInfoProvider taskInfoProvider, @@ -54,9 +51,16 @@ public KinesisIndexTaskClient( } @Override - protected JavaType constructPartitionOffsetMapType(Class mapType) + protected Class getPartitionType() + { + return String.class; + } + + @Override + protected Class getSequenceType() { - return mapper.getTypeFactory().constructMapType(mapType, String.class, String.class); + return String.class; } + } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java similarity index 95% rename from extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java index 95287f73cf99..65e663c5dda9 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.joda.time.DateTime; @@ -30,7 +30,7 @@ import javax.annotation.Nullable; import java.util.Set; -public class KinesisIOConfig extends SeekableStreamIOConfig +public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig { private static final boolean DEFAULT_PAUSE_AFTER_READ = true; public static final int DEFAULT_RECORDS_PER_FETCH = 4000; @@ -48,7 +48,7 @@ public class KinesisIOConfig extends SeekableStreamIOConfig private final boolean deaggregate; @JsonCreator - public KinesisIOConfig( + public KinesisIndexTaskIOConfig( @JsonProperty("taskGroupId") @Nullable Integer taskGroupId, @JsonProperty("baseSequenceName") String baseSequenceName, @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, @@ -152,7 +152,7 @@ public boolean isDeaggregate() @Override public String toString() { - return "KinesisIOConfig{" + + return "KinesisIndexTaskIOConfig{" + "baseSequenceName='" + getBaseSequenceName() + '\'' + ", startPartitions=" + getStartPartitions() + ", endPartitions=" + getEndPartitions() + diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index 9c53e15a6420..31fa15fd5004 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -26,7 +26,6 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; @@ -86,12 +85,12 @@ protected List> getRecords( } @Override - protected SeekableStreamPartitions createSeekableStreamPartitions( + protected SeekableStreamPartitions deserializeSeekableStreamPartitionsFromMetadata( ObjectMapper mapper, - Object obeject + Object object ) { - return mapper.convertValue(obeject, mapper.getTypeFactory().constructParametrizedType( + return mapper.convertValue(object, mapper.getTypeFactory().constructParametrizedType( SeekableStreamPartitions.class, SeekableStreamPartitions.class, String.class, @@ -123,10 +122,9 @@ protected Type getRunnerType() @Override protected TreeMap> getCheckPointsFromContext( TaskToolbox toolbox, - SeekableStreamIndexTask task + String checkpointsString ) throws IOException { - final String checkpointsString = task.getContextValue("checkpoints"); if (checkpointsString != null) { log.info("Checkpoints [%s]", checkpointsString); return toolbox.getObjectMapper().readValue( diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java similarity index 89% rename from extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java rename to extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java index 753e8c8809c7..72a07e54f577 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; @@ -31,7 +31,7 @@ import java.io.File; import java.util.Objects; -public class KinesisTuningConfig extends SeekableStreamTuningConfig +public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningConfig { private static final int DEFAULT_RECORD_BUFFER_SIZE = 10000; @@ -49,7 +49,7 @@ public class KinesisTuningConfig extends SeekableStreamTuningConfig private final int maxRecordsPerPoll; @JsonCreator - public KinesisTuningConfig( + public KinesisIndexTaskTuningConfig( @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, @@ -83,7 +83,7 @@ public KinesisTuningConfig( maxTotalRows, intermediatePersistPeriod, basePersistDirectory, - maxPendingPersists, + 0, indexSpec, true, reportParseExceptions, @@ -112,37 +112,6 @@ public KinesisTuningConfig( ); } - @Override - public KinesisTuningConfig copyOf() - { - return new KinesisTuningConfig( - getMaxRowsInMemory(), - getMaxBytesInMemory(), - getMaxRowsPerSegment(), - getMaxTotalRows(), - getIntermediatePersistPeriod(), - getBasePersistDirectory(), - 0, - getIndexSpec(), - true, - isReportParseExceptions(), - getHandoffConditionTimeout(), - isResetOffsetAutomatically(), - isSkipSequenceNumberAvailabilityCheck(), - getRecordBufferSize(), - getRecordBufferOfferTimeout(), - getRecordBufferFullWait(), - getFetchSequenceNumberTimeout(), - getFetchThreads(), - getSegmentWriteOutMediumFactory(), - isLogParseExceptions(), - getMaxParseExceptions(), - getMaxSavedParseExceptions(), - getMaxRecordsPerPoll(), - getIntermediateHandoffPeriod() - ); - } - @JsonProperty public int getRecordBufferSize() { @@ -180,9 +149,9 @@ public int getMaxRecordsPerPoll() } @Override - public KinesisTuningConfig withBasePersistDirectory(File dir) + public KinesisIndexTaskTuningConfig withBasePersistDirectory(File dir) { - return new KinesisTuningConfig( + return new KinesisIndexTaskTuningConfig( getMaxRowsInMemory(), getMaxBytesInMemory(), getMaxRowsPerSegment(), @@ -219,7 +188,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - KinesisTuningConfig that = (KinesisTuningConfig) o; + KinesisIndexTaskTuningConfig that = (KinesisIndexTaskTuningConfig) o; return getMaxRowsInMemory() == that.getMaxRowsInMemory() && getMaxBytesInMemory() == that.getMaxBytesInMemory() && getMaxRowsPerSegment() == that.getMaxRowsPerSegment() && @@ -274,7 +243,7 @@ public int hashCode() @Override public String toString() { - return "KinesisTuningConfig{" + + return "KinesisIndexTaskTuningConfig{" + "maxRowsInMemory=" + getMaxRowsInMemory() + ", maxBytesInMemory=" + getMaxBytesInMemory() + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java index 8abf7f3778ed..748b7fa83912 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java @@ -40,7 +40,7 @@ public List getJacksonModules() .registerSubtypes( new NamedType(KinesisIndexTask.class, "index_kinesis"), new NamedType(KinesisDataSourceMetadata.class, "kinesis"), - new NamedType(KinesisIOConfig.class, "kinesis"), + new NamedType(KinesisIndexTaskIOConfig.class, "kinesis"), new NamedType(KinesisSupervisorTuningConfig.class, "kinesis"), new NamedType(KinesisSupervisorSpec.class, "kinesis") ) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index bbd85d2bc3cf..1083c23c5c6b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -297,7 +297,7 @@ public KinesisRecordSupplier( int recordBufferFullWait, int fetchSequenceNumberTimeout, int maxRecordsPerPoll - ) throws ClassNotFoundException, IllegalAccessException, NoSuchMethodException + ) throws RuntimeException { Preconditions.checkNotNull(amazonKinesis); this.kinesis = amazonKinesis; @@ -331,7 +331,10 @@ public KinesisRecordSupplier( log.error( "cannot find class[com.amazonaws.services.kinesis.clientlibrary.types.UserRecord], " + "note that when using deaggregate=true, you must provide the Kinesis Client Library jar in the classpath"); - throw e; + throw new RuntimeException(e); + } + catch (Exception e) { + throw new RuntimeException(e); } } else { deaggregateHandle = null; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 7ba29be4d035..24748b40ce18 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -22,35 +22,34 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; -import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata; -import org.apache.druid.indexing.kinesis.KinesisIOConfig; import org.apache.druid.indexing.kinesis.KinesisIndexTask; import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; +import org.apache.druid.indexing.kinesis.KinesisIndexTaskIOConfig; +import org.apache.druid.indexing.kinesis.KinesisIndexTaskTuningConfig; import org.apache.druid.indexing.kinesis.KinesisRecordSupplier; import org.apache.druid.indexing.kinesis.KinesisSequenceNumber; -import org.apache.druid.indexing.kinesis.KinesisTuningConfig; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; -import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; -import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; +import org.apache.druid.indexing.seekablestream.utils.RandomId; import org.apache.druid.java.util.common.StringUtils; import org.joda.time.DateTime; @@ -59,9 +58,7 @@ import java.util.Map; import java.util.Set; import java.util.TreeMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeoutException; /** * Supervisor responsible for managing the KinesisIndexTask for a single dataSource. At a high level, the class accepts a @@ -104,7 +101,7 @@ public KinesisSupervisor( } @Override - protected SeekableStreamIOConfig createIoConfig( + protected SeekableStreamIndexTaskIOConfig createIoConfig( int groupId, Map startPartitions, Map endPartitions, @@ -116,7 +113,7 @@ protected SeekableStreamIOConfig createIoConfig( ) { KinesisSupervisorIOConfig ioConfig = (KinesisSupervisorIOConfig) ioConfigg; - return new KinesisIOConfig( + return new KinesisIndexTaskIOConfig( groupId, baseSequenceName, new SeekableStreamPartitions<>(ioConfig.getStream(), startPartitions), @@ -143,10 +140,10 @@ protected List> createIndexTasks( String baseSequenceName, ObjectMapper sortingMapper, TreeMap> sequenceOffsets, - SeekableStreamIOConfig taskIoConfig, - SeekableStreamTuningConfig taskTuningConfig, + SeekableStreamIndexTaskIOConfig taskIoConfig, + SeekableStreamIndexTaskTuningConfig taskTuningConfig, RowIngestionMetersFactory rowIngestionMetersFactory - ) throws JsonProcessingException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) throws JsonProcessingException { final String checkpoints = sortingMapper.writerFor(new TypeReference>>() { @@ -164,13 +161,13 @@ protected List> createIndexTasks( .build(); List> taskList = new ArrayList<>(); for (int i = 0; i < replicas; i++) { - String taskId = Joiner.on("_").join(baseSequenceName, getRandomId()); + String taskId = Joiner.on("_").join(baseSequenceName, RandomId.getRandomId()); taskList.add(new KinesisIndexTask( taskId, new TaskResource(baseSequenceName, 1), spec.getDataSchema(), - (KinesisTuningConfig) taskTuningConfig, - (KinesisIOConfig) taskIoConfig, + (KinesisIndexTaskTuningConfig) taskTuningConfig, + (KinesisIndexTaskIOConfig) taskIoConfig, context, null, null, @@ -183,10 +180,10 @@ protected List> createIndexTasks( @Override protected RecordSupplier setupRecordSupplier() - throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException + throws RuntimeException { KinesisSupervisorIOConfig ioConfig = spec.getIoConfig(); - KinesisTuningConfig taskTuningConfig = spec.getTuningConfig(); + KinesisIndexTaskTuningConfig taskTuningConfig = spec.getTuningConfig(); return new KinesisRecordSupplier( KinesisRecordSupplier.getAmazonKinesisClient( @@ -213,15 +210,7 @@ protected RecordSupplier setupRecordSupplier() @Override protected void scheduleReporting(ScheduledExecutorService reportingExec) { - // Implement this for Kinesis which uses approximate time from latest instead of offset lag -/* - reportingExec.scheduleAtFixedRate( - computeAndEmitLag(taskClient), - ioConfig.getStartDelay().getMillis() + 10000, // wait for tasks to start up - Math.max(monitorSchedulerConfig.getEmitterPeriod().getMillis(), 60 * 1000), - TimeUnit.MILLISECONDS - ); -*/ + // not yet implemented, see issue #6739 } @Override @@ -290,23 +279,6 @@ protected OrderedSequenceNumber makeSequenceNumber( return KinesisSequenceNumber.of(seq, isExclusive); } - // the following are for unit testing purposes only - @Override - @VisibleForTesting - protected void runInternal() - throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException, NoSuchMethodException, - IllegalAccessException, ClassNotFoundException - { - super.runInternal(); - } - - @Override - @VisibleForTesting - protected Runnable updateCurrentAndLatestOffsets() - { - return super.updateCurrentAndLatestOffsets(); - } - @Override protected void updateLatestSequenceFromStream( RecordSupplier recordSupplier, Set> streamPartitions @@ -315,40 +287,12 @@ protected void updateLatestSequenceFromStream( // do nothing } - @Override - @VisibleForTesting - protected void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException - { - super.gracefulShutdownInternal(); - } - - @Override - @VisibleForTesting - protected void resetInternal(DataSourceMetadata dataSourceMetadata) - { - super.resetInternal(dataSourceMetadata); - } - @Override protected String baseTaskName() { return "index_kinesis"; } - @Override - @VisibleForTesting - protected void moveTaskGroupToPendingCompletion(int taskGroupId) - { - super.moveTaskGroupToPendingCompletion(taskGroupId); - } - - @Override - @VisibleForTesting - protected int getNoticesQueueSize() - { - return super.getNoticesQueueSize(); - } - @Override protected String getNotSetMarker() { @@ -360,148 +304,4 @@ protected String getEndOfPartitionMarker() { return SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER; } - - @Override - @VisibleForTesting - protected void addTaskGroupToActivelyReadingTaskGroup( - int taskGroupId, - ImmutableMap partitionOffsets, - Optional minMsgTime, - Optional maxMsgTime, - Set tasks, - Set exclusiveStartingSequencePartitions - ) - { - super.addTaskGroupToActivelyReadingTaskGroup( - taskGroupId, - partitionOffsets, - minMsgTime, - maxMsgTime, - tasks, - exclusiveStartingSequencePartitions - ); - } - - @Override - @VisibleForTesting - protected void addTaskGroupToPendingCompletionTaskGroup( - int taskGroupId, - ImmutableMap partitionOffsets, - Optional minMsgTime, - Optional maxMsgTime, - Set tasks, - Set exclusiveStartingSequencePartitions - ) - { - super.addTaskGroupToPendingCompletionTaskGroup( - taskGroupId, - partitionOffsets, - minMsgTime, - maxMsgTime, - tasks, - exclusiveStartingSequencePartitions - ); - } - -// Implement this for Kinesis which uses approximate time from latest instead of offset lag -/* - private Runnable computeAndEmitLag(final KinesisIndexTaskClient taskClient) - { - return new Runnable() - { - @Override - public void run() - { - try { - final Map> topics = lagComputingConsumer.listTopics(); - final List partitionInfoList = topics.get(ioConfig.getStream()); - lagComputingConsumer.assign( - Lists.transform(partitionInfoList, new Function() - { - @Override - public TopicPartition apply(PartitionInfo input) - { - return new TopicPartition(ioConfig.getStream(), input.partition()); - } - }) - ); - final Map offsetsResponse = new ConcurrentHashMap<>(); - final List> futures = new ArrayList<>(); - for (TaskGroup taskGroup : taskGroups.values()) { - for (String taskId : taskGroup.taskIds()) { - futures.add(Futures.transform( - taskClient.getCurrentOffsetsAsync(taskId, false), - new Function, Void>() - { - @Override - public Void apply(Map taskResponse) - { - if (taskResponse != null) { - for (final Map.Entry partitionOffsets : taskResponse.entrySet()) { - offsetsResponse.compute(partitionOffsets.getKey(), new BiFunction() - { - @Override - public Long apply(Integer key, Long existingOffsetInMap) - { - // If existing value is null use the offset returned by task - // otherwise use the max (makes sure max offset is taken from replicas) - return existingOffsetInMap == null - ? partitionOffsets.getValue() - : Math.max(partitionOffsets.getValue(), existingOffsetInMap); - } - }); - } - } - return null; - } - } - ) - ); - } - } - // not using futureTimeoutInSeconds as its min value is 120 seconds - // and minimum emission period for this metric is 60 seconds - Futures.successfulAsList(futures).get(30, TimeUnit.SECONDS); - - // for each partition, seek to end to get the highest offset - // check the offsetsResponse map for the latest consumed offset - // if partition info not present in offsetsResponse then use lastCurrentOffsets map - // if not present there as well, fail the compute - - long lag = 0; - for (PartitionInfo partitionInfo : partitionInfoList) { - long diff; - final TopicPartition topicPartition = new TopicPartition(ioConfig.getStream(), partitionInfo.partition()); - lagComputingConsumer.seekToEnd(ImmutableList.of(topicPartition)); - if (offsetsResponse.get(topicPartition.partition()) != null) { - diff = lagComputingConsumer.position(topicPartition) - offsetsResponse.get(topicPartition.partition()); - lastCurrentOffsets.put(topicPartition.partition(), offsetsResponse.get(topicPartition.partition())); - } else if (lastCurrentOffsets.get(topicPartition.partition()) != null) { - diff = lagComputingConsumer.position(topicPartition) - lastCurrentOffsets.get(topicPartition.partition()); - } else { - throw new ISE("Could not find latest consumed offset for partition [%d]", topicPartition.partition()); - } - lag += diff; - log.debug( - "Topic - [%s] Partition - [%d] : Partition lag [%,d], Total lag so far [%,d]", - topicPartition.topic(), - topicPartition.partition(), - diff, - lag - ); - } - emitter.emit( - ServiceMetricEvent.builder().setDimension("dataSource", dataSource).build("ingest/kinesis/lag", lag) - ); - } - catch (InterruptedException e) { - // do nothing, probably we are shutting down - } - catch (Exception e) { - log.warn(e, "Unable to compute Kinesis lag"); - } - } - }; - } - */ } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java index dcfb7a083693..559c04a2e9c0 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.indexing.kinesis.KinesisIOConfig; +import org.apache.druid.indexing.kinesis.KinesisIndexTaskIOConfig; import org.apache.druid.indexing.kinesis.KinesisRegion; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.joda.time.Period; @@ -88,8 +88,8 @@ public KinesisSupervisorIOConfig( this.endpoint = endpoint != null ? endpoint : (region != null ? region.getEndpoint() : KinesisRegion.US_EAST_1.getEndpoint()); - this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : KinesisIOConfig.DEFAULT_RECORDS_PER_FETCH; - this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : KinesisIOConfig.DEFAULT_FETCH_DELAY_MILLIS; + this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : KinesisIndexTaskIOConfig.DEFAULT_RECORDS_PER_FETCH; + this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : KinesisIndexTaskIOConfig.DEFAULT_FETCH_DELAY_MILLIS; this.awsAccessKeyId = awsAccessKeyId; this.awsSecretAccessKey = awsSecretAccessKey; this.awsAssumedRoleArn = awsAssumedRoleArn; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 8bfa7762ffec..498481ec828e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -20,7 +20,8 @@ package org.apache.druid.indexing.kinesis.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.kinesis.KinesisTuningConfig; +import org.apache.druid.indexing.kinesis.KinesisIndexTaskTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -30,7 +31,8 @@ import javax.annotation.Nullable; import java.io.File; -public class KinesisSupervisorTuningConfig extends KinesisTuningConfig implements SeekableStreamSupervisorTuningConfig +public class KinesisSupervisorTuningConfig extends KinesisIndexTaskTuningConfig + implements SeekableStreamSupervisorTuningConfig { private final Integer workerThreads; private final Integer chatThreads; @@ -171,4 +173,35 @@ public String toString() '}'; } + @Override + public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig() + { + return new KinesisIndexTaskTuningConfig( + getMaxRowsInMemory(), + getMaxBytesInMemory(), + getMaxRowsPerSegment(), + getMaxTotalRows(), + getIntermediatePersistPeriod(), + getBasePersistDirectory(), + 0, + getIndexSpec(), + true, + isReportParseExceptions(), + getHandoffConditionTimeout(), + isResetOffsetAutomatically(), + isSkipSequenceNumberAvailabilityCheck(), + getRecordBufferSize(), + getRecordBufferOfferTimeout(), + getRecordBufferFullWait(), + getFetchSequenceNumberTimeout(), + getFetchThreads(), + getSegmentWriteOutMediumFactory(), + isLogParseExceptions(), + getMaxParseExceptions(), + getMaxSavedParseExceptions(), + getMaxRecordsPerPoll(), + getIntermediateHandoffPeriod() + ); + } + } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index 4a13de87ffea..65710c9942f2 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -59,7 +59,7 @@ public void testSerdeWithDefaults() throws Exception + " \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}}\n" + "}"; - KinesisIOConfig config = (KinesisIOConfig) mapper.readValue( + KinesisIndexTaskIOConfig config = (KinesisIndexTaskIOConfig) mapper.readValue( mapper.writeValueAsString( mapper.readValue( jsonStr, @@ -119,7 +119,7 @@ public void testSerdeWithNonDefaults() throws Exception + " \"deaggregate\": true\n" + "}"; - KinesisIOConfig config = (KinesisIOConfig) mapper.readValue( + KinesisIndexTaskIOConfig config = (KinesisIndexTaskIOConfig) mapper.readValue( mapper.writeValueAsString( mapper.readValue( jsonStr, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index cdac31a16f8c..4efae927a875 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -369,7 +369,7 @@ public void testRunAfterDataInserted() throws Exception final KinesisIndexTask task = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -453,7 +453,7 @@ public void testRunBeforeDataInserted() throws Exception final KinesisIndexTask task = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -571,7 +571,7 @@ public void testIncrementalHandOff() throws Exception ); final KinesisIndexTask task = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, baseSequenceName, startPartitions, @@ -713,7 +713,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception final KinesisIndexTask task = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, baseSequenceName, startPartitions, @@ -830,7 +830,7 @@ public void testRunWithMinimumMessageTime() throws Exception final KinesisIndexTask task = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -916,7 +916,7 @@ public void testRunWithMaximumMessageTime() throws Exception final KinesisIndexTask task = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1012,7 +1012,7 @@ public void testRunWithTransformSpec() throws Exception ) ) ), - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1095,7 +1095,7 @@ public void testRunOnNothing() throws Exception final KinesisIndexTask task = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1161,7 +1161,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception final KinesisIndexTask task = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1241,7 +1241,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio final KinesisIndexTask task = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1321,7 +1321,7 @@ public void testReportParseExceptions() throws Exception final KinesisIndexTask task = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1390,7 +1390,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception final KinesisIndexTask task = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1503,7 +1503,7 @@ public void testMultipleParseExceptionsFailure() throws Exception final KinesisIndexTask task = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1595,7 +1595,7 @@ public void testRunReplicas() throws Exception final KinesisIndexTask task1 = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1623,7 +1623,7 @@ public void testRunReplicas() throws Exception ); final KinesisIndexTask task2 = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1712,7 +1712,7 @@ public void testRunConflicting() throws Exception final KinesisIndexTask task1 = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1740,7 +1740,7 @@ public void testRunConflicting() throws Exception ); final KinesisIndexTask task2 = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence1", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1829,7 +1829,7 @@ public void testRunConflictingWithoutTransactions() throws Exception final KinesisIndexTask task1 = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1857,7 +1857,7 @@ public void testRunConflictingWithoutTransactions() throws Exception ); final KinesisIndexTask task2 = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence1", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -1943,7 +1943,7 @@ public void testRunOneTaskTwoPartitions() throws Exception final KinesisIndexTask task = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence1", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -2040,7 +2040,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception final KinesisIndexTask task1 = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -2068,7 +2068,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception ); final KinesisIndexTask task2 = createTask( null, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence1", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -2160,7 +2160,7 @@ public void testRestore() throws Exception final KinesisIndexTask task1 = createTask( "task1", - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -2222,7 +2222,7 @@ public void testRestore() throws Exception // Start a new task final KinesisIndexTask task2 = createTask( task1.getId(), - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -2309,7 +2309,7 @@ public void testRunWithPauseAndResume() throws Exception final KinesisIndexTask task = createTask( "task1", - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -2450,7 +2450,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception final KinesisIndexTask task = createTask( "task1", - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequence0", new SeekableStreamPartitions<>(stream, ImmutableMap.of( @@ -2552,7 +2552,7 @@ public boolean apply(TaskLock lock) private KinesisIndexTask createTask( final String taskId, - final KinesisIOConfig ioConfig + final KinesisIndexTaskIOConfig ioConfig ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { return createTask(taskId, DATA_SCHEMA, ioConfig); @@ -2560,7 +2560,7 @@ private KinesisIndexTask createTask( private KinesisIndexTask createTask( final String taskId, - final KinesisIOConfig ioConfig, + final KinesisIndexTaskIOConfig ioConfig, final Map context ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { @@ -2570,10 +2570,10 @@ private KinesisIndexTask createTask( private KinesisIndexTask createTask( final String taskId, final DataSchema dataSchema, - final KinesisIOConfig ioConfig + final KinesisIndexTaskIOConfig ioConfig ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { - final KinesisTuningConfig tuningConfig = new KinesisTuningConfig( + final KinesisIndexTaskTuningConfig tuningConfig = new KinesisIndexTaskTuningConfig( 1000, null, maxRowsPerSegment, @@ -2619,11 +2619,11 @@ private KinesisIndexTask createTask( private KinesisIndexTask createTask( final String taskId, final DataSchema dataSchema, - final KinesisIOConfig ioConfig, + final KinesisIndexTaskIOConfig ioConfig, final Map context ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException { - final KinesisTuningConfig tuningConfig = new KinesisTuningConfig( + final KinesisIndexTaskTuningConfig tuningConfig = new KinesisIndexTaskTuningConfig( maxRowsInMemory, null, maxRowsPerSegment, @@ -3008,8 +3008,8 @@ public TestableKinesisIndexTask( @JsonProperty("id") String id, @JsonProperty("resource") TaskResource taskResource, @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("tuningConfig") KinesisTuningConfig tuningConfig, - @JsonProperty("ioConfig") KinesisIOConfig ioConfig, + @JsonProperty("tuningConfig") KinesisIndexTaskTuningConfig tuningConfig, + @JsonProperty("ioConfig") KinesisIndexTaskIOConfig ioConfig, @JsonProperty("context") Map context, @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java similarity index 92% rename from extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java rename to extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index 67a7f3d51666..283dd152da2b 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.TuningConfig; @@ -34,11 +35,11 @@ import java.io.File; -public class KinesisTuningConfigTest +public class KinesisIndexTaskTuningConfigTest { private final ObjectMapper mapper; - public KinesisTuningConfigTest() + public KinesisIndexTaskTuningConfigTest() { mapper = new DefaultObjectMapper(); mapper.registerModules((Iterable) new KinesisIndexingServiceModule().getJacksonModules()); @@ -52,7 +53,7 @@ public void testSerdeWithDefaults() throws Exception { String jsonStr = "{\"type\": \"kinesis\"}"; - KinesisTuningConfig config = (KinesisTuningConfig) mapper.readValue( + KinesisIndexTaskTuningConfig config = (KinesisIndexTaskTuningConfig) mapper.readValue( mapper.writeValueAsString( mapper.readValue( jsonStr, @@ -102,7 +103,7 @@ public void testSerdeWithNonDefaults() throws Exception + " \"fetchThreads\": 2\n" + "}"; - KinesisTuningConfig config = (KinesisTuningConfig) mapper.readValue( + KinesisIndexTaskTuningConfig config = (KinesisIndexTaskTuningConfig) mapper.readValue( mapper.writeValueAsString( mapper.readValue( jsonStr, @@ -159,9 +160,9 @@ public void testResetOffsetAndSkipSequenceNotBothTrue() throws Exception } @Test - public void testCopyOf() + public void testConvert() { - KinesisTuningConfig original = new KinesisTuningConfig( + KinesisSupervisorTuningConfig original = new KinesisSupervisorTuningConfig( 1, (long) 3, 2, @@ -175,6 +176,12 @@ public void testCopyOf() 5L, true, false, + null, + null, + null, + null, + null, + null, 1000, 500, 500, @@ -184,10 +191,9 @@ public void testCopyOf() null, null, null, - null, null ); - KinesisTuningConfig copy = original.copyOf(); + KinesisIndexTaskTuningConfig copy = (KinesisIndexTaskTuningConfig) original.convertToTaskTuningConfig(); Assert.assertEquals(1, copy.getMaxRowsInMemory()); Assert.assertEquals(3, copy.getMaxBytesInMemory()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 944a2a118b3c..8752a54912ce 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -43,10 +43,10 @@ import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata; -import org.apache.druid.indexing.kinesis.KinesisIOConfig; import org.apache.druid.indexing.kinesis.KinesisIndexTask; import org.apache.druid.indexing.kinesis.KinesisIndexTaskClient; import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; +import org.apache.druid.indexing.kinesis.KinesisIndexTaskIOConfig; import org.apache.druid.indexing.kinesis.KinesisRecordSupplier; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -272,9 +272,9 @@ public void testNoInitialState() throws Exception KinesisIndexTask task = captured.getValue(); Assert.assertEquals(dataSchema, task.getDataSchema()); - Assert.assertEquals(tuningConfig.copyOf(), task.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), task.getTuningConfig()); - KinesisIOConfig taskConfig = task.getIOConfig(); + KinesisIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); Assert.assertFalse("minimumMessageTime", taskConfig.getMinimumMessageTime().isPresent()); @@ -577,7 +577,7 @@ public void testDatasourceMetadata() throws Exception verifyAll(); KinesisIndexTask task = captured.getValue(); - KinesisIOConfig taskConfig = task.getIOConfig(); + KinesisIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); Assert.assertEquals( "2", @@ -1407,9 +1407,9 @@ public void testBeginPublishAndQueueNextTasks() throws Exception for (Task task : captured.getValues()) { KinesisIndexTask KinesisIndexTask = (KinesisIndexTask) task; Assert.assertEquals(dataSchema, KinesisIndexTask.getDataSchema()); - Assert.assertEquals(tuningConfig.copyOf(), KinesisIndexTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), KinesisIndexTask.getTuningConfig()); - KinesisIOConfig taskConfig = KinesisIndexTask.getIOConfig(); + KinesisIndexTaskIOConfig taskConfig = KinesisIndexTask.getIOConfig(); Assert.assertEquals("sequenceName-1", taskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", taskConfig.isUseTransaction()); @@ -1547,9 +1547,9 @@ public void testDiscoverExistingPublishingTask() throws Exception KinesisIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig.copyOf(), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); - KinesisIOConfig capturedTaskConfig = capturedTask.getIOConfig(); + KinesisIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(cloud.localstack.TestUtils.TEST_ACCESS_KEY, capturedTaskConfig.getAwsAccessKeyId()); Assert.assertEquals(cloud.localstack.TestUtils.TEST_SECRET_KEY, capturedTaskConfig.getAwsSecretAccessKey()); Assert.assertEquals(LocalstackTestRunner.getEndpointKinesis(), capturedTaskConfig.getEndpoint()); @@ -1688,9 +1688,9 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() KinesisIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig.copyOf(), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); - KinesisIOConfig capturedTaskConfig = capturedTask.getIOConfig(); + KinesisIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(cloud.localstack.TestUtils.TEST_ACCESS_KEY, capturedTaskConfig.getAwsAccessKeyId()); Assert.assertEquals(cloud.localstack.TestUtils.TEST_SECRET_KEY, capturedTaskConfig.getAwsSecretAccessKey()); Assert.assertEquals(LocalstackTestRunner.getEndpointKinesis(), capturedTaskConfig.getEndpoint()); @@ -2063,7 +2063,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception verifyAll(); for (Task task : captured.getValues()) { - KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); + KinesisIndexTaskIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); Assert.assertEquals( "0", taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) @@ -2184,7 +2184,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception verifyAll(); for (Task task : captured.getValues()) { - KinesisIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); + KinesisIndexTaskIOConfig taskConfig = ((KinesisIndexTask) task).getIOConfig(); Assert.assertEquals( "0", taskConfig.getStartPartitions().getPartitionSequenceNumberMap().get(shardId1) @@ -2966,7 +2966,7 @@ public void testCheckpointForInactiveTaskGroup() @Test(timeout = 60_000L) public void testCheckpointForUnknownTaskGroup() - throws InterruptedException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException + throws InterruptedException { supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); @@ -3633,14 +3633,14 @@ private KinesisIndexTask createKinesisIndexTask( SeekableStreamPartitions endPartitions, DateTime minimumMessageTime, DateTime maximumMessageTime - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { return new KinesisIndexTask( id, null, getDataSchema(dataSource), tuningConfig, - new KinesisIOConfig( + new KinesisIndexTaskIOConfig( null, "sequenceName-" + taskGroupId, startPartitions, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java index 9214d6003163..592059b7ecda 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.type.TypeFactory; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; @@ -56,6 +57,7 @@ import java.net.Socket; import java.net.URL; import java.nio.charset.StandardCharsets; +import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; @@ -159,6 +161,33 @@ protected T deserialize(String content, Class typeReference) throws IOExc return objectMapper.readValue(content, typeReference); } + protected T deserializeMap(String content, Class mapClass, Class keyClass, Class valueClass) + throws IOException + { + return deserialize(content, objectMapper.getTypeFactory().constructMapType(mapClass, keyClass, valueClass)); + } + + protected T deserializeNestedValueMap( + String content, + Class mapClass, + Class keyClass, + Class valueMapClass, + Class valueMapClassKey, + Class valueMapClassValue + ) + throws IOException + { + TypeFactory factory = objectMapper.getTypeFactory(); + return deserialize( + content, + factory.constructMapType( + mapClass, + factory.constructType(keyClass), + factory.constructMapType(valueMapClass, valueMapClassKey, valueMapClassValue) + ) + ); + } + protected byte[] serialize(Object value) throws JsonProcessingException { return objectMapper.writeValueAsBytes(value); @@ -334,7 +363,7 @@ private FullResponseHolder submitRequest( // Since workers are free to move tasks around to different ports, there is a chance that a task may have been // moved but our view of its location has not been updated yet from ZK. To detect this case, we send a header // identifying our expected recipient in the request; if this doesn't correspond to the worker we messaged, the - // worker will return an HTTP 404 with its ID in the response header. If we get a mismatching task ID, then + // worker will return an HTTP 404 with its RandomId in the response header. If we get a mismatching task RandomId, then // we will wait for a short period then retry the request indefinitely, expecting the task's location to // eventually be updated. @@ -406,14 +435,14 @@ private FullResponseHolder submitRequest(Request request) throws IOException, Ch /** * Throws if it's possible to throw the given Throwable. - * + *

* - The input throwable shouldn't be null. * - If Throwable is an {@link ExecutionException}, this calls itself recursively with the cause of ExecutionException. * - If Throwable is an {@link IOException} or a {@link ChannelException}, this simply throws it. * - If Throwable is an {@link InterruptedException}, this interrupts the current thread and throws a RuntimeException - * wrapping the InterruptedException + * wrapping the InterruptedException * - Otherwise, this simply returns the given Throwable. - * + *

* Note that if the given Throable is an ExecutionException, this can return the cause of ExecutionException. */ private RuntimeException throwIfPossible(Throwable t) throws IOException, ChannelException diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskInfoProvider.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskInfoProvider.java index 748cbd560dec..7c5ef4f90e2f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskInfoProvider.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskInfoProvider.java @@ -26,14 +26,14 @@ public interface TaskInfoProvider { /** - * @param id the task ID + * @param id the task RandomId * * @return a TaskLocation associated with the task or TaskLocation.unknown() if no associated entry could be found */ TaskLocation getTaskLocation(String id); /** - * @param id the task ID + * @param id the task RandomId * * @return an Optional.of() with the current status of the task or Optional.absent() if the task could not be found */ diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 3ed45482cc46..33ce22370636 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -35,7 +35,7 @@ * Represents a task that can run on a worker. The general contracts surrounding Tasks are: *

    *
  • Tasks must operate on a single datasource.
  • - *
  • Tasks should be immutable, since the task ID is used as a proxy for the task in many locations.
  • + *
  • Tasks should be immutable, since the task RandomId is used as a proxy for the task in many locations.
  • *
  • Task IDs must be unique. This can be done by naming them using UUIDs or the current timestamp.
  • *
  • Tasks are each part of a "task group", which is a set of tasks that can share interval locks. These are * useful for producing sharded segments.
  • @@ -64,17 +64,17 @@ public interface Task { /** - * Returns ID of this task. Must be unique across all tasks ever created. + * Returns RandomId of this task. Must be unique across all tasks ever created. * - * @return task ID + * @return task RandomId */ String getId(); /** - * Returns group ID of this task. Tasks with the same group ID can share locks. If tasks do not need to share locks, - * a common convention is to set group ID equal to task ID. + * Returns group RandomId of this task. Tasks with the same group RandomId can share locks. If tasks do not need to share locks, + * a common convention is to set group RandomId equal to task RandomId. * - * @return task group ID + * @return task group RandomId */ String getGroupId(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskResource.java index e74c4d038300..c771de667a3f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskResource.java @@ -40,9 +40,9 @@ public TaskResource( } /** - * Returns availability group ID of this task. Tasks the same availability group cannot be assigned to the same - * worker. If tasks do not have this restriction, a common convention is to set the availability group ID to the - * task ID. + * Returns availability group RandomId of this task. Tasks the same availability group cannot be assigned to the same + * worker. If tasks do not have this restriction, a common convention is to set the availability group RandomId to the + * task RandomId. * * @return task availability group */ diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java index 085d2f119b2a..08e0c0054e0f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java @@ -79,19 +79,19 @@ public interface ParallelIndexTaskRunner List> getCompleteSubTaskSpecs(); /** - * Returns the {@link SubTaskSpec} of the given ID or null if it's not found. + * Returns the {@link SubTaskSpec} of the given RandomId or null if it's not found. */ @Nullable SubTaskSpec getSubTaskSpec(String subTaskSpecId); /** - * Returns {@link SubTaskSpecStatus} of the given ID or null if it's not found. + * Returns {@link SubTaskSpecStatus} of the given RandomId or null if it's not found. */ @Nullable SubTaskSpecStatus getSubTaskState(String subTaskSpecId); /** - * Returns {@link TaskHistory} of the given ID or null if it's not found. + * Returns {@link TaskHistory} of the given RandomId or null if it's not found. */ @Nullable TaskHistory getCompleteSubTaskSpecAttemptHistory(String subTaskSpecId); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index a800a35f2d9f..bdf3c72ac3f1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -77,7 +77,7 @@ public void insert(Task task, TaskStatus status) throws EntryExistsException Preconditions.checkNotNull(status, "status"); Preconditions.checkArgument( task.getId().equals(status.getId()), - "Task/Status ID mismatch[%s/%s]", + "Task/Status RandomId mismatch[%s/%s]", task.getId(), status.getId() ); @@ -121,7 +121,7 @@ public void setStatus(TaskStatus status) Preconditions.checkNotNull(status, "status"); final String taskid = status.getId(); - Preconditions.checkState(tasks.containsKey(taskid), "Task ID must already be present: %s", taskid); + Preconditions.checkState(tasks.containsKey(taskid), "Task RandomId must already be present: %s", taskid); Preconditions.checkState(tasks.get(taskid).getStatus().isRunnable(), "Task status must be runnable: %s", taskid); log.info("Updating task %s to status: %s", taskid, status); tasks.put(taskid, tasks.get(taskid).withStatus(status)); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 8690c68ded0d..5360cdf31d2b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -125,7 +125,7 @@ public void insert(final Task task, final TaskStatus status) throws EntryExistsE Preconditions.checkNotNull(status, "status"); Preconditions.checkArgument( task.getId().equals(status.getId()), - "Task/Status ID mismatch[%s/%s]", + "Task/Status RandomId mismatch[%s/%s]", task.getId(), status.getId() ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunner.java index a9b631713d39..76f25e7ae4dc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunner.java @@ -275,7 +275,7 @@ public ListenableFuture run(final Task task) * There might be a race between {@link #run(Task)} and this method, but it shouldn't happen in real applications * because this method is called only in unit tests. See TaskLifecycleTest. * - * @param taskid task ID to clean up resources for + * @param taskid task RandomId to clean up resources for */ @Override public void shutdown(final String taskid, String reason) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java index 9a9c41dfbb83..5125854d0402 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java @@ -387,7 +387,7 @@ public void shutdown(final String taskId, String reasonFormat, Object... args) * @param taskStatus new task status * * @throws NullPointerException if task or status is null - * @throws IllegalArgumentException if the task ID does not match the status ID + * @throws IllegalArgumentException if the task RandomId does not match the status RandomId * @throws IllegalStateException if this queue is currently shut down */ private void notifyStatus(final Task task, final TaskStatus taskStatus, String reasonFormat, Object... args) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java index 81596596daa6..caa3883ee09e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java @@ -81,7 +81,7 @@ public interface TaskRunner * Inform the task runner it can clean up any resources associated with a task. This implies shutdown of any * currently-running tasks. * - * @param taskid task ID to clean up resources for + * @param taskid task RandomId to clean up resources for * @param reason reason to kill this task */ void shutdown(String taskid, String reason); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java index d8a4806bb57c..64866dc9269c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java @@ -38,7 +38,7 @@ public interface TaskStorage * * @param task task to add * @param status task status - * @throws EntryExistsException if the task ID already exists + * @throws EntryExistsException if the task RandomId already exists */ void insert(Task task, TaskStatus status) throws EntryExistsException; @@ -52,7 +52,7 @@ public interface TaskStorage /** * Persists lock state in the storage facility. - * @param taskid task ID + * @param taskid task RandomId * @param taskLock lock state */ void addLock(String taskid, TaskLock taskLock); @@ -70,7 +70,7 @@ public interface TaskStorage * Removes lock state from the storage facility. It is harmless to keep old locks in the storage facility, but * this method can help reclaim wasted space. * - * @param taskid task ID + * @param taskid task RandomId * @param taskLock lock state */ void removeLock(String taskid, TaskLock taskLock); @@ -83,21 +83,21 @@ public interface TaskStorage void removeTasksOlderThan(long timestamp); /** - * Returns task as stored in the storage facility. If the task ID does not exist, this will return an + * Returns task as stored in the storage facility. If the task RandomId does not exist, this will return an * absentee Optional. * * NOTE: This method really feels like it should be combined with {@link #getStatus}. Expect that in the future. * - * @param taskid task ID + * @param taskid task RandomId * @return optional task */ Optional getTask(String taskid); /** - * Returns task status as stored in the storage facility. If the task ID does not exist, this will return + * Returns task status as stored in the storage facility. If the task RandomId does not exist, this will return * an absentee Optional. * - * @param taskid task ID + * @param taskid task RandomId * @return task status */ Optional getStatus(String taskid); @@ -119,7 +119,7 @@ public interface TaskStorage /** * Returns all actions taken by a task. * - * @param taskid task ID + * @param taskid task RandomId * @return list of task actions */ @Deprecated @@ -162,7 +162,7 @@ List> getRecentlyCreatedAlreadyFinishedTaskInfo( /** * Returns a list of locks for a particular task. * - * @param taskid task ID + * @param taskid task RandomId * @return list of TaskLocks for the given task */ List getLocks(String taskid); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java index 6a12b40bc11b..61fcb8d89d14 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -90,7 +90,7 @@ public TaskInfo getTaskInfo(String taskId) * with the result set perhaps growing boundlessly and we do not do anything to protect against that. Use at your * own risk and know that at some point, we might adjust this to actually enforce some sort of limits. * - * @param taskid task ID + * @param taskid task RandomId * @return set of segments created by the specified task */ @Deprecated diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskPayloadResponse.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskPayloadResponse.java index b73f9d9f2e5e..ab5e9c2536da 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskPayloadResponse.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskPayloadResponse.java @@ -27,7 +27,7 @@ public class TaskPayloadResponse { - private final String task; // Task ID, named "task" in the JSONification of this class. + private final String task; // Task RandomId, named "task" in the JSONification of this class. private final Task payload; @JsonCreator diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskStatusResponse.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskStatusResponse.java index 82e3335c7e26..dfbcc226fd8c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskStatusResponse.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskStatusResponse.java @@ -28,7 +28,7 @@ public class TaskStatusResponse { - private final String task; // Task ID, named "task" in the JSONification of this class. + private final String task; // Task RandomId, named "task" in the JSONification of this class. @Nullable private final TaskStatusPlus status; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java index 96d2894accb1..2501048c6e73 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java @@ -28,20 +28,21 @@ import java.util.Map; import java.util.Objects; -public abstract class SeekableStreamDataSourceMetadata implements DataSourceMetadata +public abstract class SeekableStreamDataSourceMetadata + implements DataSourceMetadata { - private final SeekableStreamPartitions seekableStreamPartitions; + private final SeekableStreamPartitions seekableStreamPartitions; @JsonCreator public SeekableStreamDataSourceMetadata( - @JsonProperty("partitions") SeekableStreamPartitions seekableStreamPartitions + @JsonProperty("partitions") SeekableStreamPartitions seekableStreamPartitions ) { this.seekableStreamPartitions = seekableStreamPartitions; } @JsonProperty("partitions") - public SeekableStreamPartitions getSeekableStreamPartitions() + public SeekableStreamPartitions getSeekableStreamPartitions() { return seekableStreamPartitions; } @@ -75,17 +76,20 @@ public DataSourceMetadata plus(DataSourceMetadata other) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; + final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) { // Same stream, merge sequences. - final Map newMap = new HashMap<>(); + final Map newMap = new HashMap<>(); - for (Map.Entry entry : seekableStreamPartitions.getPartitionSequenceNumberMap().entrySet()) { + for (Map.Entry entry : seekableStreamPartitions.getPartitionSequenceNumberMap() + .entrySet()) { newMap.put(entry.getKey(), entry.getValue()); } - for (Map.Entry entry : that.getSeekableStreamPartitions().getPartitionSequenceNumberMap().entrySet()) { + for (Map.Entry entry : that.getSeekableStreamPartitions() + .getPartitionSequenceNumberMap() + .entrySet()) { newMap.put(entry.getKey(), entry.getValue()); } @@ -109,13 +113,14 @@ public DataSourceMetadata minus(DataSourceMetadata other) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; + final SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) other; if (that.getSeekableStreamPartitions().getStream().equals(seekableStreamPartitions.getStream())) { // Same stream, remove partitions present in "that" from "this" - final Map newMap = new HashMap<>(); + final Map newMap = new HashMap<>(); - for (Map.Entry entry : seekableStreamPartitions.getPartitionSequenceNumberMap().entrySet()) { + for (Map.Entry entry : seekableStreamPartitions.getPartitionSequenceNumberMap() + .entrySet()) { if (!that.getSeekableStreamPartitions().getPartitionSequenceNumberMap().containsKey(entry.getKey())) { newMap.put(entry.getKey(), entry.getValue()); } @@ -155,8 +160,8 @@ public String toString() '}'; } - protected abstract SeekableStreamDataSourceMetadata createConcreteDataSourceMetaData( + protected abstract SeekableStreamDataSourceMetadata createConcreteDataSourceMetaData( String streamId, - Map newMap + Map newMap ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 4b30f4e6dbec..3f4653c225e1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -62,18 +62,18 @@ import java.util.concurrent.ThreadLocalRandom; -public abstract class SeekableStreamIndexTask extends AbstractTask +public abstract class SeekableStreamIndexTask extends AbstractTask implements ChatHandler { public static final long LOCK_ACQUIRE_TIMEOUT_SECONDS = 15; private static final Random RANDOM = ThreadLocalRandom.current(); private final EmittingLogger log = new EmittingLogger(this.getClass()); - private final SeekableStreamIndexTaskRunner runner; + private final SeekableStreamIndexTaskRunner runner; protected final DataSchema dataSchema; protected final InputRowParser parser; - protected final SeekableStreamTuningConfig tuningConfig; - protected final SeekableStreamIOConfig ioConfig; + protected final SeekableStreamIndexTaskTuningConfig tuningConfig; + protected final SeekableStreamIndexTaskIOConfig ioConfig; protected final Optional chatHandlerProvider; protected final String type; protected final Map context; @@ -86,14 +86,14 @@ public SeekableStreamIndexTask( @JsonProperty("id") String id, @JsonProperty("resource") TaskResource taskResource, @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("tuningConfig") SeekableStreamTuningConfig tuningConfig, - @JsonProperty("ioConfig") SeekableStreamIOConfig ioConfig, + @JsonProperty("tuningConfig") SeekableStreamIndexTaskTuningConfig tuningConfig, + @JsonProperty("ioConfig") SeekableStreamIndexTaskIOConfig ioConfig, @JsonProperty("context") Map context, @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, String type - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { super( id == null ? makeTaskId(dataSchema.getDataSource(), RANDOM.nextInt(), type) : id, @@ -155,13 +155,13 @@ public DataSchema getDataSchema() } @JsonProperty - public SeekableStreamTuningConfig getTuningConfig() + public SeekableStreamIndexTaskTuningConfig getTuningConfig() { return tuningConfig; } @JsonProperty("ioConfig") - public SeekableStreamIOConfig getIOConfig() + public SeekableStreamIndexTaskIOConfig getIOConfig() { return ioConfig; } @@ -279,11 +279,12 @@ public boolean withinMinMaxRecordTime(final InputRow row) return !beforeMinimumMessageTime && !afterMaximumMessageTime; } - protected abstract SeekableStreamIndexTaskRunner createTaskRunner(); + protected abstract SeekableStreamIndexTaskRunner createTaskRunner(); - protected abstract RecordSupplier newTaskRecordSupplier() throws ClassNotFoundException, - NoSuchMethodException, - IllegalAccessException; + protected abstract RecordSupplier newTaskRecordSupplier() + throws ClassNotFoundException, + NoSuchMethodException, + IllegalAccessException; @VisibleForTesting public Appenderator getAppenderator() @@ -292,7 +293,7 @@ public Appenderator getAppenderator() } @VisibleForTesting - public SeekableStreamIndexTaskRunner getRunner() + public SeekableStreamIndexTaskRunner getRunner() { return runner; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 5493e45825fc..36c9ec60e488 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.seekablestream; -import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.ListenableFuture; @@ -43,7 +42,7 @@ import java.util.Map; import java.util.TreeMap; -public abstract class SeekableStreamIndexTaskClient extends IndexTaskClient +public abstract class SeekableStreamIndexTaskClient extends IndexTaskClient { private final EmittingLogger log = new EmittingLogger(this.getClass()); @@ -102,7 +101,7 @@ public boolean resume(final String id) } - public Map pause(final String id) + public Map pause(final String id) { log.debug("Pause task[%s]", id); @@ -117,7 +116,7 @@ public Map pause(final String id) if (response.getStatus().equals(HttpResponseStatus.OK)) { log.info("Task [%s] paused successfully", id); - return deserialize(response.getContent(), constructPartitionOffsetMapType(Map.class)); + return deserializeMap(response.getContent(), Map.class, getPartitionType(), getSequenceType()); } while (true) { @@ -212,7 +211,7 @@ public Map getMovingAverages(final String id) } } - public Map getCurrentOffsets(final String id, final boolean retry) + public Map getCurrentOffsets(final String id, final boolean retry) { log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry); @@ -224,7 +223,7 @@ public Map getCurrentOffsets(final String id, final null, retry ); - return deserialize(response.getContent(), constructPartitionOffsetMapType(Map.class)); + return deserializeMap(response.getContent(), Map.class, getPartitionType(), getSequenceType()); } catch (NoTaskLocationException e) { return ImmutableMap.of(); @@ -234,14 +233,18 @@ public Map getCurrentOffsets(final String id, final } } - public TreeMap> getCheckpoints(final String id, final boolean retry) + public TreeMap> getCheckpoints(final String id, final boolean retry) { log.debug("GetCheckpoints task[%s] retry[%s]", id, retry); try { final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "checkpoints", null, retry); - return deserialize( + return deserializeNestedValueMap( response.getContent(), - constructCheckpointMapType() + TreeMap.class, + Integer.class, + Map.class, + getPartitionType(), + getSequenceType() ); } catch (NoTaskLocationException e) { @@ -252,7 +255,7 @@ public TreeMap> getCheckpoints(final S } } - public ListenableFuture>> getCheckpointsAsync( + public ListenableFuture>> getCheckpointsAsync( final String id, final boolean retry ) @@ -260,13 +263,13 @@ public ListenableFuture>> getC return doAsync(() -> getCheckpoints(id, retry)); } - public Map getEndOffsets(final String id) + public Map getEndOffsets(final String id) { log.debug("GetEndOffsets task[%s]", id); try { final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "offsets/end", null, true); - return deserialize(response.getContent(), constructPartitionOffsetMapType(Map.class)); + return deserializeMap(response.getContent(), Map.class, getPartitionType(), getSequenceType()); } catch (NoTaskLocationException e) { return ImmutableMap.of(); @@ -278,7 +281,7 @@ public Map getEndOffsets(final String id) public boolean setEndOffsets( final String id, - final Map endOffsets, + final Map endOffsets, final boolean finalize ) throws IOException { @@ -318,26 +321,29 @@ public ListenableFuture getStartTimeAsync(final String id) } - public ListenableFuture> pauseAsync(final String id) + public ListenableFuture> pauseAsync(final String id) { return doAsync(() -> pause(id)); } public ListenableFuture setEndOffsetsAsync( final String id, - final Map endOffsets, + final Map endOffsets, final boolean finalize ) { return doAsync(() -> setEndOffsets(id, endOffsets, finalize)); } - public ListenableFuture> getCurrentOffsetsAsync(final String id, final boolean retry) + public ListenableFuture> getCurrentOffsetsAsync( + final String id, + final boolean retry + ) { return doAsync(() -> getCurrentOffsets(id, retry)); } - public ListenableFuture> getEndOffsetsAsync(final String id) + public ListenableFuture> getEndOffsetsAsync(final String id) { return doAsync(() -> getEndOffsets(id)); } @@ -354,17 +360,9 @@ public ListenableFuture getStatusAsync(fin return doAsync(() -> getStatus(id)); } - private JavaType constructCheckpointMapType() - { - ObjectMapper mapper = new ObjectMapper(); - return mapper.getTypeFactory() - .constructMapType( - TreeMap.class, - mapper.getTypeFactory().constructType(Integer.class), - constructPartitionOffsetMapType(TreeMap.class) - ); - } + protected abstract Class getPartitionType(); - protected abstract JavaType constructPartitionOffsetMapType(Class mapType); + protected abstract Class getSequenceType(); } + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java similarity index 84% rename from indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java index a6fc881fc532..c4a79898ec1b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java @@ -30,7 +30,7 @@ import java.util.Collections; import java.util.Set; -public abstract class SeekableStreamIOConfig implements IOConfig +public abstract class SeekableStreamIndexTaskIOConfig implements IOConfig { private static final boolean DEFAULT_USE_TRANSACTION = true; private static final boolean DEFAULT_SKIP_OFFSET_GAPS = false; @@ -38,25 +38,25 @@ public abstract class SeekableStreamIOConfig implem @Nullable private final Integer taskGroupId; private final String baseSequenceName; - private final SeekableStreamPartitions startPartitions; - private final SeekableStreamPartitions endPartitions; + private final SeekableStreamPartitions startPartitions; + private final SeekableStreamPartitions endPartitions; private final boolean useTransaction; private final Optional minimumMessageTime; private final Optional maximumMessageTime; private final boolean skipOffsetGaps; - private final Set exclusiveStartSequenceNumberPartitions; + private final Set exclusiveStartSequenceNumberPartitions; @JsonCreator - public SeekableStreamIOConfig( + public SeekableStreamIndexTaskIOConfig( @JsonProperty("taskGroupId") @Nullable Integer taskGroupId, // can be null for backward compabitility @JsonProperty("baseSequenceName") String baseSequenceName, - @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, - @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, + @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, + @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, @JsonProperty("useTransaction") Boolean useTransaction, @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, @JsonProperty("skipOffsetGaps") Boolean skipOffsetGaps, - @JsonProperty("exclusiveStartSequenceNumberPartitions") Set exclusiveStartSequenceNumberPartitions + @JsonProperty("exclusiveStartSequenceNumberPartitions") Set exclusiveStartSequenceNumberPartitions ) { this.taskGroupId = taskGroupId; @@ -88,7 +88,7 @@ public SeekableStreamIOConfig( // sequence number for certain partitions are discarded because they've already been // read by a previous task @JsonProperty - public Set getExclusiveStartSequenceNumberPartitions() + public Set getExclusiveStartSequenceNumberPartitions() { return exclusiveStartSequenceNumberPartitions; } @@ -107,13 +107,13 @@ public String getBaseSequenceName() } @JsonProperty - public SeekableStreamPartitions getStartPartitions() + public SeekableStreamPartitions getStartPartitions() { return startPartitions; } @JsonProperty - public SeekableStreamPartitions getEndPartitions() + public SeekableStreamPartitions getEndPartitions() { return endPartitions; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index ecab0dafef2b..184e5170fc77 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -127,10 +127,10 @@ /** * Interface for abstracting the indexing task run logic. * - * @param Partition Number Type - * @param Sequence Number Type + * @param Partition Number Type + * @param Sequence Number Type */ -public abstract class SeekableStreamIndexTaskRunner implements ChatHandler +public abstract class SeekableStreamIndexTaskRunner implements ChatHandler { public enum Status { @@ -152,9 +152,9 @@ protected enum Type private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; private static final String METADATA_PUBLISH_PARTITIONS = "publishPartitions"; - private final Map endOffsets; - private final Map currOffsets = new ConcurrentHashMap<>(); - private final Map lastPersistedOffsets = new ConcurrentHashMap<>(); + private final Map endOffsets; + private final Map currOffsets = new ConcurrentHashMap<>(); + private final Map lastPersistedOffsets = new ConcurrentHashMap<>(); // The pause lock and associated conditions are to support coordination between the Jetty threads and the main // ingestion loop. The goal is to provide callers of the API a guarantee that if pause() returns successfully @@ -190,9 +190,9 @@ protected enum Type protected final Lock pollRetryLock = new ReentrantLock(); protected final Condition isAwaitingRetry = pollRetryLock.newCondition(); - private final SeekableStreamIndexTask task; - private final SeekableStreamIOConfig ioConfig; - private final SeekableStreamTuningConfig tuningConfig; + private final SeekableStreamIndexTask task; + private final SeekableStreamIndexTaskIOConfig ioConfig; + private final SeekableStreamIndexTaskTuningConfig tuningConfig; private final InputRowParser parser; private final AuthorizerMapper authorizerMapper; private final Optional chatHandlerProvider; @@ -203,8 +203,8 @@ protected enum Type private final Set publishingSequences = Sets.newConcurrentHashSet(); private final List> publishWaitList = new ArrayList<>(); private final List> handOffWaitList = new ArrayList<>(); - private final Map initialOffsetsSnapshot = new HashMap<>(); - private final Set exclusiveStartingPartitions = new HashSet<>(); + private final Map initialOffsetsSnapshot = new HashMap<>(); + private final Set exclusiveStartingPartitions = new HashSet<>(); private volatile DateTime startTime; private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread) @@ -221,7 +221,7 @@ protected enum Type private volatile Throwable backgroundThreadException; public SeekableStreamIndexTaskRunner( - final SeekableStreamIndexTask task, + final SeekableStreamIndexTask task, final InputRowParser parser, final AuthorizerMapper authorizerMapper, final Optional chatHandlerProvider, @@ -272,14 +272,17 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception if (!restoreSequences()) { - final TreeMap> checkpoints = getCheckPointsFromContext(toolbox, task); + final TreeMap> checkpoints = getCheckPointsFromContext( + toolbox, + task.getContextValue("checkpoints") + ); if (checkpoints != null) { boolean exclusive = false; - Iterator>> sequenceOffsets = checkpoints.entrySet() - .iterator(); - Map.Entry> previous = sequenceOffsets.next(); + Iterator>> sequenceOffsets = checkpoints.entrySet() + .iterator(); + Map.Entry> previous = sequenceOffsets.next(); while (sequenceOffsets.hasNext()) { - Map.Entry> current = sequenceOffsets.next(); + Map.Entry> current = sequenceOffsets.next(); sequences.add(new SequenceMetadata( previous.getKey(), StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()), @@ -347,7 +350,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception ); Throwable caughtExceptionOuter = null; - try (final RecordSupplier recordSupplier = task.newTaskRecordSupplier()) { + try (final RecordSupplier recordSupplier = task.newTaskRecordSupplier()) { toolbox.getDataSegmentServerAnnouncer().announce(); toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); @@ -373,7 +376,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception } else { @SuppressWarnings("unchecked") final Map restoredMetadataMap = (Map) restoredMetadata; - final SeekableStreamPartitions restoredNextPartitions = createSeekableStreamPartitions( + final SeekableStreamPartitions restoredNextPartitions = deserializeSeekableStreamPartitionsFromMetadata( toolbox.getObjectMapper(), restoredMetadataMap.get(METADATA_NEXT_PARTITIONS) ); @@ -418,7 +421,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception // Set up committer. final Supplier committerSupplier = () -> { - final Map snapshot = ImmutableMap.copyOf(currOffsets); + final Map snapshot = ImmutableMap.copyOf(currOffsets); lastPersistedOffsets.clear(); lastPersistedOffsets.putAll(snapshot); @@ -446,7 +449,7 @@ public void run() // restart publishing of sequences (if any) maybePersistAndPublishSequences(committerSupplier); - Set> assignment = assignPartitions(recordSupplier); + Set> assignment = assignPartitions(recordSupplier); possiblyResetDataSourceMetadata(recordSupplier, assignment); seekToStartingSequence(recordSupplier, assignment); @@ -497,7 +500,7 @@ public void run() // calling getRecord() ensures that exceptions specific to kafka/kinesis like OffsetOutOfRangeException // are handled in the subclasses - List> records = getRecords( + List> records = getRecords( recordSupplier, toolbox ); @@ -505,11 +508,11 @@ public void run() stillReading = !assignment.isEmpty(); SequenceMetadata sequenceToCheckpoint = null; - for (OrderedPartitionableRecord record : records) { + for (OrderedPartitionableRecord record : records) { // for Kafka, the end offsets are exclusive, so skip it - if (getRunnerType() == Type.KAFKA && createSequenceNumber(record.getSequenceNumber()).equals( - createSequenceNumber(endOffsets.get(record.getPartitionId())))) { + if (getRunnerType() == Type.KAFKA && createSequenceNumber(record.getSequenceNumber()).compareTo( + createSequenceNumber(endOffsets.get(record.getPartitionId()))) == 0) { continue; } @@ -1041,13 +1044,13 @@ private void maybePersistAndPublishSequences(Supplier committerSuppli } } - private Set> assignPartitions( - RecordSupplier recordSupplier + private Set> assignPartitions( + RecordSupplier recordSupplier ) { - final Set> assignment = new HashSet<>(); - for (Map.Entry entry : currOffsets.entrySet()) { - final SequenceType endOffset = endOffsets.get(entry.getKey()); + final Set> assignment = new HashSet<>(); + for (Map.Entry entry : currOffsets.entrySet()) { + final SequenceOffsetType endOffset = endOffsets.get(entry.getKey()); if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(endOffset) || SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) || createSequenceNumber(entry.getValue()).compareTo(createSequenceNumber(endOffset)) < 0) { @@ -1070,11 +1073,12 @@ private Set> assignPartitions( private void seekToStartingSequence( - RecordSupplier recordSupplier, Set> partitions + RecordSupplier recordSupplier, + Set> partitions ) { - for (final StreamPartition partition : partitions) { - final SequenceType sequence = currOffsets.get(partition.getPartitionId()); + for (final StreamPartition partition : partitions) { + final SequenceOffsetType sequence = currOffsets.get(partition.getPartitionId()); log.info("Seeking partition[%s] to sequence[%s].", partition.getPartitionId(), sequence); recordSupplier.seek(partition, sequence); } @@ -1123,22 +1127,14 @@ private boolean possiblyPause() throws InterruptedException } private void possiblyResetDataSourceMetadata( - RecordSupplier recordSupplier, - Set> assignment + RecordSupplier recordSupplier, + Set> assignment ) { - for (final StreamPartition streamPartition : assignment) { - SequenceType sequence = currOffsets.get(streamPartition.getPartitionId()); + for (final StreamPartition streamPartition : assignment) { + SequenceOffsetType sequence = currOffsets.get(streamPartition.getPartitionId()); if (!tuningConfig.isSkipSequenceNumberAvailabilityCheck()) { - SequenceType earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); - /* - if (earliestSequenceNumber == null) { - log.warn( - "unable to verify sequence number[%s] availability, unable to fetch earliest sequence number", - sequence - ); - } - */ + SequenceOffsetType earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); if (earliestSequenceNumber == null || createSequenceNumber(earliestSequenceNumber).compareTo(createSequenceNumber(sequence)) > 0) { if (tuningConfig.isResetOffsetAutomatically()) { @@ -1207,12 +1203,12 @@ private void requestPause() protected void sendResetRequestAndWait( - Map, SequenceType> outOfRangePartitions, + Map, SequenceOffsetType> outOfRangePartitions, TaskToolbox taskToolbox ) throws IOException { - Map partitionOffsetMap = outOfRangePartitions + Map partitionOffsetMap = outOfRangePartitions .entrySet().stream().collect(Collectors.toMap(x -> x.getKey().getPartitionId(), Map.Entry::getValue)); boolean result = taskToolbox @@ -1335,13 +1331,13 @@ public Status getStatus() @GET @Path("/offsets/current") @Produces(MediaType.APPLICATION_JSON) - public Map getCurrentOffsets(@Context final HttpServletRequest req) + public Map getCurrentOffsets(@Context final HttpServletRequest req) { authorizationCheck(req, Action.READ); return getCurrentOffsets(); } - public Map getCurrentOffsets() + public Map getCurrentOffsets() { return currOffsets; } @@ -1349,13 +1345,13 @@ public Map getCurrentOffsets() @GET @Path("/offsets/end") @Produces(MediaType.APPLICATION_JSON) - public Map getEndOffsetsHTTP(@Context final HttpServletRequest req) + public Map getEndOffsetsHTTP(@Context final HttpServletRequest req) { authorizationCheck(req, Action.READ); return getEndOffsets(); } - public Map getEndOffsets() + public Map getEndOffsets() { return endOffsets; } @@ -1365,7 +1361,7 @@ public Map getEndOffsets() @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) public Response setEndOffsetsHTTP( - Map sequences, + Map sequences, @QueryParam("finish") @DefaultValue("true") final boolean finish, // this field is only for internal purposes, shouldn't be usually set by users @Context final HttpServletRequest req @@ -1415,7 +1411,7 @@ public Response getUnparseableEvents( @VisibleForTesting public Response setEndOffsets( - Map sequenceNumbers, + Map sequenceNumbers, boolean finish // this field is only for internal purposes, shouldn't be usually set by users ) throws InterruptedException { @@ -1442,12 +1438,12 @@ public Response setEndOffsets( final SequenceMetadata latestSequence = sequences.get(sequences.size() - 1); // if a partition has not been read yet (contained in initialOffsetsSnapshot), then // do not mark the starting sequence number as exclusive - Set exclusivePartitions = sequenceNumbers.keySet() - .stream() - .filter(x -> !initialOffsetsSnapshot.containsKey(x) - || ioConfig.getExclusiveStartSequenceNumberPartitions() - .contains(x)) - .collect(Collectors.toSet()); + Set exclusivePartitions = sequenceNumbers.keySet() + .stream() + .filter(x -> !initialOffsetsSnapshot.containsKey(x) + || ioConfig.getExclusiveStartSequenceNumberPartitions() + .contains(x)) + .collect(Collectors.toSet()); if ((latestSequence.getStartOffsets().equals(sequenceNumbers) && latestSequence.exclusiveStartPartitions.equals( exclusivePartitions) && !finish) || @@ -1467,7 +1463,7 @@ public Response setEndOffsets( .build(); } - for (Map.Entry entry : sequenceNumbers.entrySet()) { + for (Map.Entry entry : sequenceNumbers.entrySet()) { if (createSequenceNumber(entry.getValue()).compareTo(createSequenceNumber(currOffsets.get(entry.getKey()))) < 0) { return Response.status(Response.Status.BAD_REQUEST) @@ -1532,13 +1528,15 @@ private void resetNextCheckpointTime() @GET @Path("/checkpoints") @Produces(MediaType.APPLICATION_JSON) - public Map> getCheckpointsHTTP(@Context final HttpServletRequest req) + public Map> getCheckpointsHTTP( + @Context final HttpServletRequest req + ) { authorizationCheck(req, Action.READ); return getCheckpoints(); } - private Map> getCheckpoints() + private Map> getCheckpoints() { return new TreeMap<>(sequences.stream() .collect(Collectors.toMap( @@ -1657,8 +1655,8 @@ private class SequenceMetadata { private final int sequenceId; private final String sequenceName; - private final Set exclusiveStartPartitions; - private final Set assignments; + private final Set exclusiveStartPartitions; + private final Set assignments; private final boolean sentinel; private boolean checkpointed; @@ -1667,18 +1665,18 @@ private class SequenceMetadata * {@link #setEndOffsets)} can be called by both the main thread and the HTTP thread. */ protected final ReentrantLock lock = new ReentrantLock(); - protected final Map startOffsets; - protected final Map endOffsets; + protected final Map startOffsets; + protected final Map endOffsets; @JsonCreator public SequenceMetadata( @JsonProperty("sequenceId") int sequenceId, @JsonProperty("sequenceName") String sequenceName, - @JsonProperty("startOffsets") Map startOffsets, - @JsonProperty("endOffsets") Map endOffsets, + @JsonProperty("startOffsets") Map startOffsets, + @JsonProperty("endOffsets") Map endOffsets, @JsonProperty("checkpointed") boolean checkpointed, - @JsonProperty("exclusiveStartPartitions") Set exclusiveStartPartitions + @JsonProperty("exclusiveStartPartitions") Set exclusiveStartPartitions ) { Preconditions.checkNotNull(sequenceName); @@ -1697,7 +1695,7 @@ public SequenceMetadata( } @JsonProperty - public Set getExclusiveStartPartitions() + public Set getExclusiveStartPartitions() { return exclusiveStartPartitions; } @@ -1727,13 +1725,13 @@ public String getSequenceName() } @JsonProperty - public Map getStartOffsets() + public Map getStartOffsets() { return startOffsets; } @JsonProperty - public Map getEndOffsets() + public Map getEndOffsets() { lock.lock(); try { @@ -1750,7 +1748,7 @@ public boolean isSentinel() return sentinel; } - void setEndOffsets(Map newEndOffsets) + void setEndOffsets(Map newEndOffsets) { lock.lock(); try { @@ -1762,7 +1760,7 @@ void setEndOffsets(Map newEndOffsets) } } - void updateAssignments(Map nextPartitionOffset) + void updateAssignments(Map nextPartitionOffset) { lock.lock(); try { @@ -1785,13 +1783,14 @@ boolean isOpen() return !assignments.isEmpty(); } - boolean canHandle(OrderedPartitionableRecord record) + boolean canHandle(OrderedPartitionableRecord record) { lock.lock(); try { - final OrderedSequenceNumber partitionEndOffset = createSequenceNumber(endOffsets.get(record.getPartitionId())); - final OrderedSequenceNumber partitionStartOffset = createSequenceNumber(startOffsets.get(record.getPartitionId())); - final OrderedSequenceNumber recordOffset = createSequenceNumber(record.getSequenceNumber()); + final OrderedSequenceNumber partitionEndOffset = createSequenceNumber(endOffsets.get(record.getPartitionId())); + final OrderedSequenceNumber partitionStartOffset = createSequenceNumber(startOffsets.get( + record.getPartitionId())); + final OrderedSequenceNumber recordOffset = createSequenceNumber(record.getSequenceNumber()); if (!isOpen() || recordOffset == null || partitionEndOffset == null || partitionStartOffset == null) { return false; } @@ -1830,7 +1829,10 @@ public String toString() } } - Supplier getCommitterSupplier(String stream, Map lastPersistedOffsets) + Supplier getCommitterSupplier( + String stream, + Map lastPersistedOffsets + ) { // Set up committer. return () -> @@ -1854,8 +1856,8 @@ public Object getMetadata() // This is done because this committer would be persisting only sub set of segments // corresponding to the current sequence. Generally, lastPersistedOffsets should already // cover endOffsets but just to be sure take max of sequences and persist that - for (Map.Entry partitionOffset : endOffsets.entrySet()) { - SequenceType newOffsets = partitionOffset.getValue(); + for (Map.Entry partitionOffset : endOffsets.entrySet()) { + SequenceOffsetType newOffsets = partitionOffset.getValue(); if (lastPersistedOffsets.containsKey(partitionOffset.getKey()) && createSequenceNumber(lastPersistedOffsets.get(partitionOffset.getKey())).compareTo( createSequenceNumber(newOffsets)) > 0) { @@ -1891,7 +1893,7 @@ public void run() TransactionalSegmentPublisher createPublisher(TaskToolbox toolbox, boolean useTransaction) { return (segments, commitMetadata) -> { - final SeekableStreamPartitions finalPartitions = createSeekableStreamPartitions( + final SeekableStreamPartitions finalPartitions = deserializeSeekableStreamPartitionsFromMetadata( toolbox.getObjectMapper(), ((Map) Preconditions .checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_PUBLISH_PARTITIONS) @@ -1930,8 +1932,8 @@ TransactionalSegmentPublisher createPublisher(TaskToolbox toolbox, boolean useTr } private boolean verifyInitialRecordAndSkipExclusivePartition( - final OrderedPartitionableRecord record, - final Map intialSequenceSnapshot + final OrderedPartitionableRecord record, + final Map intialSequenceSnapshot ) { if (intialSequenceSnapshot.containsKey(record.getPartitionId())) { @@ -1965,30 +1967,88 @@ private boolean verifyInitialRecordAndSkipExclusivePartition( return true; } + /** + * deserailizes the checkpoints into of Map> + * + * @param toolbox task toolbox + * @param checkpointsString the json-serialized checkpoint string + * + * @return checkpoint + * + * @throws IOException jsonProcessingException + */ @Nullable - protected abstract TreeMap> getCheckPointsFromContext( + protected abstract TreeMap> getCheckPointsFromContext( TaskToolbox toolbox, - SeekableStreamIndexTask task + String checkpointsString ) throws IOException; - protected abstract SequenceType getSequenceNumberToStoreAfterRead(SequenceType sequenceNumber); + /** + * Calculates the sequence number used to update `currentOffsets` after finishing reading a record. + * In Kafka this returns sequenceNumeber + 1 since that's the next expected offset + * In Kinesis this simply returns sequenceNumber, since the sequence numbers in Kinesis are not + * contiguous and finding the next sequence number requires an expensive API call + * + * @param sequenceNumber the sequence number that has already been processed + * + * @return next sequence number to be stored + */ + protected abstract SequenceOffsetType getSequenceNumberToStoreAfterRead(SequenceOffsetType sequenceNumber); - protected abstract SeekableStreamPartitions createSeekableStreamPartitions( + /** + * deserialzies stored metadata into SeekableStreamPartitions + * + * @param mapper json objectMapper + * @param object metadata + * + * @return SeekableStreamPartitions + */ + protected abstract SeekableStreamPartitions deserializeSeekableStreamPartitionsFromMetadata( ObjectMapper mapper, - Object obeject + Object object ); + /** + * polls the next set of records from the recordSupplier, the main purpose of having a separate method here + * is to catch and handle exceptions specific to Kafka/Kinesis + * + * @param recordSupplier + * @param toolbox + * + * @return list of records polled, can be empty but cannot be null + * + * @throws Exception + */ @NotNull - protected abstract List> getRecords( - RecordSupplier recordSupplier, + protected abstract List> getRecords( + RecordSupplier recordSupplier, TaskToolbox toolbox ) throws Exception; - protected abstract SeekableStreamDataSourceMetadata createDataSourceMetadata( - SeekableStreamPartitions partitions + /** + * creates specific implementations of kafka/kinesis datasource metadata + * + * @param partitions partitions used to create the datasource metadata + * + * @return datasource metadata + */ + protected abstract SeekableStreamDataSourceMetadata createDataSourceMetadata( + SeekableStreamPartitions partitions ); - protected abstract OrderedSequenceNumber createSequenceNumber(SequenceType sequenceNumber); + /** + * create a specific implementation of Kafka/Kinesis sequence number/offset used for comparison mostly + * + * @param sequenceNumber + * + * @return + */ + protected abstract OrderedSequenceNumber createSequenceNumber(SequenceOffsetType sequenceNumber); + /** + * get the type{Kafka, Kinesis} of the TaskRunner, used mostly for Kafka/Kinesis specifc logic + * + * @return Type + */ protected abstract Type getRunnerType(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java similarity index 95% rename from indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java index adb727bacbb6..5c1696085e26 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java @@ -32,7 +32,7 @@ import java.io.File; import java.util.Objects; -public abstract class SeekableStreamTuningConfig implements TuningConfig, AppenderatorConfig +public abstract class SeekableStreamIndexTaskTuningConfig implements TuningConfig, AppenderatorConfig { private static final int DEFAULT_MAX_ROWS_PER_SEGMENT = 5_000_000; private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false; @@ -61,7 +61,7 @@ public abstract class SeekableStreamTuningConfig implements TuningConfig, Append private final int maxSavedParseExceptions; @JsonCreator - public SeekableStreamTuningConfig( + public SeekableStreamIndexTaskTuningConfig( @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, @@ -131,8 +131,6 @@ public SeekableStreamTuningConfig( : logParseExceptions; } - public abstract SeekableStreamTuningConfig copyOf(); - @Override @JsonProperty public int getMaxRowsInMemory() @@ -258,7 +256,7 @@ public boolean isSkipSequenceNumberAvailabilityCheck() return skipSequenceNumberAvailabilityCheck; } - public abstract SeekableStreamTuningConfig withBasePersistDirectory(File dir); + public abstract SeekableStreamIndexTaskTuningConfig withBasePersistDirectory(File dir); @Override public boolean equals(Object o) @@ -269,22 +267,24 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - SeekableStreamTuningConfig that = (SeekableStreamTuningConfig) o; + SeekableStreamIndexTaskTuningConfig that = (SeekableStreamIndexTaskTuningConfig) o; return maxRowsInMemory == that.maxRowsInMemory && - maxRowsPerSegment == that.maxRowsPerSegment && maxBytesInMemory == that.maxBytesInMemory && + maxRowsPerSegment == that.maxRowsPerSegment && maxPendingPersists == that.maxPendingPersists && reportParseExceptions == that.reportParseExceptions && handoffConditionTimeout == that.handoffConditionTimeout && resetOffsetAutomatically == that.resetOffsetAutomatically && + skipSequenceNumberAvailabilityCheck == that.skipSequenceNumberAvailabilityCheck && + logParseExceptions == that.logParseExceptions && + maxParseExceptions == that.maxParseExceptions && + maxSavedParseExceptions == that.maxSavedParseExceptions && + Objects.equals(maxTotalRows, that.maxTotalRows) && Objects.equals(intermediatePersistPeriod, that.intermediatePersistPeriod) && Objects.equals(basePersistDirectory, that.basePersistDirectory) && Objects.equals(indexSpec, that.indexSpec) && Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory) && - Objects.equals(intermediateHandoffPeriod, that.intermediateHandoffPeriod) && - logParseExceptions == that.logParseExceptions && - maxParseExceptions == that.maxParseExceptions && - maxSavedParseExceptions == that.maxSavedParseExceptions; + Objects.equals(intermediateHandoffPeriod, that.intermediateHandoffPeriod); } @Override @@ -292,8 +292,9 @@ public int hashCode() { return Objects.hash( maxRowsInMemory, - maxRowsPerSegment, maxBytesInMemory, + maxRowsPerSegment, + maxTotalRows, intermediatePersistPeriod, basePersistDirectory, maxPendingPersists, @@ -303,6 +304,7 @@ public int hashCode() resetOffsetAutomatically, segmentWriteOutMediumFactory, intermediateHandoffPeriod, + skipSequenceNumberAvailabilityCheck, logParseExceptions, maxParseExceptions, maxSavedParseExceptions @@ -311,5 +313,4 @@ public int hashCode() @Override public abstract String toString(); - } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index 7dcbaa8001cb..b2c1f8eeb69c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -29,7 +29,7 @@ import java.util.Objects; /** - * class that encapsulates a map of partitionId -> sequenceNumber. + * class that encapsulates a partitionIdToSequenceNumberMap of partitionId -> sequenceNumber. * To be backward compatible with both Kafka and Kinesis datasource metadata when * deserializing json. Redundant constructor fields stream, topic and * partitionSequenceNumberMap and partitionOffsetMap are created. Only one of topic, stream @@ -40,40 +40,41 @@ * are used for proper Jackson serialization/deserialization when processing terminologies * used by Kafka and kinesis (i.e. topic vs. stream) * - * @param partition id type - * @param sequence number type + * @param partition id type + * @param sequence number type */ -public class SeekableStreamPartitions +public class SeekableStreamPartitions { public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER"; // stream/topic private final String stream; // partitionId -> sequence number - private final Map map; + private final Map partitionIdToSequenceNumberMap; @JsonCreator public SeekableStreamPartitions( @JsonProperty("stream") final String stream, // kept for backward compatibility @JsonProperty("topic") final String topic, - @JsonProperty("partitionSequenceNumberMap") final Map partitionSequenceNumberMap, + @JsonProperty("partitionSequenceNumberMap") + final Map partitionSequenceNumberMap, // kept for backward compatibility - @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap + @JsonProperty("partitionOffsetMap") final Map partitionOffsetMap ) { this.stream = stream == null ? topic : stream; - this.map = ImmutableMap.copyOf(partitionOffsetMap == null - ? partitionSequenceNumberMap - : partitionOffsetMap); + this.partitionIdToSequenceNumberMap = ImmutableMap.copyOf(partitionOffsetMap == null + ? partitionSequenceNumberMap + : partitionOffsetMap); Preconditions.checkArgument(this.stream != null); - Preconditions.checkArgument(map != null); + Preconditions.checkArgument(partitionIdToSequenceNumberMap != null); } // constructor for backward compatibility public SeekableStreamPartitions( @NotNull final String stream, - final Map partitionOffsetMap + final Map partitionOffsetMap ) { this(stream, null, partitionOffsetMap, null); @@ -86,9 +87,9 @@ public String getStream() } @JsonProperty - public Map getPartitionSequenceNumberMap() + public Map getPartitionSequenceNumberMap() { - return map; + return partitionIdToSequenceNumberMap; } @Override @@ -102,13 +103,13 @@ public boolean equals(Object o) } SeekableStreamPartitions that = (SeekableStreamPartitions) o; return Objects.equals(stream, that.stream) && - Objects.equals(map, that.map); + Objects.equals(partitionIdToSequenceNumberMap, that.partitionIdToSequenceNumberMap); } @Override public int hashCode() { - return Objects.hash(stream, map); + return Objects.hash(stream, partitionIdToSequenceNumberMap); } @Override @@ -116,7 +117,7 @@ public String toString() { return "SeekableStreamPartitions{" + "stream/topic='" + stream + '\'' + - ", partitionSequenceNumberMap/partitionOffsetMap=" + map + + ", partitionSequenceNumberMap/partitionOffsetMap=" + partitionIdToSequenceNumberMap + '}'; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java index 35ab93af2b0a..010fdfdbc00d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java @@ -23,25 +23,25 @@ import java.util.Objects; /** - * Represents a generic record with a PartitionType (partition id) and SequenceType (sequence number) and data + * Represents a generic record with a PartitionIdType (partition id) and SequenceOffsetType (sequence number) and data * from a Kafka/Kinesis stream * - * @param partition id - * @param sequence number + * @param partition id + * @param sequence number */ -public class OrderedPartitionableRecord +public class OrderedPartitionableRecord { public static final String END_OF_SHARD_MARKER = "EOS"; private final String stream; - private final PartitionType partitionId; - private final SequenceType sequenceNumber; + private final PartitionIdType partitionId; + private final SequenceOffsetType sequenceNumber; private final List data; public OrderedPartitionableRecord( String stream, - PartitionType partitionId, - SequenceType sequenceNumber, + PartitionIdType partitionId, + SequenceOffsetType sequenceNumber, List data ) { @@ -56,12 +56,12 @@ public String getStream() return stream; } - public PartitionType getPartitionId() + public PartitionIdType getPartitionId() { return partitionId; } - public SequenceType getSequenceNumber() + public SequenceOffsetType getSequenceNumber() { return sequenceNumber; } @@ -71,7 +71,7 @@ public List getData() return data; } - public StreamPartition getStreamPartition() + public StreamPartition getStreamPartition() { return StreamPartition.of(stream, partitionId); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java index 5196b52cf34f..f193488240d9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java @@ -20,8 +20,6 @@ package org.apache.druid.indexing.seekablestream.common; -import java.util.Objects; - /** * Represents a Kafka/Kinesis stream sequence number. Mainly used to do * comparison and indicate whether the sequence number is exclusive. @@ -30,20 +28,21 @@ * sequence of some Kinesis partition and should be discarded because some * previous task has already read this sequence number * - * @param type of sequence number + * @param type of sequence number */ -public abstract class OrderedSequenceNumber implements Comparable> +public abstract class OrderedSequenceNumber + implements Comparable> { - private final T sequenceNumber; + private final SequenceOffsetType sequenceNumber; private final boolean isExclusive; - protected OrderedSequenceNumber(T sequenceNumber, boolean isExclusive) + protected OrderedSequenceNumber(SequenceOffsetType sequenceNumber, boolean isExclusive) { this.sequenceNumber = sequenceNumber; this.isExclusive = isExclusive; } - public T get() + public SequenceOffsetType get() { return sequenceNumber; } @@ -52,24 +51,4 @@ public boolean isExclusive() { return isExclusive; } - - @Override - public int hashCode() - { - return Objects.hash(sequenceNumber, isExclusive); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - OrderedSequenceNumber that = (OrderedSequenceNumber) o; - return isExclusive == that.isExclusive && - Objects.equals(sequenceNumber, that.sequenceNumber); - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java index f063260322b6..77a2c073bc0e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java @@ -32,11 +32,11 @@ * The RecordSupplier interface is a wrapper for the incoming seekable data stream * (i.e. Kafka consumer, Kinesis streams) * - * @param Partition Number Type - * @param Sequence Number Type + * @param Partition Number Type + * @param Sequence Number Type */ @Beta -public interface RecordSupplier extends Closeable +public interface RecordSupplier extends Closeable { /** * assigns the given partitions to this RecordSupplier @@ -45,7 +45,7 @@ public interface RecordSupplier extends Closeable * * @param partitions parititions to assign */ - void assign(Set> partitions); + void assign(Set> partitions); /** * seek to specified sequence number within a specific partition @@ -53,28 +53,28 @@ public interface RecordSupplier extends Closeable * @param partition partition to seek * @param sequenceNumber sequence number to seek to */ - void seek(StreamPartition partition, SequenceType sequenceNumber); + void seek(StreamPartition partition, SequenceOffsetType sequenceNumber); /** * seek a set of partitions to the earliest record position available in the stream * * @param partitions partitions to seek */ - void seekToEarliest(Set> partitions); + void seekToEarliest(Set> partitions); /** * seek a set of partitions to the latest/newest record position available in the stream * * @param partitions partitions to seek */ - void seekToLatest(Set> partitions); + void seekToLatest(Set> partitions); /** * get the current assignment * * @return set of assignments */ - Collection> getAssignment(); + Collection> getAssignment(); /** * poll the record at the current seeked to sequence in stream @@ -84,7 +84,7 @@ public interface RecordSupplier extends Closeable * @return record */ @NotNull - List> poll(long timeout); + List> poll(long timeout); /** * get the latest sequence number in stream @@ -94,7 +94,7 @@ public interface RecordSupplier extends Closeable * @return latest sequence number */ @Nullable - SequenceType getLatestSequenceNumber(StreamPartition partition); + SequenceOffsetType getLatestSequenceNumber(StreamPartition partition); /** * get the earliest sequence number in stream @@ -104,7 +104,7 @@ public interface RecordSupplier extends Closeable * @return earliest sequence number */ @Nullable - SequenceType getEarliestSequenceNumber(StreamPartition partition); + SequenceOffsetType getEarliestSequenceNumber(StreamPartition partition); /** @@ -114,7 +114,7 @@ public interface RecordSupplier extends Closeable * * @return sequence number */ - SequenceType getPosition(StreamPartition partition); + SequenceOffsetType getPosition(StreamPartition partition); /** * returns the set of partitions under the given stream @@ -123,7 +123,7 @@ public interface RecordSupplier extends Closeable * * @return set of partitions */ - Set getPartitionIds(String stream); + Set getPartitionIds(String stream); /** * close the RecordSupplier diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java index 93e8e1d931ba..5666cdb59fb3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/StreamPartition.java @@ -24,14 +24,14 @@ * mostly used by {@link RecordSupplier} and * {@link org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor} * - * @param partition id type + * @param partition id type */ -public class StreamPartition +public class StreamPartition { private final String stream; - private final PartitionType partitionId; + private final PartitionIdType partitionId; - public StreamPartition(String stream, PartitionType partitionId) + public StreamPartition(String stream, PartitionIdType partitionId) { this.stream = stream; this.partitionId = partitionId; @@ -47,7 +47,7 @@ public String getStream() return stream; } - public PartitionType getPartitionId() + public PartitionIdType getPartitionId() { return partitionId; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index f6e96a728e13..21cc3b58f5ac 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -55,13 +55,13 @@ import org.apache.druid.indexing.overlord.supervisor.Supervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; -import org.apache.druid.indexing.seekablestream.SeekableStreamIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; -import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; @@ -89,7 +89,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Random; import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; @@ -100,7 +99,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; @@ -122,14 +120,14 @@ * stream sequences. *

    * - * @param the type of the partition id, for example, partitions in Kafka are int type while partitions in Kinesis are String type - * @param the type of the sequence number or offsets, for example, Kafka uses long offsets while Kinesis uses String sequence numbers + * @param the type of the partition id, for example, partitions in Kafka are int type while partitions in Kinesis are String type + * @param the type of the sequence number or offsets, for example, Kafka uses long offsets while Kinesis uses String sequence numbers */ -public abstract class SeekableStreamSupervisor +public abstract class SeekableStreamSupervisor implements Supervisor { public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED"; - private static final Random RANDOM = ThreadLocalRandom.current(); + private static final long MAX_RUN_FREQUENCY_MILLIS = 1000; private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120; private static final int MAX_INITIALIZATION_RETRIES = 20; @@ -156,22 +154,22 @@ private class TaskGroup // this task group has completed successfully, at which point this will be destroyed and a new task group will be // created with new starting sequences. This allows us to create replacement tasks for failed tasks that process the // same sequences, even if the values in [partitionGroups] has been changed. - final ImmutableMap startingSequences; + final ImmutableMap startingSequences; final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); final Optional minimumMessageTime; final Optional maximumMessageTime; - final Set exclusiveStartSequenceNumberPartitions; - final TreeMap> checkpointSequences = new TreeMap<>(); + final Set exclusiveStartSequenceNumberPartitions; + final TreeMap> checkpointSequences = new TreeMap<>(); final String baseSequenceName; DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action TaskGroup( int groupId, - ImmutableMap startingSequences, + ImmutableMap startingSequences, Optional minimumMessageTime, Optional maximumMessageTime, - Set exclusiveStartSequenceNumberPartitions + Set exclusiveStartSequenceNumberPartitions ) { this.groupId = groupId; @@ -185,7 +183,7 @@ private class TaskGroup this.baseSequenceName = generateSequenceName(startingSequences, minimumMessageTime, maximumMessageTime); } - int addNewCheckpoint(Map checkpoint) + int addNewCheckpoint(Map checkpoint) { checkpointSequences.put(checkpointSequences.lastKey() + 1, checkpoint); return checkpointSequences.lastKey(); @@ -202,7 +200,7 @@ private class TaskData { volatile TaskStatus status; volatile DateTime startTime; - volatile Map currentSequences = new HashMap<>(); + volatile Map currentSequences = new HashMap<>(); @Override public String toString() @@ -320,14 +318,14 @@ protected class CheckpointNotice implements Notice private final Integer nullableTaskGroupId; @Deprecated private final String baseSequenceName; - private final SeekableStreamDataSourceMetadata previousCheckpoint; - private final SeekableStreamDataSourceMetadata currentCheckpoint; + private final SeekableStreamDataSourceMetadata previousCheckpoint; + private final SeekableStreamDataSourceMetadata currentCheckpoint; public CheckpointNotice( @Nullable Integer nullableTaskGroupId, @Deprecated String baseSequenceName, - SeekableStreamDataSourceMetadata previousCheckpoint, - SeekableStreamDataSourceMetadata currentCheckpoint + SeekableStreamDataSourceMetadata previousCheckpoint, + SeekableStreamDataSourceMetadata currentCheckpoint ) { this.baseSequenceName = baseSequenceName; @@ -377,12 +375,12 @@ public void handle() throws ExecutionException, InterruptedException final TaskGroup taskGroup = activelyReadingTaskGroups.get(taskGroupId); if (isValidTaskGroup(taskGroupId, taskGroup)) { - final TreeMap> checkpoints = taskGroup.checkpointSequences; + final TreeMap> checkpoints = taskGroup.checkpointSequences; // check validity of previousCheckpoint int index = checkpoints.size(); for (int sequenceId : checkpoints.descendingKeySet()) { - Map checkpoint = checkpoints.get(sequenceId); + Map checkpoint = checkpoints.get(sequenceId); // We have already verified the stream of the current checkpoint is same with that in ioConfig. // See checkpoint(). if (checkpoint.equals(previousCheckpoint.getSeekableStreamPartitions() @@ -400,7 +398,7 @@ public void handle() throws ExecutionException, InterruptedException log.info("Already checkpointed with sequences [%s]", checkpoints.lastEntry().getValue()); return; } - final Map newCheckpoint = checkpointTaskGroup(taskGroup, false).get(); + final Map newCheckpoint = checkpointTaskGroup(taskGroup, false).get(); taskGroup.addNewCheckpoint(newCheckpoint); log.info("Handled checkpoint notice, new checkpoint is [%s] for taskGroup [%s]", newCheckpoint, taskGroupId); } @@ -432,13 +430,13 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) } - // Map<{group ID}, {actively reading task group}>; see documentation for TaskGroup class + // Map<{group RandomId}, {actively reading task group}>; see documentation for TaskGroup class private final ConcurrentHashMap activelyReadingTaskGroups = new ConcurrentHashMap<>(); // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [activelyReadingTaskGroups] to here so // we can monitor its status while we queue new tasks to read the next range of sequences. This is a list since we could // have multiple sets of tasks publishing at once if time-to-publish > taskDuration. - // Map<{group ID}, List<{pending completion task groups}>> + // Map<{group RandomId}, List<{pending completion task groups}>> private final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); // The starting sequence for a new partition in [partitionGroups] is initially set to getNotSetMarker(). When a new task group @@ -450,24 +448,24 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) // cause successive tasks to again grab their starting sequence from metadata store. This mechanism allows us to // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task // failures during publishing. - // Map<{group ID}, Map<{partition ID}, {startingOffset}>> - private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); + // Map<{group RandomId}, Map<{partition RandomId}, {startingOffset}>> + private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); protected final ObjectMapper sortingMapper; - protected final List partitionIds = new CopyOnWriteArrayList<>(); + protected final List partitionIds = new CopyOnWriteArrayList<>(); protected volatile DateTime sequenceLastUpdated; - private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); + private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); private final TaskStorage taskStorage; private final TaskMaster taskMaster; private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; - private final SeekableStreamIndexTaskClient taskClient; + private final SeekableStreamIndexTaskClient taskClient; private final SeekableStreamSupervisorSpec spec; private final String dataSource; private final SeekableStreamSupervisorIOConfig ioConfig; private final SeekableStreamSupervisorTuningConfig tuningConfig; - private final SeekableStreamTuningConfig taskTuningConfig; + private final SeekableStreamIndexTaskTuningConfig taskTuningConfig; private final String supervisorId; private final TaskInfoProvider taskInfoProvider; private final long futureTimeoutInSeconds; // how long to wait for async operations to complete @@ -486,7 +484,7 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) private long lastRunTime; private int initRetryCounter = 0; private volatile DateTime firstRunTime; - private volatile RecordSupplier recordSupplier; + private volatile RecordSupplier recordSupplier; private volatile boolean started = false; private volatile boolean stopped = false; private volatile boolean lifecycleStarted = false; @@ -497,7 +495,7 @@ public SeekableStreamSupervisor( final TaskStorage taskStorage, final TaskMaster taskMaster, final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, - final SeekableStreamIndexTaskClientFactory> taskClientFactory, + final SeekableStreamIndexTaskClientFactory> taskClientFactory, final ObjectMapper mapper, final SeekableStreamSupervisorSpec spec, final RowIngestionMetersFactory rowIngestionMetersFactory, @@ -515,7 +513,7 @@ public SeekableStreamSupervisor( this.dataSource = spec.getDataSchema().getDataSource(); this.ioConfig = spec.getIoConfig(); this.tuningConfig = spec.getTuningConfig(); - this.taskTuningConfig = this.tuningConfig.copyOf(); + this.taskTuningConfig = this.tuningConfig.convertToTaskTuningConfig(); this.supervisorId = supervisorId; this.exec = Execs.singleThreaded(supervisorId); this.scheduledExec = Execs.scheduledSingleThreaded(supervisorId + "-Scheduler-%d"); @@ -583,16 +581,6 @@ public Optional getTaskStatus(String id) ); } - protected static String getRandomId() - { - final StringBuilder suffix = new StringBuilder(8); - for (int i = 0; i < Integer.BYTES * 2; ++i) { - suffix.append((char) ('a' + ((RANDOM.nextInt() >>> (i * 4)) & 0x0F))); - } - return suffix.toString(); - } - - @Override public void start() { @@ -761,8 +749,6 @@ protected void tryInit() TimeUnit.MILLISECONDS ); - // different in kafka and kinesis as emitLag and checkpointing - // are not yet implemented in Kinesis scheduleReporting(reportingExec); started = true; @@ -797,24 +783,24 @@ public SupervisorReport getStatus() return generateReport(true); } - private SupervisorReport> generateReport( + private SupervisorReport> generateReport( boolean includeOffsets ) { int numPartitions = partitionGroups.values().stream().mapToInt(Map::size).sum(); - final SeekableStreamSupervisorReportPayload payload = createReportPayload( + final SeekableStreamSupervisorReportPayload payload = createReportPayload( numPartitions, includeOffsets ); - SupervisorReport> report = new SupervisorReport<>( + SupervisorReport> report = new SupervisorReport<>( dataSource, DateTimes.nowUtc(), payload ); - List> taskReports = new ArrayList<>(); + List> taskReports = new ArrayList<>(); try { for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) { @@ -822,7 +808,7 @@ private SupervisorReport currentOffsets = entry.getValue().currentSequences; + Map currentOffsets = entry.getValue().currentSequences; Long remainingSeconds = null; if (startTime != null) { long elapsedMillis = System.currentTimeMillis() - startTime.getMillis(); @@ -850,7 +836,7 @@ private SupervisorReport currentOffsets = entry.getValue().currentSequences; + Map currentOffsets = entry.getValue().currentSequences; Long remainingSeconds = null; if (taskGroup.completionTimeout != null) { remainingSeconds = Math.max(0, taskGroup.completionTimeout.getMillis() - System.currentTimeMillis()) @@ -968,13 +954,13 @@ private Map> getCurrentTotalStats() @VisibleForTesting - protected void addTaskGroupToActivelyReadingTaskGroup( + public void addTaskGroupToActivelyReadingTaskGroup( int taskGroupId, - ImmutableMap partitionOffsets, + ImmutableMap partitionOffsets, Optional minMsgTime, Optional maxMsgTime, Set tasks, - Set exclusiveStartingSequencePartitions + Set exclusiveStartingSequencePartitions ) { TaskGroup group = new TaskGroup( @@ -987,20 +973,20 @@ protected void addTaskGroupToActivelyReadingTaskGroup( group.tasks.putAll(tasks.stream().collect(Collectors.toMap(x -> x, x -> new TaskData()))); if (activelyReadingTaskGroups.putIfAbsent(taskGroupId, group) != null) { throw new ISE( - "trying to add taskGroup with ID [%s] to actively reading task groups, but group already exists.", + "trying to add taskGroup with RandomId [%s] to actively reading task groups, but group already exists.", taskGroupId ); } } @VisibleForTesting - protected void addTaskGroupToPendingCompletionTaskGroup( + public void addTaskGroupToPendingCompletionTaskGroup( int taskGroupId, - ImmutableMap partitionOffsets, + ImmutableMap partitionOffsets, Optional minMsgTime, Optional maxMsgTime, Set tasks, - Set exclusiveStartingSequencePartitions + Set exclusiveStartingSequencePartitions ) { TaskGroup group = new TaskGroup( @@ -1016,7 +1002,7 @@ protected void addTaskGroupToPendingCompletionTaskGroup( } @VisibleForTesting - protected void runInternal() + public void runInternal() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException { @@ -1079,7 +1065,7 @@ public void statusChanged(String taskId, TaskStatus status) } @VisibleForTesting - protected void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException + public void gracefulShutdownInternal() throws ExecutionException, InterruptedException, TimeoutException { for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) { for (Entry entry : taskGroup.tasks.entrySet()) { @@ -1095,7 +1081,7 @@ protected void gracefulShutdownInternal() throws ExecutionException, Interrupted } @VisibleForTesting - protected void resetInternal(DataSourceMetadata dataSourceMetadata) + public void resetInternal(DataSourceMetadata dataSourceMetadata) { if (dataSourceMetadata == null) { // Reset everything @@ -1118,7 +1104,7 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) } // Reset only the partitions in dataSourceMetadata if it has not been reset yet @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata resetMetadata = (SeekableStreamDataSourceMetadata) dataSourceMetadata; + final SeekableStreamDataSourceMetadata resetMetadata = (SeekableStreamDataSourceMetadata) dataSourceMetadata; if (resetMetadata.getSeekableStreamPartitions().getStream().equals(ioConfig.getStream())) { // metadata can be null @@ -1131,19 +1117,19 @@ protected void resetInternal(DataSourceMetadata dataSourceMetadata) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata currentMetadata = (SeekableStreamDataSourceMetadata) metadata; + final SeekableStreamDataSourceMetadata currentMetadata = (SeekableStreamDataSourceMetadata) metadata; // defend against consecutive reset requests from replicas // as well as the case where the metadata store do not have an entry for the reset partitions boolean doReset = false; - for (Entry resetPartitionOffset : resetMetadata.getSeekableStreamPartitions() - .getPartitionSequenceNumberMap() - .entrySet()) { - final SequenceType partitionOffsetInMetadataStore = currentMetadata == null - ? null - : currentMetadata.getSeekableStreamPartitions() - .getPartitionSequenceNumberMap() - .get(resetPartitionOffset.getKey()); + for (Entry resetPartitionOffset : resetMetadata.getSeekableStreamPartitions() + .getPartitionSequenceNumberMap() + .entrySet()) { + final SequenceOffsetType partitionOffsetInMetadataStore = currentMetadata == null + ? null + : currentMetadata.getSeekableStreamPartitions() + .getPartitionSequenceNumberMap() + .get(resetPartitionOffset.getKey()); final TaskGroup partitionTaskGroup = activelyReadingTaskGroups.get( getTaskGroupIdForPartition(resetPartitionOffset.getKey()) ); @@ -1215,9 +1201,9 @@ private void killTasksInGroup(TaskGroup taskGroup, String reasonFormat, Object.. } } - private void killTaskGroupForPartitions(Set partitions, String reasonFormat, Object... args) + private void killTaskGroupForPartitions(Set partitions, String reasonFormat, Object... args) { - for (PartitionType partition : partitions) { + for (PartitionIdType partition : partitions) { int taskGroupId = getTaskGroupIdForPartition(partition); killTasksInGroup(activelyReadingTaskGroups.get(taskGroupId), reasonFormat, args); partitionGroups.remove(taskGroupId); @@ -1252,7 +1238,7 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti taskCount++; @SuppressWarnings("unchecked") - final SeekableStreamIndexTask seekableStreamIndexTask = (SeekableStreamIndexTask) task; + final SeekableStreamIndexTask seekableStreamIndexTask = (SeekableStreamIndexTask) task; final String taskId = task.getId(); // Determine which task group this task belongs to based on one of the partitions handled by this task. If we @@ -1262,11 +1248,11 @@ private void discoverTasks() throws ExecutionException, InterruptedException, Ti // state, we will permit it to complete even if it doesn't match our current partition allocation to support // seamless schema migration. - Iterator it = seekableStreamIndexTask.getIOConfig() - .getStartPartitions() - .getPartitionSequenceNumberMap() - .keySet() - .iterator(); + Iterator it = seekableStreamIndexTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceNumberMap() + .keySet() + .iterator(); final Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null); if (taskGroupId != null) { @@ -1301,19 +1287,20 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) // update partitionGroups with the publishing task's sequences (if they are greater than what is // existing) so that the next tasks will start reading from where this task left off - Map publishingTaskEndOffsets = taskClient.getEndOffsets(taskId); + Map publishingTaskEndOffsets = taskClient.getEndOffsets( + taskId); - for (Entry entry : publishingTaskEndOffsets.entrySet()) { - PartitionType partition = entry.getKey(); - SequenceType sequence = entry.getValue(); - ConcurrentHashMap partitionOffsets = partitionGroups.get( + for (Entry entry : publishingTaskEndOffsets.entrySet()) { + PartitionIdType partition = entry.getKey(); + SequenceOffsetType sequence = entry.getValue(); + ConcurrentHashMap partitionOffsets = partitionGroups.get( getTaskGroupIdForPartition(partition) ); boolean succeeded; do { succeeded = true; - SequenceType previousOffset = partitionOffsets.putIfAbsent(partition, sequence); + SequenceOffsetType previousOffset = partitionOffsets.putIfAbsent(partition, sequence); if (previousOffset != null && (makeSequenceNumber(previousOffset).compareTo(makeSequenceNumber(sequence))) < 0) { succeeded = partitionOffsets.replace(partition, previousOffset, sequence); @@ -1321,10 +1308,10 @@ public Boolean apply(SeekableStreamIndexTaskRunner.Status status) } while (!succeeded); } } else { - for (PartitionType partition : seekableStreamIndexTask.getIOConfig() - .getStartPartitions() - .getPartitionSequenceNumberMap() - .keySet()) { + for (PartitionIdType partition : seekableStreamIndexTask.getIOConfig() + .getStartPartitions() + .getPartitionSequenceNumberMap() + .keySet()) { if (!taskGroupId.equals(getTaskGroupIdForPartition(partition))) { log.warn( "Stopping task [%s] which does not match the expected partition allocation", @@ -1437,12 +1424,12 @@ private void verifyAndMergeCheckpoints(final Collection taskGroupsToV private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) { final int groupId = taskGroup.groupId; - final List>>> taskSequences = new ArrayList<>(); - final List>>> futures = new ArrayList<>(); + final List>>> taskSequences = new ArrayList<>(); + final List>>> futures = new ArrayList<>(); final List taskIds = new ArrayList<>(); for (String taskId : taskGroup.taskIds()) { - final ListenableFuture>> checkpointsFuture = taskClient.getCheckpointsAsync( + final ListenableFuture>> checkpointsFuture = taskClient.getCheckpointsAsync( taskId, true ); @@ -1451,10 +1438,10 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) } try { - List>> futuresResult = + List>> futuresResult = Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); for (int i = 0; i < futuresResult.size(); i++) { - final TreeMap> checkpoints = futuresResult.get(i); + final TreeMap> checkpoints = futuresResult.get(i); final String taskId = taskIds.get(i); if (checkpoints == null) { try { @@ -1487,14 +1474,14 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) } @SuppressWarnings("unchecked") - final SeekableStreamDataSourceMetadata latestDataSourceMetadata = (SeekableStreamDataSourceMetadata) rawDataSourceMetadata; + final SeekableStreamDataSourceMetadata latestDataSourceMetadata = (SeekableStreamDataSourceMetadata) rawDataSourceMetadata; final boolean hasValidOffsetsFromDb = latestDataSourceMetadata != null && latestDataSourceMetadata.getSeekableStreamPartitions() != null && ioConfig.getStream().equals( latestDataSourceMetadata.getSeekableStreamPartitions().getStream() ); - final Map latestOffsetsFromDb; + final Map latestOffsetsFromDb; if (hasValidOffsetsFromDb) { latestOffsetsFromDb = latestDataSourceMetadata.getSeekableStreamPartitions().getPartitionSequenceNumberMap(); } else { @@ -1509,7 +1496,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) int taskIndex = 0; while (taskIndex < taskSequences.size()) { - TreeMap> taskCheckpoints = taskSequences.get(taskIndex).rhs; + TreeMap> taskCheckpoints = taskSequences.get(taskIndex).rhs; String taskId = taskSequences.get(taskIndex).lhs; if (earliestConsistentSequenceId.get() == -1) { // find the first replica task with earliest sequenceId consistent with datasource metadata in the metadata @@ -1517,8 +1504,8 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) if (taskCheckpoints.entrySet().stream().anyMatch( sequenceCheckpoint -> sequenceCheckpoint.getValue().entrySet().stream().allMatch( partitionOffset -> { - OrderedSequenceNumber sequence = makeSequenceNumber(partitionOffset.getValue()); - OrderedSequenceNumber latestOffset = makeSequenceNumber( + OrderedSequenceNumber sequence = makeSequenceNumber(partitionOffset.getValue()); + OrderedSequenceNumber latestOffset = makeSequenceNumber( latestOffsetsFromDb == null ? partitionOffset.getValue() : latestOffsetsFromDb.getOrDefault( partitionOffset @@ -1528,12 +1515,12 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) ) ); - return sequence.equals(latestOffset); + return sequence.compareTo(latestOffset) == 0; } ) && earliestConsistentSequenceId.compareAndSet(-1, sequenceCheckpoint.getKey())) || ( pendingCompletionTaskGroups.getOrDefault(groupId, new CopyOnWriteArrayList<>()).size() > 0 && earliestConsistentSequenceId.compareAndSet(-1, taskCheckpoints.firstKey()))) { - final SortedMap> latestCheckpoints = new TreeMap<>( + final SortedMap> latestCheckpoints = new TreeMap<>( taskCheckpoints.tailMap(earliestConsistentSequenceId.get()) ); log.info("Setting taskGroup sequences to [%s] for group [%d]", latestCheckpoints, groupId); @@ -1596,7 +1583,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup) private void addDiscoveredTaskToPendingCompletionTaskGroups( int groupId, String taskId, - Map startingPartitions + Map startingPartitions ) { final CopyOnWriteArrayList taskGroupList = pendingCompletionTaskGroups.computeIfAbsent( @@ -1657,7 +1644,7 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) } @SuppressWarnings("unchecked") - SeekableStreamIndexTask task = (SeekableStreamIndexTask) taskOptional + SeekableStreamIndexTask task = (SeekableStreamIndexTask) taskOptional .get(); String taskSequenceName = task.getIOConfig().getBaseSequenceName(); @@ -1679,14 +1666,14 @@ private boolean isTaskCurrent(int taskGroupId, String taskId) @VisibleForTesting protected String generateSequenceName( - Map startPartitions, + Map startPartitions, Optional minimumMessageTime, Optional maximumMessageTime ) { StringBuilder sb = new StringBuilder(); - for (Entry entry : startPartitions.entrySet()) { + for (Entry entry : startPartitions.entrySet()) { sb.append(StringUtils.format("+%s(%s)", entry.getKey().toString(), entry.getValue().toString())); } String partitionOffsetStr = sb.toString().substring(1); @@ -1717,7 +1704,7 @@ protected String generateSequenceName( private void updatePartitionDataFromStream() { - Set partitionIds; + Set partitionIds; try { synchronized (recordSupplierLock) { partitionIds = recordSupplier.getPartitionIds(ioConfig.getStream()); @@ -1736,7 +1723,7 @@ private void updatePartitionDataFromStream() log.debug("Found [%d] partitions for stream [%s]", partitionIds.size(), ioConfig.getStream()); - Set closedPartitions = getOffsetsFromMetadataStorage() + Set closedPartitions = getOffsetsFromMetadataStorage() .entrySet() .stream() .filter(x -> OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(x.getValue())) @@ -1744,7 +1731,7 @@ private void updatePartitionDataFromStream() .collect(Collectors.toSet()); boolean initialPartitionDiscovery = this.partitionIds.isEmpty(); - for (PartitionType partitionId : partitionIds) { + for (PartitionIdType partitionId : partitionIds) { if (closedPartitions.contains(partitionId)) { log.info("partition [%s] is closed and has no more data, skipping.", partitionId); continue; @@ -1757,7 +1744,7 @@ private void updatePartitionDataFromStream() int taskGroupId = getTaskGroupIdForPartition(partitionId); - ConcurrentHashMap partitionMap = partitionGroups.computeIfAbsent( + ConcurrentHashMap partitionMap = partitionGroups.computeIfAbsent( taskGroupId, k -> new ConcurrentHashMap<>() ); @@ -1842,7 +1829,7 @@ public Boolean apply(@Nullable DateTime startTime) private void checkTaskDuration() throws ExecutionException, InterruptedException, TimeoutException { - final List>> futures = new ArrayList<>(); + final List>> futures = new ArrayList<>(); final List futureGroupIds = new ArrayList<>(); for (Entry entry : activelyReadingTaskGroups.entrySet()) { @@ -1874,12 +1861,12 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } } - List> results = Futures.successfulAsList(futures) - .get(futureTimeoutInSeconds, TimeUnit.SECONDS); + List> results = Futures.successfulAsList(futures) + .get(futureTimeoutInSeconds, TimeUnit.SECONDS); for (int j = 0; j < results.size(); j++) { Integer groupId = futureGroupIds.get(j); TaskGroup group = activelyReadingTaskGroups.get(groupId); - Map endOffsets = results.get(j); + Map endOffsets = results.get(j); if (endOffsets != null) { // set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion @@ -1887,7 +1874,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException pendingCompletionTaskGroups.computeIfAbsent(groupId, k -> new CopyOnWriteArrayList<>()).add(group); // set endOffsets as the next startOffsets - for (Entry entry : endOffsets.entrySet()) { + for (Entry entry : endOffsets.entrySet()) { partitionGroups.get(groupId).put(entry.getKey(), entry.getValue()); } } else { @@ -1908,7 +1895,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } } - private ListenableFuture> checkpointTaskGroup( + private ListenableFuture> checkpointTaskGroup( final TaskGroup taskGroup, final boolean finalize ) @@ -1929,11 +1916,11 @@ private ListenableFuture> checkpointTaskGroup( // publishing failed and we need to re-ingest) return Futures.transform( stopTasksInGroup(taskGroup, "task[%s] succeeded in the taskGroup", task.status.getId()), - new Function>() + new Function>() { @Nullable @Override - public Map apply(@Nullable Object input) + public Map apply(@Nullable Object input) { return null; } @@ -1952,7 +1939,7 @@ public Map apply(@Nullable Object input) } // 2) Pause running tasks - final List>> pauseFutures = new ArrayList<>(); + final List>> pauseFutures = new ArrayList<>(); final List pauseTaskIds = ImmutableList.copyOf(taskGroup.taskIds()); for (final String taskId : pauseTaskIds) { pauseFutures.add(taskClient.pauseAsync(taskId)); @@ -1960,16 +1947,16 @@ public Map apply(@Nullable Object input) return Futures.transform( Futures.successfulAsList(pauseFutures), - new Function>, Map>() + new Function>, Map>() { @Nullable @Override - public Map apply(List> input) + public Map apply(List> input) { // 3) Build a map of the highest sequence read by any task in the group for each partition - final Map endOffsets = new HashMap<>(); + final Map endOffsets = new HashMap<>(); for (int i = 0; i < input.size(); i++) { - Map result = input.get(i); + Map result = input.get(i); if (result == null || result.isEmpty()) { // kill tasks that didn't return a value String taskId = pauseTaskIds.get(i); @@ -1977,7 +1964,7 @@ public Map apply(List sequence : result.entrySet()) { + for (Entry sequence : result.entrySet()) { if (!endOffsets.containsKey(sequence.getKey()) || makeSequenceNumber(endOffsets.get(sequence.getKey())).compareTo( makeSequenceNumber(sequence.getValue())) < 0) { @@ -2251,10 +2238,10 @@ private void createNewTasks() ) : Optional.absent()); - Map> startingOffsets = generateStartingSequencesForPartitionGroup( + Map> startingOffsets = generateStartingSequencesForPartitionGroup( groupId); - ImmutableMap simpleStartingOffsets = startingOffsets + ImmutableMap simpleStartingOffsets = startingOffsets .entrySet() .stream() .filter(x -> x.getValue().get() != null) @@ -2263,15 +2250,15 @@ private void createNewTasks() ImmutableMap::copyOf )); - Set exclusiveStartSequenceNumberPartitions = !useExclusiveStartingSequence - ? Collections.emptySet() - : startingOffsets - .entrySet() - .stream() - .filter(x -> x.getValue().get() != null - && x.getValue().isExclusive()) - .map(Entry::getKey) - .collect(Collectors.toSet()); + Set exclusiveStartSequenceNumberPartitions = !useExclusiveStartingSequence + ? Collections.emptySet() + : startingOffsets + .entrySet() + .stream() + .filter(x -> x.getValue().get() != null + && x.getValue().isExclusive()) + .map(Entry::getKey) + .collect(Collectors.toSet()); activelyReadingTaskGroups.put( groupId, @@ -2323,7 +2310,7 @@ private void addNotice(Notice notice) } @VisibleForTesting - protected void moveTaskGroupToPendingCompletion(int taskGroupId) + public void moveTaskGroupToPendingCompletion(int taskGroupId) { final TaskGroup taskGroup = activelyReadingTaskGroups.remove(taskGroupId); if (taskGroup != null) { @@ -2332,19 +2319,19 @@ protected void moveTaskGroupToPendingCompletion(int taskGroupId) } @VisibleForTesting - protected int getNoticesQueueSize() + public int getNoticesQueueSize() { return notices.size(); } - private ImmutableMap> generateStartingSequencesForPartitionGroup( + private ImmutableMap> generateStartingSequencesForPartitionGroup( int groupId ) { - ImmutableMap.Builder> builder = ImmutableMap.builder(); - for (Entry entry : partitionGroups.get(groupId).entrySet()) { - PartitionType partition = entry.getKey(); - SequenceType sequence = entry.getValue(); + ImmutableMap.Builder> builder = ImmutableMap.builder(); + for (Entry entry : partitionGroups.get(groupId).entrySet()) { + PartitionIdType partition = entry.getKey(); + SequenceOffsetType sequence = entry.getValue(); if (!getNotSetMarker().equals(sequence)) { // if we are given a startingOffset (set by a previous task group which is pending completion) then use it @@ -2354,7 +2341,7 @@ private ImmutableMap> generat } else { // if we don't have a startingOffset (first run or we had some previous failures and reset the sequences) then // get the sequence from metadata storage (if available) or Kafka/Kinesis (otherwise) - OrderedSequenceNumber offsetFromStorage = getOffsetFromStorageForPartition(partition); + OrderedSequenceNumber offsetFromStorage = getOffsetFromStorageForPartition(partition); if (offsetFromStorage != null) { builder.put(partition, offsetFromStorage); @@ -2368,10 +2355,10 @@ private ImmutableMap> generat * Queries the dataSource metadata table to see if there is a previous ending sequence for this partition. If it doesn't * find any data, it will retrieve the latest or earliest Kafka/Kinesis sequence depending on the useEarliestOffset config. */ - private OrderedSequenceNumber getOffsetFromStorageForPartition(PartitionType partition) + private OrderedSequenceNumber getOffsetFromStorageForPartition(PartitionIdType partition) { - final Map metadataOffsets = getOffsetsFromMetadataStorage(); - SequenceType sequence = metadataOffsets.get(partition); + final Map metadataOffsets = getOffsetsFromMetadataStorage(); + SequenceOffsetType sequence = metadataOffsets.get(partition); if (sequence != null) { log.debug("Getting sequence [%s] from metadata storage for partition [%s]", sequence, partition); if (!taskTuningConfig.isSkipSequenceNumberAvailabilityCheck()) { @@ -2415,13 +2402,13 @@ private OrderedSequenceNumber getOffsetFromStorageForPartition(Par } } - private Map getOffsetsFromMetadataStorage() + private Map getOffsetsFromMetadataStorage() { final DataSourceMetadata dataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource); if (dataSourceMetadata instanceof SeekableStreamDataSourceMetadata && checkSourceMetadataMatch(dataSourceMetadata)) { @SuppressWarnings("unchecked") - SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata) + SeekableStreamPartitions partitions = ((SeekableStreamDataSourceMetadata) dataSourceMetadata) .getSeekableStreamPartitions(); if (partitions != null) { if (!ioConfig.getStream().equals(partitions.getStream())) { @@ -2441,10 +2428,10 @@ && checkSourceMetadataMatch(dataSourceMetadata)) { } @Nullable - private SequenceType getOffsetFromStreamForPartition(PartitionType partition, boolean useEarliestOffset) + private SequenceOffsetType getOffsetFromStreamForPartition(PartitionIdType partition, boolean useEarliestOffset) { synchronized (recordSupplierLock) { - StreamPartition topicPartition = new StreamPartition<>(ioConfig.getStream(), partition); + StreamPartition topicPartition = new StreamPartition<>(ioConfig.getStream(), partition); if (!recordSupplier.getAssignment().contains(topicPartition)) { recordSupplier.assign(Collections.singleton(topicPartition)); } @@ -2459,17 +2446,17 @@ private void createTasksForGroup(int groupId, int replicas) throws JsonProcessingException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException { TaskGroup group = activelyReadingTaskGroups.get(groupId); - Map startPartitions = group.startingSequences; - Map endPartitions = new HashMap<>(); - for (PartitionType partition : startPartitions.keySet()) { + Map startPartitions = group.startingSequences; + Map endPartitions = new HashMap<>(); + for (PartitionIdType partition : startPartitions.keySet()) { endPartitions.put(partition, getEndOfPartitionMarker()); } - Set exclusiveStartSequenceNumberPartitions = activelyReadingTaskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; + Set exclusiveStartSequenceNumberPartitions = activelyReadingTaskGroups.get(groupId).exclusiveStartSequenceNumberPartitions; DateTime minimumMessageTime = activelyReadingTaskGroups.get(groupId).minimumMessageTime.orNull(); DateTime maximumMessageTime = activelyReadingTaskGroups.get(groupId).maximumMessageTime.orNull(); - SeekableStreamIOConfig newIoConfig = createIoConfig( + SeekableStreamIndexTaskIOConfig newIoConfig = createIoConfig( groupId, startPartitions, endPartitions, @@ -2481,7 +2468,7 @@ private void createTasksForGroup(int groupId, int replicas) ); - List> taskList = createIndexTasks( + List> taskList = createIndexTasks( replicas, group.baseSequenceName, sortingMapper, @@ -2507,7 +2494,7 @@ private void createTasksForGroup(int groupId, int replicas) } @VisibleForTesting - protected Runnable updateCurrentAndLatestOffsets() + public Runnable updateCurrentAndLatestOffsets() { return () -> { try { @@ -2532,7 +2519,7 @@ private void updateCurrentOffsets() throws InterruptedException, ExecutionExcept ).map( task -> Futures.transform( taskClient.getCurrentOffsetsAsync(task.getKey(), false), - (Function, Void>) (currentSequences) -> { + (Function, Void>) (currentSequences) -> { if (currentSequences != null && !currentSequences.isEmpty()) { task.getValue().currentSequences = currentSequences; @@ -2549,7 +2536,7 @@ private void updateCurrentOffsets() throws InterruptedException, ExecutionExcept protected void updateLatestOffsetsFromStream() { synchronized (recordSupplierLock) { - Set partitionIds = null; + Set partitionIds = null; try { partitionIds = recordSupplier.getPartitionIds(ioConfig.getStream()); } @@ -2558,7 +2545,7 @@ protected void updateLatestOffsetsFromStream() Throwables.propagate(e); } - Set> partitions = partitionIds + Set> partitions = partitionIds .stream() .map(e -> new StreamPartition<>(ioConfig.getStream(), e)) .collect(Collectors.toSet()); @@ -2572,11 +2559,11 @@ protected void updateLatestOffsetsFromStream() } protected abstract void updateLatestSequenceFromStream( - RecordSupplier recordSupplier, - Set> partitions + RecordSupplier recordSupplier, + Set> partitions ); - protected Map getHighestCurrentOffsets() + protected Map getHighestCurrentOffsets() { return activelyReadingTaskGroups .values() @@ -2590,7 +2577,7 @@ protected Map getHighestCurrentOffsets() )); } - private OrderedSequenceNumber makeSequenceNumber(SequenceType seq) + private OrderedSequenceNumber makeSequenceNumber(SequenceOffsetType seq) { return makeSequenceNumber(seq, false); } @@ -2649,8 +2636,8 @@ public void checkpoint( new CheckpointNotice( taskGroupId, baseSequenceName, - (SeekableStreamDataSourceMetadata) previousCheckPoint, - (SeekableStreamDataSourceMetadata) currentCheckPoint + (SeekableStreamDataSourceMetadata) previousCheckPoint, + (SeekableStreamDataSourceMetadata) currentCheckPoint ) ); } @@ -2660,14 +2647,14 @@ public void checkpoint( * * @return specific instance of Kafka/Kinesis IOConfig */ - protected abstract SeekableStreamIOConfig createIoConfig( + protected abstract SeekableStreamIndexTaskIOConfig createIoConfig( int groupId, - Map startPartitions, - Map endPartitions, + Map startPartitions, + Map endPartitions, String baseSequenceName, DateTime minimumMessageTime, DateTime maximumMessageTime, - Set exclusiveStartSequenceNumberPartitions, + Set exclusiveStartSequenceNumberPartitions, SeekableStreamSupervisorIOConfig ioConfig ); @@ -2679,13 +2666,13 @@ protected abstract SeekableStreamIOConfig createIoConfig( * * @throws JsonProcessingException */ - protected abstract List> createIndexTasks( + protected abstract List> createIndexTasks( int replicas, String baseSequenceName, ObjectMapper sortingMapper, - TreeMap> sequenceOffsets, - SeekableStreamIOConfig taskIoConfig, - SeekableStreamTuningConfig taskTuningConfig, + TreeMap> sequenceOffsets, + SeekableStreamIndexTaskIOConfig taskIoConfig, + SeekableStreamIndexTaskTuningConfig taskTuningConfig, RowIngestionMetersFactory rowIngestionMetersFactory ) throws JsonProcessingException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException; @@ -2697,7 +2684,7 @@ protected abstract List> cr * * @return taskgroup id */ - protected abstract int getTaskGroupIdForPartition(PartitionType partition); + protected abstract int getTaskGroupIdForPartition(PartitionIdType partition); /** * checks if the passed in DataSourceMetadata is a specific instance @@ -2727,19 +2714,19 @@ protected abstract List> cr * * @return specific instance of datasource metadata */ - protected abstract SeekableStreamDataSourceMetadata createDataSourceMetaData( + protected abstract SeekableStreamDataSourceMetadata createDataSourceMetaData( String - stream, Map map + stream, Map map ); /** - * wraps the passed in SequenceType sequence number into a {@link OrderedSequenceNumber} object + * wraps the passed in SequenceOffsetType sequence number into a {@link OrderedSequenceNumber} object * to facilitate comparison and accomodate exclusive starting sequennce in kinesis * * @return specific instance of [Kafka/Kinesis]OrderedSequenceNumber */ - protected abstract OrderedSequenceNumber makeSequenceNumber( - SequenceType seq, + protected abstract OrderedSequenceNumber makeSequenceNumber( + SequenceOffsetType seq, boolean isExclusive ); @@ -2755,14 +2742,14 @@ protected abstract OrderedSequenceNumber makeSequenceNumber( * * @return map of partition id -> lag */ - protected abstract Map getLagPerPartition(Map currentOffsets); + protected abstract Map getLagPerPartition(Map currentOffsets); /** * returns an instance of a specific Kinesis/Kafka recordSupplier * * @return specific instance of Kafka/Kinesis RecordSupplier */ - protected abstract RecordSupplier setupRecordSupplier() + protected abstract RecordSupplier setupRecordSupplier() throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException; /** @@ -2770,7 +2757,7 @@ protected abstract RecordSupplier setupRecordSuppli * * @return specific instance of Kafka/Kinesis Supervisor Report Payload */ - protected abstract SeekableStreamSupervisorReportPayload createReportPayload( + protected abstract SeekableStreamSupervisorReportPayload createReportPayload( int numPartitions, boolean includeOffsets ); @@ -2781,12 +2768,12 @@ protected abstract SeekableStreamSupervisorReportPayload +public abstract class SeekableStreamSupervisorReportPayload { private final String dataSource; private final String id; @@ -39,8 +39,8 @@ public abstract class SeekableStreamSupervisorReportPayload activeTasks; private final List publishingTasks; - private final Map latestOffsets; - private final Map minimumLag; + private final Map latestOffsets; + private final Map minimumLag; private final Long aggregateLag; private final DateTime offsetsLastUpdated; private final boolean suspended; @@ -51,8 +51,8 @@ public SeekableStreamSupervisorReportPayload( int partitions, int replicas, long durationSeconds, - @Nullable Map latestOffsets, - @Nullable Map minimumLag, + @Nullable Map latestOffsets, + @Nullable Map minimumLag, @Nullable Long aggregateLag, @Nullable DateTime offsetsLastUpdated, boolean suspended @@ -131,13 +131,13 @@ public List getPublishingTasks() } @JsonProperty - public Map getLatestOffsets() + public Map getLatestOffsets() { return latestOffsets; } @JsonProperty - public Map getMinimumLag() + public Map getMinimumLag() { return minimumLag; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java index 753d03f3448d..dfc8b6a4498e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java @@ -20,7 +20,7 @@ package org.apache.druid.indexing.seekablestream.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.seekablestream.SeekableStreamTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.joda.time.Duration; import org.joda.time.Period; @@ -55,5 +55,5 @@ static Duration defaultDuration(final Period period, final String theDefault) @Override String toString(); - SeekableStreamTuningConfig copyOf(); + SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java index 7bd23442b035..a7864c7672de 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java @@ -26,24 +26,24 @@ import javax.annotation.Nullable; import java.util.Map; -public class TaskReportData +public class TaskReportData { private final String id; - private final Map startingOffsets; + private final Map startingOffsets; private final DateTime startTime; private final Long remainingSeconds; private final TaskType type; - private Map currentOffsets; - private final Map lag; + private Map currentOffsets; + private final Map lag; public TaskReportData( String id, - @Nullable Map startingOffsets, - @Nullable Map currentOffsets, + @Nullable Map startingOffsets, + @Nullable Map currentOffsets, @Nullable DateTime startTime, Long remainingSeconds, TaskType type, - @Nullable Map lag + @Nullable Map lag ) { this.id = id; @@ -63,14 +63,14 @@ public String getId() @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getStartingOffsets() + public Map getStartingOffsets() { return startingOffsets; } @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getCurrentOffsets() + public Map getCurrentOffsets() { return currentOffsets; } @@ -95,12 +95,12 @@ public TaskType getType() @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public Map getLag() + public Map getLag() { return lag; } - public void setCurrentSequenceNumbers(Map currentOffsets) + public void setCurrentSequenceNumbers(Map currentOffsets) { this.currentOffsets = currentOffsets; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/utils/RandomId.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/utils/RandomId.java new file mode 100644 index 000000000000..631a4660f4b9 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/utils/RandomId.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.seekablestream.utils; + +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; + +public class RandomId +{ + private static final Random RANDOM = ThreadLocalRandom.current(); + + public static String getRandomId() + { + final StringBuilder suffix = new StringBuilder(8); + for (int i = 0; i < Integer.BYTES * 2; ++i) { + suffix.append((char) ('a' + ((RANDOM.nextInt() >>> (i * 4)) & 0x0F))); + } + return suffix.toString(); + } +} From 26adc38285015cf64d63478bc927131eda4ca676 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Fri, 14 Dec 2018 15:50:19 -0800 Subject: [PATCH 68/87] comments*2 --- ...ementalPublishingKafkaIndexTaskRunner.java | 42 ++++++++++++++++-- .../kafka/LegacyKafkaIndexTaskRunner.java | 13 ++++++ .../indexing/kafka/KafkaIndexTaskTest.java | 8 ++-- .../kinesis/KinesisIndexTaskRunner.java | 44 +++++++++++++++++++ .../SeekableStreamIndexTaskRunner.java | 44 +++---------------- 5 files changed, 107 insertions(+), 44 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index f70ac02cdd19..01b2d40c9b38 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -49,6 +49,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -59,7 +60,6 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamIndexTaskRunner { private static final EmittingLogger log = new EmittingLogger(IncrementalPublishingKafkaIndexTaskRunner.class); - private final KafkaIndexTaskTuningConfig tuningConfig; private final KafkaIndexTask task; public IncrementalPublishingKafkaIndexTaskRunner( @@ -80,7 +80,6 @@ public IncrementalPublishingKafkaIndexTaskRunner( rowIngestionMetersFactory ); this.task = task; - this.tuningConfig = task.getTuningConfig(); } @Override @@ -133,7 +132,7 @@ private void possiblyResetOffsetsOrWait( { final Map resetPartitions = new HashMap<>(); boolean doReset = false; - if (tuningConfig.isResetOffsetAutomatically()) { + if (task.getTuningConfig().isResetOffsetAutomatically()) { for (Map.Entry outOfRangePartition : outOfRangePartitions.entrySet()) { final TopicPartition topicPartition = outOfRangePartition.getKey(); final long nextOffset = outOfRangePartition.getValue(); @@ -203,6 +202,43 @@ protected Type getRunnerType() return Type.KAFKA; } + @Override + protected void possiblyResetDataSourceMetadata( + TaskToolbox toolbox, + RecordSupplier recordSupplier, + Set> assignment, + Map currOffsets + ) + { + for (final StreamPartition streamPartition : assignment) { + Long sequence = currOffsets.get(streamPartition.getPartitionId()); + Long earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); + if (earliestSequenceNumber == null + || createSequenceNumber(earliestSequenceNumber).compareTo(createSequenceNumber(sequence)) > 0) { + if (task.getTuningConfig().isResetOffsetAutomatically()) { + log.info("Attempting to reset sequences automatically for all partitions"); + try { + sendResetRequestAndWait( + assignment.stream() + .collect(Collectors.toMap(x -> x, x -> currOffsets.get(x.getPartitionId()))), + toolbox + ); + } + catch (IOException e) { + throw new ISE(e, "Exception while attempting to automatically reset sequences"); + } + } else { + throw new ISE( + "Starting sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) and resetOffsetAutomatically is not enabled", + sequence, + streamPartition.getPartitionId(), + earliestSequenceNumber + ); + } + } + } + } + @Nullable @Override protected TreeMap> getCheckPointsFromContext( diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 7f104eea3258..0593473c5105 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -52,6 +52,7 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -696,6 +697,18 @@ private boolean possiblyPause() throws InterruptedException return false; } + @Override + protected void possiblyResetDataSourceMetadata( + TaskToolbox toolbox, + RecordSupplier recordSupplier, + Set> assignment, + Map currOffsets + ) + { + throw new UnsupportedOperationException(); + } + + @Override protected SeekableStreamPartitions deserializeSeekableStreamPartitionsFromMetadata( ObjectMapper mapper, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 91b54a13dba9..2ad94c54239c 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -2146,7 +2146,7 @@ public boolean apply(TaskLock lock) private KafkaIndexTask createTask( final String taskId, final KafkaIndexTaskIOConfig ioConfig - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { return createTask(taskId, DATA_SCHEMA, ioConfig); } @@ -2155,7 +2155,7 @@ private KafkaIndexTask createTask( final String taskId, final KafkaIndexTaskIOConfig ioConfig, final Map context - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { return createTask(taskId, DATA_SCHEMA, ioConfig, context); } @@ -2164,7 +2164,7 @@ private KafkaIndexTask createTask( final String taskId, final DataSchema dataSchema, final KafkaIndexTaskIOConfig ioConfig - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( 1000, @@ -2213,7 +2213,7 @@ private KafkaIndexTask createTask( final DataSchema dataSchema, final KafkaIndexTaskIOConfig ioConfig, final Map context - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( 1000, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index 31fa15fd5004..c7aabb8b60d5 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -31,6 +31,8 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; @@ -42,13 +44,17 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeMap; +import java.util.stream.Collectors; public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner { private static final EmittingLogger log = new EmittingLogger(KinesisIndexTaskRunner.class); private static final long POLL_TIMEOUT = 100; + private final KinesisIndexTask task; + KinesisIndexTaskRunner( KinesisIndexTask task, InputRowParser parser, @@ -66,6 +72,7 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner recordSupplier, + Set> assignment, + Map currOffsets + ) + { + for (final StreamPartition streamPartition : assignment) { + String sequence = currOffsets.get(streamPartition.getPartitionId()); + String earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); + if (earliestSequenceNumber == null + || createSequenceNumber(earliestSequenceNumber).compareTo(createSequenceNumber(sequence)) > 0) { + if (task.getTuningConfig().isResetOffsetAutomatically()) { + log.info("Attempting to reset sequences automatically for all partitions"); + try { + sendResetRequestAndWait( + assignment.stream() + .collect(Collectors.toMap(x -> x, x -> currOffsets.get(x.getPartitionId()))), + toolbox + ); + } + catch (IOException e) { + throw new ISE(e, "Exception while attempting to automatically reset sequences"); + } + } else { + throw new ISE( + "Starting sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) and resetOffsetAutomatically is not enabled", + sequence, + streamPartition.getPartitionId(), + earliestSequenceNumber + ); + } + } + } + } + @Nullable @Override protected TreeMap> getCheckPointsFromContext( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 184e5170fc77..a6fc4c1b6096 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -450,7 +450,7 @@ public void run() maybePersistAndPublishSequences(committerSupplier); Set> assignment = assignPartitions(recordSupplier); - possiblyResetDataSourceMetadata(recordSupplier, assignment); + possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment, currOffsets); seekToStartingSequence(recordSupplier, assignment); ingestionState = IngestionState.BUILD_SEGMENTS; @@ -470,7 +470,7 @@ public void run() // The partition assignments may have changed while paused by a call to setEndOffsets() so reassign // partitions upon resuming. This is safe even if the end sequences have not been modified. assignment = assignPartitions(recordSupplier); - possiblyResetDataSourceMetadata(recordSupplier, assignment); + possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment, currOffsets); seekToStartingSequence(recordSupplier, assignment); if (assignment.isEmpty()) { @@ -1126,42 +1126,12 @@ private boolean possiblyPause() throws InterruptedException return false; } - private void possiblyResetDataSourceMetadata( + protected abstract void possiblyResetDataSourceMetadata( + TaskToolbox toolbox, RecordSupplier recordSupplier, - Set> assignment - ) - { - for (final StreamPartition streamPartition : assignment) { - SequenceOffsetType sequence = currOffsets.get(streamPartition.getPartitionId()); - if (!tuningConfig.isSkipSequenceNumberAvailabilityCheck()) { - SequenceOffsetType earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); - if (earliestSequenceNumber == null - || createSequenceNumber(earliestSequenceNumber).compareTo(createSequenceNumber(sequence)) > 0) { - if (tuningConfig.isResetOffsetAutomatically()) { - log.info("Attempting to reset sequences automatically for all partitions"); - try { - sendResetRequestAndWait( - assignment.stream() - .collect(Collectors.toMap(x -> x, x -> currOffsets.get(x.getPartitionId()))), - toolbox - ); - } - catch (IOException e) { - throw new ISE(e, "Exception while attempting to automatically reset sequences"); - } - } else { - throw new ISE( - "Starting sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) and resetOffsetAutomatically is not enabled", - sequence, - streamPartition.getPartitionId(), - earliestSequenceNumber - ); - } - } - } - } - } - + Set> assignment, + Map currOffsets + ); private void handleParseException(ParseException pe, OrderedPartitionableRecord record) { From f6049a9e3c2b45c32077d049b505b761c7ee420e Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Fri, 14 Dec 2018 16:18:07 -0800 Subject: [PATCH 69/87] revert string replace changes --- .../druid/indexing/common/IndexTaskClient.java | 2 +- .../indexing/common/TaskInfoProvider.java | 4 ++-- .../druid/indexing/common/task/Task.java | 12 ++++++------ .../indexing/common/task/TaskResource.java | 6 +++--- .../parallel/ParallelIndexTaskRunner.java | 6 +++--- .../overlord/HeapMemoryTaskStorage.java | 4 ++-- .../indexing/overlord/MetadataTaskStorage.java | 2 +- .../overlord/SingleTaskBackgroundRunner.java | 2 +- .../druid/indexing/overlord/TaskQueue.java | 2 +- .../druid/indexing/overlord/TaskRunner.java | 2 +- .../druid/indexing/overlord/TaskStorage.java | 18 +++++++++--------- .../overlord/TaskStorageQueryAdapter.java | 2 +- .../overlord/http/TaskPayloadResponse.java | 2 +- .../overlord/http/TaskStatusResponse.java | 2 +- 14 files changed, 33 insertions(+), 33 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java index 592059b7ecda..1b65ad490c2f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/IndexTaskClient.java @@ -363,7 +363,7 @@ private FullResponseHolder submitRequest( // Since workers are free to move tasks around to different ports, there is a chance that a task may have been // moved but our view of its location has not been updated yet from ZK. To detect this case, we send a header // identifying our expected recipient in the request; if this doesn't correspond to the worker we messaged, the - // worker will return an HTTP 404 with its RandomId in the response header. If we get a mismatching task RandomId, then + // worker will return an HTTP 404 with its ID in the response header. If we get a mismatching task ID, then // we will wait for a short period then retry the request indefinitely, expecting the task's location to // eventually be updated. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskInfoProvider.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskInfoProvider.java index 7c5ef4f90e2f..748cbd560dec 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskInfoProvider.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskInfoProvider.java @@ -26,14 +26,14 @@ public interface TaskInfoProvider { /** - * @param id the task RandomId + * @param id the task ID * * @return a TaskLocation associated with the task or TaskLocation.unknown() if no associated entry could be found */ TaskLocation getTaskLocation(String id); /** - * @param id the task RandomId + * @param id the task ID * * @return an Optional.of() with the current status of the task or Optional.absent() if the task could not be found */ diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 33ce22370636..3ed45482cc46 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -35,7 +35,7 @@ * Represents a task that can run on a worker. The general contracts surrounding Tasks are: *

      *
    • Tasks must operate on a single datasource.
    • - *
    • Tasks should be immutable, since the task RandomId is used as a proxy for the task in many locations.
    • + *
    • Tasks should be immutable, since the task ID is used as a proxy for the task in many locations.
    • *
    • Task IDs must be unique. This can be done by naming them using UUIDs or the current timestamp.
    • *
    • Tasks are each part of a "task group", which is a set of tasks that can share interval locks. These are * useful for producing sharded segments.
    • @@ -64,17 +64,17 @@ public interface Task { /** - * Returns RandomId of this task. Must be unique across all tasks ever created. + * Returns ID of this task. Must be unique across all tasks ever created. * - * @return task RandomId + * @return task ID */ String getId(); /** - * Returns group RandomId of this task. Tasks with the same group RandomId can share locks. If tasks do not need to share locks, - * a common convention is to set group RandomId equal to task RandomId. + * Returns group ID of this task. Tasks with the same group ID can share locks. If tasks do not need to share locks, + * a common convention is to set group ID equal to task ID. * - * @return task group RandomId + * @return task group ID */ String getGroupId(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskResource.java index c771de667a3f..e74c4d038300 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskResource.java @@ -40,9 +40,9 @@ public TaskResource( } /** - * Returns availability group RandomId of this task. Tasks the same availability group cannot be assigned to the same - * worker. If tasks do not have this restriction, a common convention is to set the availability group RandomId to the - * task RandomId. + * Returns availability group ID of this task. Tasks the same availability group cannot be assigned to the same + * worker. If tasks do not have this restriction, a common convention is to set the availability group ID to the + * task ID. * * @return task availability group */ diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java index 08e0c0054e0f..085d2f119b2a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java @@ -79,19 +79,19 @@ public interface ParallelIndexTaskRunner List> getCompleteSubTaskSpecs(); /** - * Returns the {@link SubTaskSpec} of the given RandomId or null if it's not found. + * Returns the {@link SubTaskSpec} of the given ID or null if it's not found. */ @Nullable SubTaskSpec getSubTaskSpec(String subTaskSpecId); /** - * Returns {@link SubTaskSpecStatus} of the given RandomId or null if it's not found. + * Returns {@link SubTaskSpecStatus} of the given ID or null if it's not found. */ @Nullable SubTaskSpecStatus getSubTaskState(String subTaskSpecId); /** - * Returns {@link TaskHistory} of the given RandomId or null if it's not found. + * Returns {@link TaskHistory} of the given ID or null if it's not found. */ @Nullable TaskHistory getCompleteSubTaskSpecAttemptHistory(String subTaskSpecId); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index bdf3c72ac3f1..a800a35f2d9f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -77,7 +77,7 @@ public void insert(Task task, TaskStatus status) throws EntryExistsException Preconditions.checkNotNull(status, "status"); Preconditions.checkArgument( task.getId().equals(status.getId()), - "Task/Status RandomId mismatch[%s/%s]", + "Task/Status ID mismatch[%s/%s]", task.getId(), status.getId() ); @@ -121,7 +121,7 @@ public void setStatus(TaskStatus status) Preconditions.checkNotNull(status, "status"); final String taskid = status.getId(); - Preconditions.checkState(tasks.containsKey(taskid), "Task RandomId must already be present: %s", taskid); + Preconditions.checkState(tasks.containsKey(taskid), "Task ID must already be present: %s", taskid); Preconditions.checkState(tasks.get(taskid).getStatus().isRunnable(), "Task status must be runnable: %s", taskid); log.info("Updating task %s to status: %s", taskid, status); tasks.put(taskid, tasks.get(taskid).withStatus(status)); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 5360cdf31d2b..8690c68ded0d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -125,7 +125,7 @@ public void insert(final Task task, final TaskStatus status) throws EntryExistsE Preconditions.checkNotNull(status, "status"); Preconditions.checkArgument( task.getId().equals(status.getId()), - "Task/Status RandomId mismatch[%s/%s]", + "Task/Status ID mismatch[%s/%s]", task.getId(), status.getId() ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunner.java index 76f25e7ae4dc..a9b631713d39 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunner.java @@ -275,7 +275,7 @@ public ListenableFuture run(final Task task) * There might be a race between {@link #run(Task)} and this method, but it shouldn't happen in real applications * because this method is called only in unit tests. See TaskLifecycleTest. * - * @param taskid task RandomId to clean up resources for + * @param taskid task ID to clean up resources for */ @Override public void shutdown(final String taskid, String reason) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java index 5125854d0402..9a9c41dfbb83 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java @@ -387,7 +387,7 @@ public void shutdown(final String taskId, String reasonFormat, Object... args) * @param taskStatus new task status * * @throws NullPointerException if task or status is null - * @throws IllegalArgumentException if the task RandomId does not match the status RandomId + * @throws IllegalArgumentException if the task ID does not match the status ID * @throws IllegalStateException if this queue is currently shut down */ private void notifyStatus(final Task task, final TaskStatus taskStatus, String reasonFormat, Object... args) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java index caa3883ee09e..81596596daa6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskRunner.java @@ -81,7 +81,7 @@ public interface TaskRunner * Inform the task runner it can clean up any resources associated with a task. This implies shutdown of any * currently-running tasks. * - * @param taskid task RandomId to clean up resources for + * @param taskid task ID to clean up resources for * @param reason reason to kill this task */ void shutdown(String taskid, String reason); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java index 64866dc9269c..d8a4806bb57c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java @@ -38,7 +38,7 @@ public interface TaskStorage * * @param task task to add * @param status task status - * @throws EntryExistsException if the task RandomId already exists + * @throws EntryExistsException if the task ID already exists */ void insert(Task task, TaskStatus status) throws EntryExistsException; @@ -52,7 +52,7 @@ public interface TaskStorage /** * Persists lock state in the storage facility. - * @param taskid task RandomId + * @param taskid task ID * @param taskLock lock state */ void addLock(String taskid, TaskLock taskLock); @@ -70,7 +70,7 @@ public interface TaskStorage * Removes lock state from the storage facility. It is harmless to keep old locks in the storage facility, but * this method can help reclaim wasted space. * - * @param taskid task RandomId + * @param taskid task ID * @param taskLock lock state */ void removeLock(String taskid, TaskLock taskLock); @@ -83,21 +83,21 @@ public interface TaskStorage void removeTasksOlderThan(long timestamp); /** - * Returns task as stored in the storage facility. If the task RandomId does not exist, this will return an + * Returns task as stored in the storage facility. If the task ID does not exist, this will return an * absentee Optional. * * NOTE: This method really feels like it should be combined with {@link #getStatus}. Expect that in the future. * - * @param taskid task RandomId + * @param taskid task ID * @return optional task */ Optional getTask(String taskid); /** - * Returns task status as stored in the storage facility. If the task RandomId does not exist, this will return + * Returns task status as stored in the storage facility. If the task ID does not exist, this will return * an absentee Optional. * - * @param taskid task RandomId + * @param taskid task ID * @return task status */ Optional getStatus(String taskid); @@ -119,7 +119,7 @@ public interface TaskStorage /** * Returns all actions taken by a task. * - * @param taskid task RandomId + * @param taskid task ID * @return list of task actions */ @Deprecated @@ -162,7 +162,7 @@ List> getRecentlyCreatedAlreadyFinishedTaskInfo( /** * Returns a list of locks for a particular task. * - * @param taskid task RandomId + * @param taskid task ID * @return list of TaskLocks for the given task */ List getLocks(String taskid); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java index 61fcb8d89d14..6a12b40bc11b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -90,7 +90,7 @@ public TaskInfo getTaskInfo(String taskId) * with the result set perhaps growing boundlessly and we do not do anything to protect against that. Use at your * own risk and know that at some point, we might adjust this to actually enforce some sort of limits. * - * @param taskid task RandomId + * @param taskid task ID * @return set of segments created by the specified task */ @Deprecated diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskPayloadResponse.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskPayloadResponse.java index ab5e9c2536da..b73f9d9f2e5e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskPayloadResponse.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskPayloadResponse.java @@ -27,7 +27,7 @@ public class TaskPayloadResponse { - private final String task; // Task RandomId, named "task" in the JSONification of this class. + private final String task; // Task ID, named "task" in the JSONification of this class. private final Task payload; @JsonCreator diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskStatusResponse.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskStatusResponse.java index dfbcc226fd8c..82e3335c7e26 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskStatusResponse.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/TaskStatusResponse.java @@ -28,7 +28,7 @@ public class TaskStatusResponse { - private final String task; // Task RandomId, named "task" in the JSONification of this class. + private final String task; // Task ID, named "task" in the JSONification of this class. @Nullable private final TaskStatusPlus status; From e24553e395462e826c161aafacb93bc1d1d38f09 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 17 Dec 2018 11:28:54 -0800 Subject: [PATCH 70/87] comments --- .../kafka/KafkaIndexTaskTuningConfig.java | 6 +- .../kafka/supervisor/KafkaSupervisor.java | 7 - .../kafka/KafkaIndexTaskTuningConfigTest.java | 2 +- .../kafka/supervisor/KafkaSupervisorTest.java | 8 +- .../KafkaSupervisorTuningConfigTest.java | 2 +- .../kinesis-indexing-service/pom.xml | 21 +-- .../kinesis/KinesisIndexTaskClient.java | 2 - .../kinesis/KinesisIndexTaskRunner.java | 46 ++--- .../kinesis/KinesisIndexTaskTuningConfig.java | 8 +- .../kinesis/KinesisIndexingServiceModule.java | 8 + .../kinesis/KinesisRecordSupplier.java | 2 +- .../kinesis/KinesisSequenceNumber.java | 16 -- .../KinesisSupervisorReportPayload.java | 2 +- .../kinesis/KinesisIndexTaskTest.java | 171 +++++++++--------- .../KinesisIndexTaskTuningConfigTest.java | 2 +- .../kinesis/KinesisRecordSupplierTest.java | 14 +- .../supervisor/KinesisSupervisorTest.java | 43 ++--- .../KinesisSupervisorTuningConfigTest.java | 2 +- .../SeekableStreamIndexTask.java | 5 +- .../SeekableStreamIndexTaskRunner.java | 22 ++- .../SeekableStreamIndexTaskTuningConfig.java | 2 +- .../common/OrderedPartitionableRecord.java | 10 +- .../supervisor/SeekableStreamSupervisor.java | 35 ++-- 23 files changed, 205 insertions(+), 231 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java index 8b6747a1a1f1..9ceb21ed0df2 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java @@ -60,7 +60,7 @@ public KafkaIndexTaskTuningConfig( maxTotalRows, intermediatePersistPeriod, basePersistDirectory, - 0, + maxPendingPersists, indexSpec, true, reportParseExceptions, @@ -85,7 +85,7 @@ public KafkaIndexTaskTuningConfig withBasePersistDirectory(File dir) getMaxTotalRows(), getIntermediatePersistPeriod(), dir, - 0, + getMaxPendingPersists(), getIndexSpec(), true, isReportParseExceptions(), @@ -109,7 +109,7 @@ public String toString() ", maxBytesInMemory=" + getMaxBytesInMemory() + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + - ", maxPendingPersists=" + 0 + + ", maxPendingPersists=" + getMaxPendingPersists() + ", indexSpec=" + getIndexSpec() + ", reportParseExceptions=" + isReportParseExceptions() + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index d4a465c80e04..e1298d1aa8b6 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -380,11 +380,4 @@ public KafkaSupervisorIOConfig getIoConfig() { return spec.getIoConfig(); } - - @Override - @VisibleForTesting - protected void tryInit() - { - super.tryInit(); - } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java index 8ee3db67450e..8a6ee373ab36 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java @@ -98,7 +98,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertNotEquals(null, config.getMaxTotalRows()); Assert.assertEquals(1000, config.getMaxTotalRows().longValue()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); - Assert.assertEquals(0, config.getMaxPendingPersists()); + Assert.assertEquals(100, config.getMaxPendingPersists()); Assert.assertEquals(true, config.isReportParseExceptions()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 19b1aa7b79fd..533e8a10193e 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -2161,8 +2161,7 @@ public void testNoDataIngestionTasks() throws Exception @Test(timeout = 60_000L) public void testCheckpointForInactiveTaskGroup() - throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException, NoSuchMethodException, - IllegalAccessException, ClassNotFoundException + throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException { supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); //not adding any events @@ -2265,7 +2264,7 @@ public void testCheckpointForInactiveTaskGroup() @Test(timeout = 60_000L) public void testCheckpointForUnknownTaskGroup() - throws InterruptedException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException + throws InterruptedException { supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); //not adding any events @@ -2344,8 +2343,7 @@ public void testCheckpointForUnknownTaskGroup() @Test(timeout = 60_000L) public void testCheckpointWithNullTaskGroupId() - throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException, NoSuchMethodException, - IllegalAccessException, ClassNotFoundException + throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException { supervisor = getSupervisor(1, 3, true, "PT1S", null, null, false); //not adding any events diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java index 4cddabac70d0..45470ffe10de 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java @@ -107,7 +107,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(100, config.getMaxRowsInMemory()); Assert.assertEquals(100, config.getMaxRowsPerSegment()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); - Assert.assertEquals(0, config.getMaxPendingPersists()); + Assert.assertEquals(100, config.getMaxPendingPersists()); Assert.assertEquals(true, config.isReportParseExceptions()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); Assert.assertEquals(12, (int) config.getWorkerThreads()); diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index ed6f90ca77a5..db79d856671c 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -53,19 +53,6 @@ ${project.parent.version} provided - - io.netty - netty - 3.10.4.Final - provided - - - - com.fasterxml.jackson.dataformat - jackson-dataformat-cbor - ${jackson.version} - - com.amazonaws aws-java-sdk-kinesis @@ -89,11 +76,11 @@ org.easymock easymock + test junit junit - 4.12 test @@ -117,12 +104,6 @@ test-jar test - - cloud.localstack - localstack-utils - 0.1.13 - test - diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java index db9c21ef7eef..8f1c66a1acde 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClient.java @@ -27,8 +27,6 @@ public class KinesisIndexTaskClient extends SeekableStreamIndexTaskClient { - private static ObjectMapper mapper = new ObjectMapper(); - KinesisIndexTaskClient( HttpClient httpClient, ObjectMapper jsonMapper, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index c7aabb8b60d5..f6b662f90ba6 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -133,30 +133,32 @@ protected void possiblyResetDataSourceMetadata( Map currOffsets ) { - for (final StreamPartition streamPartition : assignment) { - String sequence = currOffsets.get(streamPartition.getPartitionId()); - String earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); - if (earliestSequenceNumber == null - || createSequenceNumber(earliestSequenceNumber).compareTo(createSequenceNumber(sequence)) > 0) { - if (task.getTuningConfig().isResetOffsetAutomatically()) { - log.info("Attempting to reset sequences automatically for all partitions"); - try { - sendResetRequestAndWait( - assignment.stream() - .collect(Collectors.toMap(x -> x, x -> currOffsets.get(x.getPartitionId()))), - toolbox + if (!task.getTuningConfig().isSkipSequenceNumberAvailabilityCheck()) { + for (final StreamPartition streamPartition : assignment) { + String sequence = currOffsets.get(streamPartition.getPartitionId()); + String earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); + if (earliestSequenceNumber == null + || createSequenceNumber(earliestSequenceNumber).compareTo(createSequenceNumber(sequence)) > 0) { + if (task.getTuningConfig().isResetOffsetAutomatically()) { + log.info("Attempting to reset sequences automatically for all partitions"); + try { + sendResetRequestAndWait( + assignment.stream() + .collect(Collectors.toMap(x -> x, x -> currOffsets.get(x.getPartitionId()))), + toolbox + ); + } + catch (IOException e) { + throw new ISE(e, "Exception while attempting to automatically reset sequences"); + } + } else { + throw new ISE( + "Starting sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) and resetOffsetAutomatically is not enabled", + sequence, + streamPartition.getPartitionId(), + earliestSequenceNumber ); } - catch (IOException e) { - throw new ISE(e, "Exception while attempting to automatically reset sequences"); - } - } else { - throw new ISE( - "Starting sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) and resetOffsetAutomatically is not enabled", - sequence, - streamPartition.getPartitionId(), - earliestSequenceNumber - ); } } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java index 72a07e54f577..98915a4080f4 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java @@ -83,7 +83,7 @@ public KinesisIndexTaskTuningConfig( maxTotalRows, intermediatePersistPeriod, basePersistDirectory, - 0, + maxPendingPersists, indexSpec, true, reportParseExceptions, @@ -158,7 +158,7 @@ public KinesisIndexTaskTuningConfig withBasePersistDirectory(File dir) getMaxTotalRows(), getIntermediatePersistPeriod(), dir, - 0, + getMaxPendingPersists(), getIndexSpec(), true, isReportParseExceptions(), @@ -221,7 +221,7 @@ public int hashCode() getMaxRowsPerSegment(), getIntermediatePersistPeriod(), getBasePersistDirectory(), - 0, + getMaxPendingPersists(), getIndexSpec(), true, isReportParseExceptions(), @@ -249,7 +249,7 @@ public String toString() ", maxRowsPerSegment=" + getMaxRowsPerSegment() + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + - ", maxPendingPersists=" + 0 + + ", maxPendingPersists=" + getMaxPendingPersists() + ", indexSpec=" + getIndexSpec() + ", reportParseExceptions=" + isReportParseExceptions() + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java index 748b7fa83912..bf6ee6081090 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java @@ -24,8 +24,11 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; +import com.google.inject.TypeLiteral; +import org.apache.druid.guice.LazySingleton; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.initialization.DruidModule; import java.util.List; @@ -50,5 +53,10 @@ public List getJacksonModules() @Override public void configure(Binder binder) { + binder.bind( + new TypeLiteral>() + { + } + ).to(KinesisIndexTaskClientFactory.class).in(LazySingleton.class); } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 1083c23c5c6b..4dee7fd8de44 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -297,7 +297,7 @@ public KinesisRecordSupplier( int recordBufferFullWait, int fetchSequenceNumberTimeout, int maxRecordsPerPoll - ) throws RuntimeException + ) { Preconditions.checkNotNull(amazonKinesis); this.kinesis = amazonKinesis; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java index a33349976503..65059e729a2f 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java @@ -26,7 +26,6 @@ import javax.validation.constraints.NotNull; import java.math.BigInteger; -import java.util.Objects; public class KinesisSequenceNumber extends OrderedSequenceNumber { @@ -60,16 +59,6 @@ public static KinesisSequenceNumber of(String sequenceNumber, boolean isExclusiv return new KinesisSequenceNumber(sequenceNumber, isExclusive); } - @Override - public boolean equals(Object o) - { - if (!(o instanceof KinesisSequenceNumber)) { - return false; - } - return this.compareTo((KinesisSequenceNumber) o) == 0; - } - - @Override public int compareTo(@NotNull OrderedSequenceNumber o) { @@ -85,9 +74,4 @@ public int compareTo(@NotNull OrderedSequenceNumber o) } } - @Override - public int hashCode() - { - return Objects.hash(super.hashCode(), intSequence); - } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java index 3c29d88c0b80..a1f2749b147f 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java @@ -53,7 +53,7 @@ public String toString() { return "{" + "className=" + this.getClass().getSimpleName() + - "dataSource='" + getDataSource() + '\'' + + ", dataSource='" + getDataSource() + '\'' + ", stream='" + getStream() + '\'' + ", partitions=" + getPartitions() + ", replicas=" + getReplicas() + diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 4efae927a875..0f2e021d5022 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -19,8 +19,6 @@ package org.apache.druid.indexing.kinesis; -import cloud.localstack.LocalstackTestRunner; -import cloud.localstack.TestUtils; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -63,6 +61,7 @@ import org.apache.druid.indexing.common.TaskReportFileWriter; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; +import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionToolbox; @@ -384,11 +383,11 @@ public void testRunAfterDataInserted() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -468,11 +467,11 @@ public void testRunBeforeDataInserted() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -580,11 +579,11 @@ public void testIncrementalHandOff() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -722,11 +721,11 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -845,11 +844,11 @@ public void testRunWithMinimumMessageTime() throws Exception null, DateTimes.of("2010"), null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -931,11 +930,11 @@ public void testRunWithMaximumMessageTime() throws Exception null, null, DateTimes.of("2010"), - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1027,11 +1026,11 @@ public void testRunWithTransformSpec() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1110,11 +1109,11 @@ public void testRunOnNothing() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1176,11 +1175,11 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1256,11 +1255,11 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1336,11 +1335,11 @@ public void testReportParseExceptions() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1405,11 +1404,11 @@ public void testMultipleParseExceptionsSuccess() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1518,11 +1517,11 @@ public void testMultipleParseExceptionsFailure() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1610,11 +1609,11 @@ public void testRunReplicas() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1638,11 +1637,11 @@ public void testRunReplicas() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1727,11 +1726,11 @@ public void testRunConflicting() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1755,11 +1754,11 @@ public void testRunConflicting() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1844,11 +1843,11 @@ public void testRunConflictingWithoutTransactions() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1872,11 +1871,11 @@ public void testRunConflictingWithoutTransactions() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -1962,11 +1961,11 @@ public void testRunOneTaskTwoPartitions() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -2055,11 +2054,11 @@ public void testRunTwoTasksTwoPartitions() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -2083,11 +2082,11 @@ public void testRunTwoTasksTwoPartitions() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -2175,11 +2174,11 @@ public void testRestore() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -2237,11 +2236,11 @@ public void testRestore() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", ImmutableSet.of(shardId1), null, null, @@ -2324,11 +2323,11 @@ public void testRunWithPauseAndResume() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -2465,11 +2464,11 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception null, null, null, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - TestUtils.TEST_ACCESS_KEY, - TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -2553,7 +2552,7 @@ public boolean apply(TaskLock lock) private KinesisIndexTask createTask( final String taskId, final KinesisIndexTaskIOConfig ioConfig - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { return createTask(taskId, DATA_SCHEMA, ioConfig); } @@ -2562,7 +2561,7 @@ private KinesisIndexTask createTask( final String taskId, final KinesisIndexTaskIOConfig ioConfig, final Map context - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { return createTask(taskId, DATA_SCHEMA, ioConfig, context); } @@ -2571,7 +2570,7 @@ private KinesisIndexTask createTask( final String taskId, final DataSchema dataSchema, final KinesisIndexTaskIOConfig ioConfig - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { final KinesisIndexTaskTuningConfig tuningConfig = new KinesisIndexTaskTuningConfig( 1000, @@ -2621,7 +2620,7 @@ private KinesisIndexTask createTask( final DataSchema dataSchema, final KinesisIndexTaskIOConfig ioConfig, final Map context - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { final KinesisIndexTaskTuningConfig tuningConfig = new KinesisIndexTaskTuningConfig( maxRowsInMemory, @@ -2712,7 +2711,7 @@ public QueryRunner decorate( private void makeToolboxFactory() throws IOException { directory = tempFolder.newFolder(); - final org.apache.druid.indexing.common.TestUtils testUtils = new org.apache.druid.indexing.common.TestUtils(); + final TestUtils testUtils = new TestUtils(); rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory(); final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); for (Module module : new KinesisIndexingServiceModule().getJacksonModules()) { @@ -3014,7 +3013,7 @@ public TestableKinesisIndexTask( @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory - ) throws NoSuchMethodException, IllegalAccessException, ClassNotFoundException + ) { super( id, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index 283dd152da2b..ab07212e1ea2 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -117,7 +117,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(100, config.getMaxRowsInMemory()); Assert.assertEquals(100, config.getMaxRowsPerSegment()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); - Assert.assertEquals(0, config.getMaxPendingPersists()); + Assert.assertEquals(100, config.getMaxPendingPersists()); Assert.assertTrue(config.getBuildV9Directly()); Assert.assertTrue(config.isReportParseExceptions()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index dd696ca89262..64d598f727db 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -163,7 +163,6 @@ public void tearDownTest() @Test public void testSupplierSetup() - throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException { Capture captured = Capture.newInstance(); expect(kinesis.describeStream(capture(captured))).andReturn(describeStreamResult).once(); @@ -219,8 +218,7 @@ private static List> cleanRecords(Lis } @Test - public void testPoll() - throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException, InterruptedException + public void testPoll() throws InterruptedException { recordsPerFetch = 100; @@ -291,7 +289,7 @@ public void testPoll() @Test public void testSeek() - throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException + throws InterruptedException { recordsPerFetch = 100; @@ -366,7 +364,7 @@ public void testSeek() @Test public void testSeekToLatest() - throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException + throws InterruptedException { recordsPerFetch = 100; @@ -425,7 +423,6 @@ public void testSeekToLatest() @Test(expected = ISE.class) public void testSeekUnassigned() - throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException { StreamPartition shard0 = StreamPartition.of(stream, shardId0); StreamPartition shard1 = StreamPartition.of(stream, shardId1); @@ -453,7 +450,7 @@ public void testSeekUnassigned() @Test public void testPollAfterSeek() - throws InterruptedException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException + throws InterruptedException { // tests that after doing a seek, the now invalid records in buffer is cleaned up properly recordsPerFetch = 100; @@ -529,8 +526,7 @@ public void testPollAfterSeek() @Test - public void testPollDeaggregate() - throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException, InterruptedException + public void testPollDeaggregate() throws InterruptedException { recordsPerFetch = 100; diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 8752a54912ce..601891b3803e 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.kinesis.supervisor; -import cloud.localstack.LocalstackTestRunner; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; @@ -173,12 +172,6 @@ public KinesisSupervisorTest() @BeforeClass public static void setupClass() { - cloud.localstack.TestUtils.setEnv("AWS_CBOR_DISABLE", "1"); - /* Disable SSL certificate checks for local testing */ - if (LocalstackTestRunner.useSSL()) { - cloud.localstack.TestUtils.disableSslCertChecking(); - } - dataSchema = getDataSchema(DATASOURCE); } @@ -1550,9 +1543,9 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); KinesisIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); - Assert.assertEquals(cloud.localstack.TestUtils.TEST_ACCESS_KEY, capturedTaskConfig.getAwsAccessKeyId()); - Assert.assertEquals(cloud.localstack.TestUtils.TEST_SECRET_KEY, capturedTaskConfig.getAwsSecretAccessKey()); - Assert.assertEquals(LocalstackTestRunner.getEndpointKinesis(), capturedTaskConfig.getEndpoint()); + Assert.assertEquals("awsKey", capturedTaskConfig.getAwsAccessKeyId()); + Assert.assertEquals("awsSecret", capturedTaskConfig.getAwsSecretAccessKey()); + Assert.assertEquals("awsEndpoint", capturedTaskConfig.getEndpoint()); Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); @@ -1691,9 +1684,9 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); KinesisIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); - Assert.assertEquals(cloud.localstack.TestUtils.TEST_ACCESS_KEY, capturedTaskConfig.getAwsAccessKeyId()); - Assert.assertEquals(cloud.localstack.TestUtils.TEST_SECRET_KEY, capturedTaskConfig.getAwsSecretAccessKey()); - Assert.assertEquals(LocalstackTestRunner.getEndpointKinesis(), capturedTaskConfig.getEndpoint()); + Assert.assertEquals("awsKey", capturedTaskConfig.getAwsAccessKeyId()); + Assert.assertEquals("awsSecret", capturedTaskConfig.getAwsSecretAccessKey()); + Assert.assertEquals("awsEndpoint", capturedTaskConfig.getEndpoint()); Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); @@ -2812,8 +2805,7 @@ public void testNoDataIngestionTasks() throws Exception @Test(timeout = 60_000L) public void testCheckpointForInactiveTaskGroup() - throws InterruptedException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException, - ExecutionException, TimeoutException, JsonProcessingException + throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException { supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false); //not adding any events @@ -3091,8 +3083,7 @@ public void testCheckpointForUnknownTaskGroup() @Test(timeout = 60_000L) public void testCheckpointWithNullTaskGroupId() - throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException, NoSuchMethodException, - IllegalAccessException, ClassNotFoundException + throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException { supervisor = getSupervisor(1, 3, true, "PT1S", null, null, false); //not adding any events @@ -3498,7 +3489,7 @@ private KinesisSupervisor getSupervisor( { KinesisSupervisorIOConfig KinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( stream, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, replicas, taskCount, @@ -3511,8 +3502,8 @@ private KinesisSupervisor getSupervisor( earlyMessageRejectionPeriod, recordsPerFetch, fetchDelayMillis, - cloud.localstack.TestUtils.TEST_ACCESS_KEY, - cloud.localstack.TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, false @@ -3649,11 +3640,11 @@ private KinesisIndexTask createKinesisIndexTask( null, minimumMessageTime, maximumMessageTime, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, null, - cloud.localstack.TestUtils.TEST_ACCESS_KEY, - cloud.localstack.TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, null, @@ -3678,7 +3669,7 @@ private KinesisSupervisor getSupervisor( { KinesisSupervisorIOConfig KinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( stream, - LocalstackTestRunner.getEndpointKinesis(), + "awsEndpoint", null, replicas, taskCount, @@ -3691,8 +3682,8 @@ private KinesisSupervisor getSupervisor( earlyMessageRejectionPeriod, null, null, - cloud.localstack.TestUtils.TEST_ACCESS_KEY, - cloud.localstack.TestUtils.TEST_SECRET_KEY, + "awsKey", + "awsSecret", null, null, false diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java index 46600426732b..370f3ea2ea0d 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java @@ -107,7 +107,7 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(100, config.getMaxRowsInMemory()); Assert.assertEquals(100, config.getMaxRowsPerSegment()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); - Assert.assertEquals(0, config.getMaxPendingPersists()); + Assert.assertEquals(100, config.getMaxPendingPersists()); Assert.assertEquals(true, config.getBuildV9Directly()); Assert.assertEquals(true, config.isReportParseExceptions()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 3f4653c225e1..3421f08289f8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -281,10 +281,7 @@ public boolean withinMinMaxRecordTime(final InputRow row) protected abstract SeekableStreamIndexTaskRunner createTaskRunner(); - protected abstract RecordSupplier newTaskRecordSupplier() - throws ClassNotFoundException, - NoSuchMethodException, - IllegalAccessException; + protected abstract RecordSupplier newTaskRecordSupplier(); @VisibleForTesting public Appenderator getAppenderator() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index a6fc4c1b6096..99851eb9123b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -1126,12 +1126,6 @@ private boolean possiblyPause() throws InterruptedException return false; } - protected abstract void possiblyResetDataSourceMetadata( - TaskToolbox toolbox, - RecordSupplier recordSupplier, - Set> assignment, - Map currOffsets - ); private void handleParseException(ParseException pe, OrderedPartitionableRecord record) { @@ -2021,4 +2015,20 @@ protected abstract SeekableStreamDataSourceMetadata recordSupplier, + Set> assignment, + Map currOffsets + ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java index 5c1696085e26..816949ce4e16 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java @@ -96,7 +96,7 @@ public SeekableStreamIndexTaskTuningConfig( ? defaults.getIntermediatePersistPeriod() : intermediatePersistPeriod; this.basePersistDirectory = defaults.getBasePersistDirectory(); - this.maxPendingPersists = 0; + this.maxPendingPersists = maxPendingPersists == null ? 0 : maxPendingPersists; this.indexSpec = indexSpec == null ? defaults.getIndexSpec() : indexSpec; this.reportParseExceptions = reportParseExceptions == null ? defaults.isReportParseExceptions() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java index 010fdfdbc00d..3a6f10b31dce 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java @@ -26,11 +26,17 @@ * Represents a generic record with a PartitionIdType (partition id) and SequenceOffsetType (sequence number) and data * from a Kafka/Kinesis stream * - * @param partition id - * @param sequence number + * @param partition id + * @param sequence number */ public class OrderedPartitionableRecord { + /** + * In Kinesis, when a shard is closed due to shard splitting, a null ShardIterator is returned. + * The EOS marker is placed at the end of the Kinesis Record Supplier buffer, such that when + * an indexing task pulls the record 'EOS', it knows the shard has been closed and should stop + * reading and start publishing + */ public static final String END_OF_SHARD_MARKER = "EOS"; private final String stream; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 21cc3b58f5ac..4ca2d93ee26b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -218,8 +218,7 @@ public String toString() */ private interface Notice { - void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException, - NoSuchMethodException, IllegalAccessException, ClassNotFoundException; + void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException; } private static class StatsFromTaskResult @@ -259,8 +258,7 @@ public Map getStats() private class RunNotice implements Notice { @Override - public void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException, - NoSuchMethodException, IllegalAccessException, ClassNotFoundException + public void handle() throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException { long nowTime = System.currentTimeMillis(); if (nowTime - lastRunTime < MAX_RUN_FREQUENCY_MILLIS) { @@ -700,7 +698,7 @@ public void reset(DataSourceMetadata dataSourceMetadata) @VisibleForTesting - protected void tryInit() + public void tryInit() { synchronized (stateChangeLock) { if (started) { @@ -1003,8 +1001,7 @@ public void addTaskGroupToPendingCompletionTaskGroup( @VisibleForTesting public void runInternal() - throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException, NoSuchMethodException, - IllegalAccessException, ClassNotFoundException + throws ExecutionException, InterruptedException, TimeoutException, JsonProcessingException { possiblyRegisterListener(); updatePartitionDataFromStream(); @@ -2214,7 +2211,7 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep } private void createNewTasks() - throws JsonProcessingException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException + throws JsonProcessingException { // update the checkpoints in the taskGroup to latest ones so that new tasks do not read what is already published verifyAndMergeCheckpoints( @@ -2443,7 +2440,7 @@ private SequenceOffsetType getOffsetFromStreamForPartition(PartitionIdType parti } private void createTasksForGroup(int groupId, int replicas) - throws JsonProcessingException, NoSuchMethodException, IllegalAccessException, ClassNotFoundException + throws JsonProcessingException { TaskGroup group = activelyReadingTaskGroups.get(groupId); Map startPartitions = group.startingSequences; @@ -2674,7 +2671,7 @@ protected abstract List makeSequenceNumber( * * @return specific instance of Kafka/Kinesis RecordSupplier */ - protected abstract RecordSupplier setupRecordSupplier() - throws IllegalAccessException, NoSuchMethodException, ClassNotFoundException; + protected abstract RecordSupplier setupRecordSupplier(); /** * creates a specific instance of Kafka/Kinesis Supervisor Report Payload @@ -2781,7 +2777,22 @@ private boolean checkSequenceAvailability( } + /** + * a special sequence number that is used to indicate that the sequence offset + * for a particular partition has not yet been calculated by the supervisor. When + * the not_set marker is read by the supervisor, it will first attempt to restore it + * from metadata storage, if that fails, from the Kafka/Kinesis + * + * @return sequence offset that represets NOT_SET + */ protected abstract SequenceOffsetType getNotSetMarker(); + /** + * returns the logical maximum number for a Kafka partition or Kinesis shard. This is + * used to set the initial endoffsets when creating a new task, since we don't know + * what sequence offsets to read to initially + * + * @return end of partition sequence offset + */ protected abstract SequenceOffsetType getEndOfPartitionMarker(); } From b11d5147e8082d2d2d8acc886999329abf0f01e7 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Mon, 17 Dec 2018 13:44:48 -0800 Subject: [PATCH 71/87] teamcity --- .../org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 0f2e021d5022..a10a2a916ca0 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -2922,7 +2922,7 @@ private List readSegmentColumn(final String column, final SegmentDescrip Predicates.alwaysFalse(), false ); - IndexIO indexIO = new org.apache.druid.indexing.common.TestUtils().getTestIndexIO(); + IndexIO indexIO = new TestUtils().getTestIndexIO(); QueryableIndex index = indexIO.loadIndex(outputLocation); DictionaryEncodedColumn theColumn = (DictionaryEncodedColumn) index.getColumnHolder(column) .getColumn(); From ca82037a249222cd1c34be3708ec267fc1022135 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 18 Dec 2018 12:51:51 -0800 Subject: [PATCH 72/87] comments part 1 --- ...ementalPublishingKafkaIndexTaskRunner.java | 6 ++ .../indexing/kafka/KafkaIndexTaskModule.java | 4 +- .../indexing/kafka/KafkaRecordSupplier.java | 4 -- .../kafka/LegacyKafkaIndexTaskRunner.java | 6 ++ .../kafka/supervisor/KafkaSupervisor.java | 6 ++ .../KafkaSupervisorReportPayload.java | 2 +- .../KafkaSupervisorTuningConfig.java | 4 +- .../kafka/KafkaIndexTaskTuningConfigTest.java | 2 +- .../indexing/kinesis/KinesisIndexTask.java | 11 ++- .../KinesisIndexTaskClientFactory.java | 5 +- .../kinesis/KinesisIndexTaskIOConfig.java | 23 +----- .../kinesis/KinesisIndexTaskRunner.java | 6 ++ .../kinesis/KinesisIndexingServiceModule.java | 4 ++ .../kinesis/KinesisRecordSupplier.java | 4 +- .../kinesis/KinesisSequenceNumber.java | 10 ++- .../kinesis/supervisor/KinesisSupervisor.java | 21 ++++-- .../supervisor/KinesisSupervisorIOConfig.java | 28 ++------ .../KinesisSupervisorReportPayload.java | 5 +- .../supervisor/KinesisSupervisorSpec.java | 13 +++- .../KinesisSupervisorTuningConfig.java | 4 +- .../indexing/kinesis/KinesisIOConfigTest.java | 6 -- .../kinesis/KinesisIndexTaskTest.java | 70 ++++--------------- .../KinesisIndexTaskTuningConfigTest.java | 2 +- .../kinesis/KinesisRecordSupplierTest.java | 2 +- .../KinesisSupervisorIOConfigTest.java | 6 -- .../supervisor/KinesisSupervisorTest.java | 22 +++--- .../SeekableStreamIndexTaskRunner.java | 15 ++-- .../common/OrderedPartitionableRecord.java | 8 --- .../supervisor/SeekableStreamSupervisor.java | 12 ++-- 29 files changed, 134 insertions(+), 177 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 01b2d40c9b38..c9487f25a348 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -239,6 +239,12 @@ protected void possiblyResetDataSourceMetadata( } } + @Override + protected boolean isEndOfShard(Long seqNum) + { + return false; + } + @Nullable @Override protected TreeMap> getCheckPointsFromContext( diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java index c74359fff1e9..47b5df82aebe 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java @@ -26,9 +26,9 @@ import com.google.inject.Binder; import com.google.inject.TypeLiteral; import org.apache.druid.guice.LazySingleton; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.initialization.DruidModule; import java.util.List; @@ -54,7 +54,7 @@ public List getJacksonModules() public void configure(Binder binder) { binder.bind( - new TypeLiteral>() + new TypeLiteral>() { } ).to(KafkaIndexTaskClientFactory.class).in(LazySingleton.class); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 4a49351f47bb..844994038a6a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -41,15 +41,11 @@ import java.util.List; import java.util.Map; import java.util.Properties; -import java.util.Random; import java.util.Set; -import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; public class KafkaRecordSupplier implements RecordSupplier { - private static final Random RANDOM = ThreadLocalRandom.current(); - private final KafkaConsumer consumer; private final Map consumerProperties; private final ObjectMapper sortingMapper; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 0593473c5105..ced7a4b723e1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -624,6 +624,12 @@ public void run() ); } + @Override + protected boolean isEndOfShard(Long seqNum) + { + return false; + } + @Override protected List> getRecords( RecordSupplier recordSupplier, TaskToolbox toolbox diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index e1298d1aa8b6..c113bcc7a3f0 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -355,6 +355,12 @@ protected Long getEndOfPartitionMarker() return END_OF_PARTITION; } + @Override + protected boolean isEndOfShard(Long seqNum) + { + return false; + } + @Override protected void updateLatestSequenceFromStream( RecordSupplier recordSupplier, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java index 17b8b449e1e3..d5f4efa02a20 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorReportPayload.java @@ -59,7 +59,7 @@ public KafkaSupervisorReportPayload( @Override public String toString() { - return "{" + + return "KafkaSupervisorReportPayload{" + "dataSource='" + getDataSource() + '\'' + ", topic='" + getStream() + '\'' + ", partitions=" + getPartitions() + diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index 230f9ad3e710..8e4c6e9f9137 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -155,7 +155,7 @@ public String toString() ", maxBytesInMemory=" + TuningConfigs.getMaxBytesInMemoryOrDefault(getMaxBytesInMemory()) + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + - ", maxPendingPersists=" + 0 + + ", maxPendingPersists=" + getMaxPendingPersists() + ", indexSpec=" + getIndexSpec() + ", reportParseExceptions=" + isReportParseExceptions() + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + @@ -184,7 +184,7 @@ public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig() getMaxTotalRows(), getIntermediatePersistPeriod(), getBasePersistDirectory(), - 0, + getMaxPendingPersists(), getIndexSpec(), true, isReportParseExceptions(), diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java index 8a6ee373ab36..abcf07f6c10b 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java @@ -139,7 +139,7 @@ public void testConvert() Assert.assertEquals(10L, copy.getMaxTotalRows().longValue()); Assert.assertEquals(new Period("PT3S"), copy.getIntermediatePersistPeriod()); Assert.assertEquals(new File("/tmp/xxx"), copy.getBasePersistDirectory()); - Assert.assertEquals(0, copy.getMaxPendingPersists()); + Assert.assertEquals(4, copy.getMaxPendingPersists()); Assert.assertEquals(new IndexSpec(), copy.getIndexSpec()); Assert.assertEquals(true, copy.isReportParseExceptions()); Assert.assertEquals(5L, copy.getHandoffConditionTimeout()); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 522224c37b1c..0bd9826ea78e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; @@ -34,6 +35,8 @@ public class KinesisIndexTask extends SeekableStreamIndexTask { + private final AWSCredentialsConfig awsCredentialsConfig; + @JsonCreator public KinesisIndexTask( @JsonProperty("id") String id, @@ -44,7 +47,8 @@ public KinesisIndexTask( @JsonProperty("context") Map context, @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, - @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, + @JacksonInject AWSCredentialsConfig awsCredentialsConfig ) { super( @@ -59,6 +63,7 @@ public KinesisIndexTask( rowIngestionMetersFactory, "index_kinesis" ); + this.awsCredentialsConfig = awsCredentialsConfig; } @@ -88,8 +93,8 @@ protected KinesisRecordSupplier newTaskRecordSupplier() return new KinesisRecordSupplier( KinesisRecordSupplier.getAmazonKinesisClient( ioConfig.getEndpoint(), - ioConfig.getAwsAccessKeyId(), - ioConfig.getAwsSecretAccessKey(), + awsCredentialsConfig.getAccessKey().getPassword(), + awsCredentialsConfig.getSecretKey().getPassword(), ioConfig.getAwsAssumedRoleArn(), ioConfig.getAwsExternalId() ), diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java index 986cb5f0408a..4f61892f177d 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java @@ -31,7 +31,10 @@ public class KinesisIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory { @Inject - public KinesisIndexTaskClientFactory(@EscalatedGlobal HttpClient httpClient, @Json ObjectMapper mapper) + public KinesisIndexTaskClientFactory( + @EscalatedGlobal HttpClient httpClient, + @Json ObjectMapper mapper + ) { super( httpClient, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java index 65e663c5dda9..044ee0bfc653 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java @@ -24,7 +24,6 @@ import com.google.common.base.Preconditions; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; -import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -40,8 +39,6 @@ public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig exclusiveStartSequenceNumberPartitions, @JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn, @JsonProperty("awsExternalId") String awsExternalId, @@ -82,14 +77,12 @@ public KinesisIndexTaskIOConfig( Preconditions.checkArgument(endPartitions.getPartitionSequenceNumberMap() .values() .stream() - .noneMatch(x -> x.equals(OrderedPartitionableRecord.END_OF_SHARD_MARKER))); + .noneMatch(x -> x.equals(KinesisSequenceNumber.END_OF_SHARD_MARKER))); this.pauseAfterRead = pauseAfterRead != null ? pauseAfterRead : DEFAULT_PAUSE_AFTER_READ; this.endpoint = Preconditions.checkNotNull(endpoint, "endpoint"); this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : DEFAULT_RECORDS_PER_FETCH; this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : DEFAULT_FETCH_DELAY_MILLIS; - this.awsAccessKeyId = awsAccessKeyId; - this.awsSecretAccessKey = awsSecretAccessKey; this.awsAssumedRoleArn = awsAssumedRoleArn; this.awsExternalId = awsExternalId; this.deaggregate = deaggregate; @@ -119,18 +112,6 @@ public int getFetchDelayMillis() return fetchDelayMillis; } - @JsonProperty - public String getAwsAccessKeyId() - { - return awsAccessKeyId; - } - - @JsonProperty - public String getAwsSecretAccessKey() - { - return awsSecretAccessKey; - } - @JsonProperty public String getAwsAssumedRoleArn() { @@ -163,8 +144,6 @@ public String toString() ", endpoint='" + endpoint + '\'' + ", recordsPerFetch=" + recordsPerFetch + ", fetchDelayMillis=" + fetchDelayMillis + - ", awsAccessKeyId='" + awsAccessKeyId + '\'' + - ", awsSecretAccessKey=" + "************************" + ", exclusiveStartSequenceNumberPartitions=" + getExclusiveStartSequenceNumberPartitions() + ", awsAssumedRoleArn='" + awsAssumedRoleArn + '\'' + ", awsExternalId='" + awsExternalId + '\'' + diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index f6b662f90ba6..7f218baea418 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -164,6 +164,12 @@ protected void possiblyResetDataSourceMetadata( } } + @Override + protected boolean isEndOfShard(String seqNum) + { + return KinesisSequenceNumber.END_OF_SHARD_MARKER.equals(seqNum); + } + @Nullable @Override protected TreeMap> getCheckPointsFromContext( diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java index bf6ee6081090..e32f710e98a2 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java @@ -25,6 +25,8 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.TypeLiteral; +import org.apache.druid.common.aws.AWSCredentialsConfig; +import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig; @@ -58,5 +60,7 @@ public void configure(Binder binder) { } ).to(KinesisIndexTaskClientFactory.class).in(LazySingleton.class); + + JsonConfigProvider.bind(binder, "druid.kinesis", AWSCredentialsConfig.class); } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 4dee7fd8de44..97eb7e3d0c2b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -150,7 +150,7 @@ private Runnable getRecordRunnable() OrderedPartitionableRecord endOfShardRecord = new OrderedPartitionableRecord<>( streamPartition.getStream(), streamPartition.getPartitionId(), - OrderedPartitionableRecord.END_OF_SHARD_MARKER, + KinesisSequenceNumber.END_OF_SHARD_MARKER, null ); @@ -658,7 +658,7 @@ private String getSequenceNumberInternal(StreamPartition partition, Stri if (shardIterator == null) { log.info("Partition[%s] returned a null shard iterator, is the shard closed?", partition.getPartitionId()); - return OrderedPartitionableRecord.END_OF_SHARD_MARKER; + return KinesisSequenceNumber.END_OF_SHARD_MARKER; } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java index 65059e729a2f..9a96e4ed5985 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSequenceNumber.java @@ -21,7 +21,6 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; -import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import javax.validation.constraints.NotNull; @@ -30,6 +29,13 @@ public class KinesisSequenceNumber extends OrderedSequenceNumber { + /** + * In Kinesis, when a shard is closed due to shard splitting, a null ShardIterator is returned. + * The EOS marker is placed at the end of the Kinesis Record Supplier buffer, such that when + * an indexing task pulls the record 'EOS', it knows the shard has been closed and should stop + * reading and start publishing + */ + public static final String END_OF_SHARD_MARKER = "EOS"; // this flag is used to indicate either END_OF_SHARD_MARKER // or NO_END_SEQUENCE_NUMBER so that they can be properly compared // with other sequence numbers @@ -39,7 +45,7 @@ public class KinesisSequenceNumber extends OrderedSequenceNumber private KinesisSequenceNumber(@NotNull String sequenceNumber, boolean isExclusive) { super(sequenceNumber, isExclusive); - if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(sequenceNumber) + if (END_OF_SHARD_MARKER.equals(sequenceNumber) || SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(sequenceNumber)) { isMaxSequenceNumber = true; this.intSequence = null; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 24748b40ce18..e6471e476a9f 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; @@ -74,6 +75,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor { private static final String NOT_SET = "-1"; private final KinesisSupervisorSpec spec; + private final AWSCredentialsConfig awsCredentialsConfig; public KinesisSupervisor( final TaskStorage taskStorage, @@ -82,7 +84,8 @@ public KinesisSupervisor( final KinesisIndexTaskClientFactory taskClientFactory, final ObjectMapper mapper, final KinesisSupervisorSpec spec, - final RowIngestionMetersFactory rowIngestionMetersFactory + final RowIngestionMetersFactory rowIngestionMetersFactory, + final AWSCredentialsConfig awsCredentialsConfig ) { super( @@ -98,6 +101,7 @@ public KinesisSupervisor( ); this.spec = spec; + this.awsCredentialsConfig = awsCredentialsConfig; } @Override @@ -125,8 +129,6 @@ protected SeekableStreamIndexTaskIOConfig createIoConfig( ioConfig.getEndpoint(), ioConfig.getRecordsPerFetch(), ioConfig.getFetchDelayMillis(), - ioConfig.getAwsAccessKeyId(), - ioConfig.getAwsSecretAccessKey(), exclusiveStartSequenceNumberPartitions, ioConfig.getAwsAssumedRoleArn(), ioConfig.getAwsExternalId(), @@ -171,7 +173,8 @@ protected List> createIndexTasks( context, null, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + awsCredentialsConfig )); } return taskList; @@ -188,8 +191,8 @@ protected RecordSupplier setupRecordSupplier() return new KinesisRecordSupplier( KinesisRecordSupplier.getAmazonKinesisClient( ioConfig.getEndpoint(), - ioConfig.getAwsAccessKeyId(), - ioConfig.getAwsSecretAccessKey(), + awsCredentialsConfig.getAccessKey().getPassword(), + awsCredentialsConfig.getSecretKey().getPassword(), ioConfig.getAwsAssumedRoleArn(), ioConfig.getAwsExternalId() ), @@ -304,4 +307,10 @@ protected String getEndOfPartitionMarker() { return SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER; } + + @Override + protected boolean isEndOfShard(String seqNum) + { + return KinesisSequenceNumber.END_OF_SHARD_MARKER.equals(seqNum); + } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java index 559c04a2e9c0..2fb43d315175 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java @@ -44,8 +44,6 @@ public class KinesisSupervisorIOConfig extends SeekableStreamSupervisorIOConfig private final Integer recordsPerFetch; private final Integer fetchDelayMillis; - private final String awsAccessKeyId; - private final String awsSecretAccessKey; private final String awsAssumedRoleArn; private final String awsExternalId; private final boolean deaggregate; @@ -66,8 +64,6 @@ public KinesisSupervisorIOConfig( @JsonProperty("earlyMessageRejectionPeriod") Period earlyMessageRejectionPeriod, @JsonProperty("recordsPerFetch") Integer recordsPerFetch, @JsonProperty("fetchDelayMillis") Integer fetchDelayMillis, - @JsonProperty("awsAccessKeyId") String awsAccessKeyId, - @JsonProperty("awsSecretAccessKey") String awsSecretAccessKey, @JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn, @JsonProperty("awsExternalId") String awsExternalId, @JsonProperty("deaggregate") boolean deaggregate @@ -88,10 +84,12 @@ public KinesisSupervisorIOConfig( this.endpoint = endpoint != null ? endpoint : (region != null ? region.getEndpoint() : KinesisRegion.US_EAST_1.getEndpoint()); - this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : KinesisIndexTaskIOConfig.DEFAULT_RECORDS_PER_FETCH; - this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : KinesisIndexTaskIOConfig.DEFAULT_FETCH_DELAY_MILLIS; - this.awsAccessKeyId = awsAccessKeyId; - this.awsSecretAccessKey = awsSecretAccessKey; + this.recordsPerFetch = recordsPerFetch != null + ? recordsPerFetch + : KinesisIndexTaskIOConfig.DEFAULT_RECORDS_PER_FETCH; + this.fetchDelayMillis = fetchDelayMillis != null + ? fetchDelayMillis + : KinesisIndexTaskIOConfig.DEFAULT_FETCH_DELAY_MILLIS; this.awsAssumedRoleArn = awsAssumedRoleArn; this.awsExternalId = awsExternalId; this.deaggregate = deaggregate; @@ -115,18 +113,6 @@ public Integer getFetchDelayMillis() return fetchDelayMillis; } - @JsonProperty - public String getAwsAccessKeyId() - { - return awsAccessKeyId; - } - - @JsonProperty - public String getAwsSecretAccessKey() - { - return awsSecretAccessKey; - } - @JsonProperty public String getAwsAssumedRoleArn() { @@ -162,8 +148,6 @@ public String toString() ", earlyMessageRejectionPeriod=" + getEarlyMessageRejectionPeriod() + ", recordsPerFetch=" + recordsPerFetch + ", fetchDelayMillis=" + fetchDelayMillis + - ", awsAccessKeyId='" + awsAccessKeyId + '\'' + - ", awsSecretAccessKey=" + "************************" + ", awsAssumedRoleArn='" + awsAssumedRoleArn + '\'' + ", awsExternalId='" + awsExternalId + '\'' + ", deaggregate=" + deaggregate + diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java index a1f2749b147f..fb08337b4642 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorReportPayload.java @@ -51,9 +51,8 @@ public KinesisSupervisorReportPayload( @Override public String toString() { - return "{" + - "className=" + this.getClass().getSimpleName() + - ", dataSource='" + getDataSource() + '\'' + + return "KinesisSupervisorReportPayload{" + + "dataSource='" + getDataSource() + '\'' + ", stream='" + getStream() + '\'' + ", partitions=" + getPartitions() + ", replicas=" + getReplicas() + diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java index 06da8d024ebf..97dc2c5f0672 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.kinesis.KinesisIndexTaskClientFactory; @@ -39,6 +40,8 @@ public class KinesisSupervisorSpec extends SeekableStreamSupervisorSpec { + private final AWSCredentialsConfig awsCredentialsConfig; + @JsonCreator public KinesisSupervisorSpec( @JsonProperty("dataSchema") DataSchema dataSchema, @@ -53,7 +56,8 @@ public KinesisSupervisorSpec( @JacksonInject @Json ObjectMapper mapper, @JacksonInject ServiceEmitter emitter, @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, - @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, + @JacksonInject AWSCredentialsConfig awsCredentialsConfig ) { super( @@ -103,6 +107,7 @@ public KinesisSupervisorSpec( monitorSchedulerConfig, rowIngestionMetersFactory ); + this.awsCredentialsConfig = awsCredentialsConfig; } @@ -116,7 +121,8 @@ public Supervisor createSupervisor() (KinesisIndexTaskClientFactory) indexTaskClientFactory, mapper, this, - rowIngestionMetersFactory + rowIngestionMetersFactory, + awsCredentialsConfig ); } @@ -161,7 +167,8 @@ protected KinesisSupervisorSpec toggleSuspend(boolean suspend) mapper, emitter, monitorSchedulerConfig, - rowIngestionMetersFactory + rowIngestionMetersFactory, + awsCredentialsConfig ); } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 498481ec828e..7a5abb31f7f7 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -152,7 +152,7 @@ public String toString() ", maxRowsPerSegment=" + getMaxRowsPerSegment() + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + - ", maxPendingPersists=" + 0 + + ", maxPendingPersists=" + getMaxPendingPersists() + ", indexSpec=" + getIndexSpec() + ", buildV9Directly=" + true + ", reportParseExceptions=" + isReportParseExceptions() + @@ -183,7 +183,7 @@ public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig() getMaxTotalRows(), getIntermediatePersistPeriod(), getBasePersistDirectory(), - 0, + getMaxPendingPersists(), getIndexSpec(), true, isReportParseExceptions(), diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index 65710c9942f2..6ddd1c1a098b 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -86,8 +86,6 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(config.getEndpoint(), "kinesis.us-east-1.amazonaws.com"); Assert.assertEquals(config.getRecordsPerFetch(), 4000); Assert.assertEquals(config.getFetchDelayMillis(), 0); - Assert.assertNull(config.getAwsAccessKeyId()); - Assert.assertNull(config.getAwsSecretAccessKey()); Assert.assertEquals(Collections.emptySet(), config.getExclusiveStartSequenceNumberPartitions()); Assert.assertNull(config.getAwsAssumedRoleArn()); Assert.assertNull(config.getAwsExternalId()); @@ -111,8 +109,6 @@ public void testSerdeWithNonDefaults() throws Exception + " \"endpoint\": \"kinesis.us-east-2.amazonaws.com\",\n" + " \"recordsPerFetch\": 1000,\n" + " \"fetchDelayMillis\": 1000,\n" - + " \"awsAccessKeyId\": \"awsid\",\n" - + " \"awsSecretAccessKey\": \"awskey\",\n" + " \"exclusiveStartSequenceNumberPartitions\": [\"0\"],\n" + " \"awsAssumedRoleArn\": \"role\",\n" + " \"awsExternalId\": \"awsexternalid\",\n" @@ -150,8 +146,6 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(config.getExclusiveStartSequenceNumberPartitions(), ImmutableSet.of("0")); Assert.assertEquals(1000, config.getRecordsPerFetch()); Assert.assertEquals(1000, config.getFetchDelayMillis()); - Assert.assertEquals("awsid", config.getAwsAccessKeyId()); - Assert.assertEquals("awskey", config.getAwsSecretAccessKey()); Assert.assertEquals("role", config.getAwsAssumedRoleArn()); Assert.assertEquals("awsexternalid", config.getAwsExternalId()); Assert.assertTrue(config.isDeaggregate()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index a10a2a916ca0..98278e16b765 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Predicate; @@ -41,6 +42,7 @@ import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.MapCache; +import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; import org.apache.druid.data.input.impl.JSONParseSpec; @@ -386,8 +388,6 @@ public void testRunAfterDataInserted() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -470,8 +470,6 @@ public void testRunBeforeDataInserted() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -582,8 +580,6 @@ public void testIncrementalHandOff() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -724,8 +720,6 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -847,8 +841,6 @@ public void testRunWithMinimumMessageTime() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -933,8 +925,6 @@ public void testRunWithMaximumMessageTime() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1029,8 +1019,6 @@ public void testRunWithTransformSpec() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1112,8 +1100,6 @@ public void testRunOnNothing() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1178,8 +1164,6 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1258,8 +1242,6 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1338,8 +1320,6 @@ public void testReportParseExceptions() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1407,8 +1387,6 @@ public void testMultipleParseExceptionsSuccess() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1520,8 +1498,6 @@ public void testMultipleParseExceptionsFailure() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1612,8 +1588,6 @@ public void testRunReplicas() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1640,8 +1614,6 @@ public void testRunReplicas() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1729,8 +1701,6 @@ public void testRunConflicting() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1757,8 +1727,6 @@ public void testRunConflicting() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1846,8 +1814,6 @@ public void testRunConflictingWithoutTransactions() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1874,8 +1840,6 @@ public void testRunConflictingWithoutTransactions() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -1964,8 +1928,6 @@ public void testRunOneTaskTwoPartitions() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -2057,8 +2019,6 @@ public void testRunTwoTasksTwoPartitions() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -2085,8 +2045,6 @@ public void testRunTwoTasksTwoPartitions() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -2177,8 +2135,6 @@ public void testRestore() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -2239,8 +2195,6 @@ public void testRestore() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", ImmutableSet.of(shardId1), null, null, @@ -2326,8 +2280,6 @@ public void testRunWithPauseAndResume() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -2467,8 +2419,6 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -2608,7 +2558,8 @@ private KinesisIndexTask createTask( context, null, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + null ); return task; @@ -2659,7 +2610,8 @@ private KinesisIndexTask createTask( context, null, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + null ); return task; @@ -2714,6 +2666,10 @@ private void makeToolboxFactory() throws IOException final TestUtils testUtils = new TestUtils(); rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory(); final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); + objectMapper.setInjectableValues(((InjectableValues.Std) objectMapper.getInjectableValues()).addValue( + AWSCredentialsConfig.class, + new AWSCredentialsConfig() + )); for (Module module : new KinesisIndexingServiceModule().getJacksonModules()) { objectMapper.registerModule(module); } @@ -3012,7 +2968,8 @@ public TestableKinesisIndexTask( @JsonProperty("context") Map context, @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, - @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, + @JacksonInject AWSCredentialsConfig awsCredentialsConfig ) { super( @@ -3024,7 +2981,8 @@ public TestableKinesisIndexTask( context, chatHandlerProvider, authorizerMapper, - rowIngestionMetersFactory + rowIngestionMetersFactory, + awsCredentialsConfig ); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index ab07212e1ea2..4e967c414a9f 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -201,7 +201,7 @@ public void testConvert() Assert.assertEquals(100L, (long) copy.getMaxTotalRows()); Assert.assertEquals(new Period("PT3S"), copy.getIntermediatePersistPeriod()); Assert.assertEquals(new File("/tmp/xxx"), copy.getBasePersistDirectory()); - Assert.assertEquals(0, copy.getMaxPendingPersists()); + Assert.assertEquals(4, copy.getMaxPendingPersists()); Assert.assertEquals(new IndexSpec(), copy.getIndexSpec()); Assert.assertTrue(copy.getBuildV9Directly()); Assert.assertTrue(copy.isReportParseExceptions()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index 64d598f727db..f20ac86f40f3 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -213,7 +213,7 @@ private static List> cleanRecords(Lis { return records.stream() .filter(x -> !x.getSequenceNumber() - .equals(OrderedPartitionableRecord.END_OF_SHARD_MARKER)) + .equals(KinesisSequenceNumber.END_OF_SHARD_MARKER)) .collect(Collectors.toList()); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java index d3d2bfd5ffa5..037c92bb0511 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java @@ -71,8 +71,6 @@ public void testSerdeWithDefaults() throws Exception Assert.assertFalse("earlyMessageRejectionPeriod", config.getEarlyMessageRejectionPeriod().isPresent()); Assert.assertEquals((Integer) 4000, config.getRecordsPerFetch()); Assert.assertEquals((Integer) 0, config.getFetchDelayMillis()); - Assert.assertNull(config.getAwsAccessKeyId()); - Assert.assertNull(config.getAwsSecretAccessKey()); Assert.assertNull(config.getAwsAssumedRoleArn()); Assert.assertNull(config.getAwsExternalId()); Assert.assertFalse(config.isDeaggregate()); @@ -97,8 +95,6 @@ public void testSerdeWithNonDefaults() throws Exception + " \"earlyMessageRejectionPeriod\": \"PT1H\",\n" + " \"recordsPerFetch\": 4000,\n" + " \"fetchDelayMillis\": 1000,\n" - + " \"awsAccessKeyId\": \"awsid\",\n" - + " \"awsSecretAccessKey\": \"awskey\",\n" + " \"awsAssumedRoleArn\": \"role\",\n" + " \"awsExternalId\": \"awsexternalid\",\n" + " \"deaggregate\": true\n" @@ -122,8 +118,6 @@ public void testSerdeWithNonDefaults() throws Exception Assert.assertEquals(Duration.standardHours(1), config.getEarlyMessageRejectionPeriod().get()); Assert.assertEquals((Integer) 4000, config.getRecordsPerFetch()); Assert.assertEquals((Integer) 1000, config.getFetchDelayMillis()); - Assert.assertEquals("awsid", config.getAwsAccessKeyId()); - Assert.assertEquals("awskey", config.getAwsSecretAccessKey()); Assert.assertEquals("role", config.getAwsAssumedRoleArn()); Assert.assertEquals("awsexternalid", config.getAwsExternalId()); Assert.assertTrue(config.isDeaggregate()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 601891b3803e..5bd9404c90f8 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -1543,8 +1543,6 @@ public void testDiscoverExistingPublishingTask() throws Exception Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); KinesisIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); - Assert.assertEquals("awsKey", capturedTaskConfig.getAwsAccessKeyId()); - Assert.assertEquals("awsSecret", capturedTaskConfig.getAwsSecretAccessKey()); Assert.assertEquals("awsEndpoint", capturedTaskConfig.getEndpoint()); Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); @@ -1684,8 +1682,6 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); KinesisIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); - Assert.assertEquals("awsKey", capturedTaskConfig.getAwsAccessKeyId()); - Assert.assertEquals("awsSecret", capturedTaskConfig.getAwsSecretAccessKey()); Assert.assertEquals("awsEndpoint", capturedTaskConfig.getEndpoint()); Assert.assertEquals("sequenceName-0", capturedTaskConfig.getBaseSequenceName()); Assert.assertTrue("isUseTransaction", capturedTaskConfig.isUseTransaction()); @@ -3502,8 +3498,6 @@ private KinesisSupervisor getSupervisor( earlyMessageRejectionPeriod, recordsPerFetch, fetchDelayMillis, - "awsKey", - "awsSecret", null, null, false @@ -3549,7 +3543,8 @@ public KinesisIndexTaskClient build( objectMapper, new NoopServiceEmitter(), new DruidMonitorSchedulerConfig(), - rowIngestionMetersFactory + rowIngestionMetersFactory, + null ), rowIngestionMetersFactory ); @@ -3643,8 +3638,6 @@ private KinesisIndexTask createKinesisIndexTask( "awsEndpoint", null, null, - "awsKey", - "awsSecret", null, null, null, @@ -3653,7 +3646,8 @@ private KinesisIndexTask createKinesisIndexTask( Collections.emptyMap(), null, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + null ); } @@ -3682,8 +3676,6 @@ private KinesisSupervisor getSupervisor( earlyMessageRejectionPeriod, null, null, - "awsKey", - "awsSecret", null, null, false @@ -3729,7 +3721,8 @@ public KinesisIndexTaskClient build( objectMapper, new NoopServiceEmitter(), new DruidMonitorSchedulerConfig(), - rowIngestionMetersFactory + rowIngestionMetersFactory, + null ), rowIngestionMetersFactory ); @@ -3790,7 +3783,8 @@ public TestableKinesisSupervisor( taskClientFactory, mapper, spec, - rowIngestionMetersFactory + rowIngestionMetersFactory, + null ); this.spec = spec; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 99851eb9123b..9f03aa6675b3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -412,7 +412,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception // Filter out partitions with END_OF_SHARD markers since these partitions have already been fully read. This // should have been done by the supervisor already so this is defensive. int numPreFilterPartitions = currOffsets.size(); - if (currOffsets.entrySet().removeIf(x -> OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(x.getValue()))) { + if (currOffsets.entrySet().removeIf(x -> isEndOfShard(x.getValue()))) { log.info( "Removed [%d] partitions from assignment which have already been closed", numPreFilterPartitions - currOffsets.size() @@ -531,8 +531,8 @@ public void run() ); } - if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(record.getSequenceNumber())) { - // shard is closed + if (isEndOfShard(record.getSequenceNumber())) { + // shard is closed, applies to Kinesis only currOffsets.put(record.getPartitionId(), record.getSequenceNumber()); } else if (createSequenceNumber(record.getSequenceNumber()).compareTo( createSequenceNumber(endOffsets.get(record.getPartitionId()))) <= 0) { @@ -643,7 +643,7 @@ public void onFailure(@ParametersAreNonnullByDefault Throwable t) } if ((currOffsets.get(record.getPartitionId()).equals(endOffsets.get(record.getPartitionId())) - || currOffsets.get(record.getPartitionId()).equals(OrderedPartitionableRecord.END_OF_SHARD_MARKER)) + || isEndOfShard(currOffsets.get(record.getPartitionId()))) && assignment.remove(record.getStreamPartition())) { log.info("Finished reading stream[%s], partition[%s].", record.getStream(), record.getPartitionId()); recordSupplier.assign(assignment); @@ -832,6 +832,11 @@ public void onFailure(@ParametersAreNonnullByDefault Throwable t) return TaskStatus.success(task.getId()); } + /** + * checks if the input seqNum marks end of shard. Used by Kinesis only + */ + protected abstract boolean isEndOfShard(SequenceOffsetType seqNum); + private void checkPublishAndHandoffFailure() throws ExecutionException, InterruptedException { // Check if any publishFuture failed. @@ -1051,7 +1056,7 @@ private Set> assignPartitions( final Set> assignment = new HashSet<>(); for (Map.Entry entry : currOffsets.entrySet()) { final SequenceOffsetType endOffset = endOffsets.get(entry.getKey()); - if (OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(endOffset) + if (isEndOfShard(endOffset) || SeekableStreamPartitions.NO_END_SEQUENCE_NUMBER.equals(endOffset) || createSequenceNumber(entry.getValue()).compareTo(createSequenceNumber(endOffset)) < 0) { assignment.add(StreamPartition.of(stream, entry.getKey())); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java index 3a6f10b31dce..5e97c1fc7a97 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java @@ -31,14 +31,6 @@ */ public class OrderedPartitionableRecord { - /** - * In Kinesis, when a shard is closed due to shard splitting, a null ShardIterator is returned. - * The EOS marker is placed at the end of the Kinesis Record Supplier buffer, such that when - * an indexing task pulls the record 'EOS', it knows the shard has been closed and should stop - * reading and start publishing - */ - public static final String END_OF_SHARD_MARKER = "EOS"; - private final String stream; private final PartitionIdType partitionId; private final SequenceOffsetType sequenceNumber; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 4ca2d93ee26b..5b488cd8395e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -62,7 +62,6 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; -import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; @@ -1723,7 +1722,7 @@ private void updatePartitionDataFromStream() Set closedPartitions = getOffsetsFromMetadataStorage() .entrySet() .stream() - .filter(x -> OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(x.getValue())) + .filter(x -> isEndOfShard(x.getValue())) .map(Entry::getKey) .collect(Collectors.toSet()); @@ -2278,7 +2277,7 @@ private void createNewTasks() Integer groupId = entry.getKey(); if (taskGroup.startingSequences == null || taskGroup.startingSequences - .values().stream().allMatch(x -> x == null || OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(x))) { + .values().stream().allMatch(x -> x == null || isEndOfShard(x))) { log.debug("Nothing to read in any partition for taskGroup [%d], skipping task creation", groupId); continue; } @@ -2332,7 +2331,7 @@ private ImmutableMap> if (!getNotSetMarker().equals(sequence)) { // if we are given a startingOffset (set by a previous task group which is pending completion) then use it - if (!OrderedPartitionableRecord.END_OF_SHARD_MARKER.equals(sequence)) { + if (!isEndOfShard(sequence)) { builder.put(partition, makeSequenceNumber(sequence, false)); } } else { @@ -2795,4 +2794,9 @@ private boolean checkSequenceAvailability( * @return end of partition sequence offset */ protected abstract SequenceOffsetType getEndOfPartitionMarker(); + + /** + * checks if seqNum marks the end of a Kinesis shard. Used by Kinesis only. + */ + protected abstract boolean isEndOfShard(SequenceOffsetType seqNum); } From d4894c1c998be536f6317de87380711e05ecafaa Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 18 Dec 2018 15:58:10 -0800 Subject: [PATCH 73/87] comments part 2 --- .../extensions-core/kinesis-ingestion.md | 10 ++- .../kafka/KafkaRecordSupplierTest.java | 60 +++++++++++++++-- .../kinesis/KinesisRecordSupplier.java | 67 +++++++++++-------- .../druid/indexing/kinesis/KinesisRegion.java | 4 +- .../kinesis/KinesisRecordSupplierTest.java | 10 +-- .../indexing/kinesis/KinesisRegionTest.java | 54 +++++++++++++++ .../SeekableStreamIndexTaskRunner.java | 4 +- .../common/OrderedPartitionableRecord.java | 19 ++++-- .../seekablestream/common/RecordSupplier.java | 6 +- .../supervisor/SeekableStreamSupervisor.java | 2 +- .../supervisor/TaskReportData.java | 7 +- 11 files changed, 183 insertions(+), 60 deletions(-) create mode 100644 extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRegionTest.java diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md index a8d596f08f38..9dde6ce18739 100644 --- a/docs/content/development/extensions-core/kinesis-ingestion.md +++ b/docs/content/development/extensions-core/kinesis-ingestion.md @@ -203,8 +203,6 @@ For Roaring bitmaps: |`earlyMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps later than this period after the task reached its taskDuration; for example if this is set to `PT1H`, the taskDuration is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps later than *2016-01-01T14:00Z* will be dropped. **Note:** Tasks sometimes run past their task duration, for example, in cases of supervisor failover. Setting earlyMessageRejectionPeriod too low may cause messages to be dropped unexpectedly whenever a task runs past its originally configured task duration.|no (default == none)| |`recordsPerFetch`|Integer|The number of records to request per GetRecords call to Kinesis. See 'Determining Fetch Settings' below.|no (default == 2000)| |`fetchDelayMillis`|Integer|Time in milliseconds to wait between subsequent GetRecords calls to Kinesis. See 'Determining Fetch Settings' below.|no (default == 1000)| -|`awsAccessKeyId`|String|The AWS access key ID to use for Kinesis API requests. If this is not provided, the service will look for credentials set in the environment variables, system properties, in the default profile configuration file, and from the EC2 instance profile provider (in this order).|no| -|`awsSecretAccessKey`|String|The AWS secret access key to use for Kinesis API requests. Only used if `awsAccessKeyId` is also provided.|no| |`awsAssumedRoleArn`|String|The AWS assumed role to use for additional permissions.|no| |`awsExternalId`|String|The AWS external id to use for additional permissions.|no| |`deaggregate`|Boolean|Whether to use the de-aggregate function of the KCL. See below for details.|no| @@ -214,6 +212,14 @@ For Roaring bitmaps: This section gives descriptions of how some supervisor APIs work specifically in Kinesis Indexing Service. For all supervisor APIs, please check [Supervisor APIs](../../operations/api-reference.html#supervisors). +### AWS Authentication +To authenticate with AWS, you must provide your AWS access key and AWS secret key via environment properties, for example: +``` +-Ddruid.kinesis.accessKey=123 -Ddruid.kinesis.secretKey=456 +``` +- The AWS access key ID is used for Kinesis API requests. If this is not provided, the service will look for credentials set in system properties, in the default profile configuration file, and from the EC2 instance profile provider (in this order). +- The AWS secret access key is used for Kinesis API requests. + ### Getting Supervisor Status Report `GET /druid/indexer/v1/supervisor//status` returns a snapshot report of the current state of the tasks managed by the given supervisor. This includes the latest diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java index 0f892540b170..a2c96b204b17 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java @@ -39,6 +39,7 @@ import org.junit.BeforeClass; import org.junit.Test; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -83,6 +84,52 @@ private static List> generateRecords(String topic ); } +// private static List> generateOrderedPartitionableRecords(String topic) +// { +// return ImmutableList.of( +// new OrderedPartitionableRecord<>( +// topic, +// 0, +// 0L, +// ImmutableList.of(JB("2008", "a", "y", "10", "20.0", "1.0")) +// ), +// new OrderedPartitionableRecord<>(topic, 0, 1L, ImmutableList.of(JB("2009", "b", "y", "10", "20.0", "1.0"))), +// new OrderedPartitionableRecord<>(topic, 0, 2L, ImmutableList.of(JB("2010", "c", "y", "10", "20.0", "1.0"))), +// new OrderedPartitionableRecord<>(topic, 0, 3L, ImmutableList.of(JB("2011", "d", "y", "10", "20.0", "1.0"))), +// new OrderedPartitionableRecord<>(topic, 0, 4L, ImmutableList.of(JB("2011", "e", "y", "10", "20.0", "1.0"))), +// new OrderedPartitionableRecord<>( +// topic, +// 0, +// 5L, +// ImmutableList.of(JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")) +// ), +// new OrderedPartitionableRecord<>(topic, 0, 6L, ImmutableList.of(StringUtils.toUtf8("unparseable"))), +// new OrderedPartitionableRecord<>(topic, 0, 7L, ImmutableList.of(StringUtils.toUtf8("unparseable2"))), +// new OrderedPartitionableRecord<>(topic, 0, 8L, ImmutableList.of()), +// new OrderedPartitionableRecord<>(topic, 0, 9L, ImmutableList.of(JB("2013", "f", "y", "10", "20.0", "1.0"))), +// new OrderedPartitionableRecord<>( +// topic, +// 0, +// 10L, +// ImmutableList.of(JB("2049", "f", "y", "notanumber", "20.0", "1.0")) +// ), +// new OrderedPartitionableRecord<>( +// topic, +// 1, +// 0L, +// ImmutableList.of(JB("2049", "f", "y", "10", "notanumber", "1.0")) +// ), +// new OrderedPartitionableRecord<>( +// topic, +// 1, +// 1L, +// ImmutableList.of(JB("2049", "f", "y", "10", "20.0", "notanumber")) +// ), +// new OrderedPartitionableRecord<>(topic, 1, 2L, ImmutableList.of(JB("2012", "g", "y", "10", "20.0", "1.0"))), +// new OrderedPartitionableRecord<>(topic, 1, 3L, ImmutableList.of(JB("2011", "h", "y", "10", "20.0", "1.0"))) +// ); +// } + private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) { try { @@ -107,7 +154,7 @@ private static String getTopicName() return "topic-" + topicPosFix++; } - private Set> createOrderedPartitionableRecords() + private List> createOrderedPartitionableRecords() { Map partitionToOffset = new HashMap<>(); return records.stream().map(r -> { @@ -122,9 +169,9 @@ private Set> createOrderedPartitionabl topic, r.partition(), offset, - r.value() == null ? null : ImmutableList.of(r.value()) + r.value() == null ? null : Collections.singletonList(r.value()) ); - }).collect(Collectors.toSet()); + }).collect(Collectors.toList()); } @BeforeClass @@ -211,7 +258,7 @@ public void testPoll() throws InterruptedException, ExecutionException recordSupplier.assign(partitions); recordSupplier.seekToEarliest(partitions); - Set> initialRecords = createOrderedPartitionableRecords(); + List> initialRecords = new ArrayList<>(createOrderedPartitionableRecords()); List> polledRecords = recordSupplier.poll(poll_timeout_millis); for (int i = 0; polledRecords.size() != initialRecords.size() && i < pollRetry; i++) { @@ -219,7 +266,6 @@ public void testPoll() throws InterruptedException, ExecutionException Thread.sleep(200); } - Assert.assertEquals(partitions, recordSupplier.getAssignment()); Assert.assertEquals(initialRecords.size(), polledRecords.size()); Assert.assertTrue(initialRecords.containsAll(polledRecords)); @@ -267,7 +313,7 @@ public void testPollAfterMoreDataAdded() throws InterruptedException, ExecutionE Thread.sleep(200); } - Set> initialRecords = createOrderedPartitionableRecords(); + List> initialRecords = createOrderedPartitionableRecords(); Assert.assertEquals(records.size(), polledRecords.size()); Assert.assertTrue(initialRecords.containsAll(polledRecords)); @@ -305,7 +351,7 @@ public void testSeek() throws InterruptedException, ExecutionException recordSupplier.seek(partition0, 2L); recordSupplier.seek(partition1, 2L); - Set> initialRecords = createOrderedPartitionableRecords(); + List> initialRecords = createOrderedPartitionableRecords(); List> polledRecords = recordSupplier.poll(poll_timeout_millis); for (int i = 0; polledRecords.size() != 11 && i < pollRetry; i++) { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 97eb7e3d0c2b..6ed7cb99ab57 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -84,8 +84,15 @@ public class KinesisRecordSupplier implements RecordSupplier private class PartitionResource { private final StreamPartition streamPartition; + + // lock for cooradinating startBackground fetch, guards started private final Object startLock = new Object(); + // shardIterator points to the record that will be polled next by recordRunnable + // can be null when shard is closed due to the user shard splitting or changing the number + // of shards in the stream, in which case a 'EOS' marker is used by the KinesisRecordSupplier + // to indicate that this shard has no more records to read + @Nullable private volatile String shardIterator; private volatile boolean started; private volatile boolean stopRequested; @@ -97,7 +104,7 @@ private class PartitionResource this.streamPartition = streamPartition; } - void start() + void startBackgroundFetch() { synchronized (startLock) { if (started) { @@ -117,7 +124,7 @@ void start() } } - public void stop() + void stopBackgroundFetch() { log.info( "Stopping scheduled fetch runnable for stream[%s] partition[%s]", @@ -196,16 +203,14 @@ private Runnable getRecordRunnable() ); - if (log.isTraceEnabled()) { - log.trace( - "Stream[%s] / partition[%s] / sequenceNum[%s] / bufferRemainingCapacity[%d]: %s", - record.getStream(), - record.getPartitionId(), - record.getSequenceNumber(), - records.remainingCapacity(), - record.getData().stream().map(StringUtils::fromUtf8).collect(Collectors.toList()) - ); - } + log.trace( + "Stream[%s] / partition[%s] / sequenceNum[%s] / bufferRemainingCapacity[%d]: %s", + record.getStream(), + record.getPartitionId(), + record.getSequenceNumber(), + records.remainingCapacity(), + record.getData().stream().map(StringUtils::fromUtf8).collect(Collectors.toList()) + ); // If the buffer was full and we weren't able to add the message, grab a new stream iterator starting // from this message and back off for a bit to let the buffer drain before retrying. @@ -328,10 +333,8 @@ public KinesisRecordSupplier( getDataHandle = lookup.unreflect(getDataMethod); } catch (ClassNotFoundException e) { - log.error( - "cannot find class[com.amazonaws.services.kinesis.clientlibrary.types.UserRecord], " - + "note that when using deaggregate=true, you must provide the Kinesis Client Library jar in the classpath"); - throw new RuntimeException(e); + throw new ISE(e, "cannot find class[com.amazonaws.services.kinesis.clientlibrary.types.UserRecord], " + + "note that when using deaggregate=true, you must provide the Kinesis Client Library jar in the classpath"); } catch (Exception e) { throw new RuntimeException(e); @@ -401,7 +404,7 @@ public void start() { checkIfClosed(); if (checkPartitionsStarted) { - partitionResources.values().forEach(PartitionResource::start); + partitionResources.values().forEach(PartitionResource::startBackgroundFetch); checkPartitionsStarted = false; } } @@ -423,14 +426,14 @@ public void assign(Set> collection) Map.Entry, PartitionResource> entry = i.next(); if (!collection.contains(entry.getKey())) { i.remove(); - entry.getValue().stop(); + entry.getValue().stopBackgroundFetch(); } } } @Override - public void seek(StreamPartition partition, String sequenceNumber) + public void seek(StreamPartition partition, String sequenceNumber) throws InterruptedException { checkIfClosed(); filterBufferAndResetFetchRunnable(ImmutableSet.of(partition)); @@ -438,7 +441,7 @@ public void seek(StreamPartition partition, String sequenceNumber) } @Override - public void seekToEarliest(Set> partitions) + public void seekToEarliest(Set> partitions) throws InterruptedException { checkIfClosed(); filterBufferAndResetFetchRunnable(partitions); @@ -446,7 +449,7 @@ public void seekToEarliest(Set> partitions) } @Override - public void seekToLatest(Set> partitions) + public void seekToLatest(Set> partitions) throws InterruptedException { checkIfClosed(); filterBufferAndResetFetchRunnable(partitions); @@ -465,16 +468,19 @@ public List> poll(long timeout) { checkIfClosed(); if (checkPartitionsStarted) { - partitionResources.values().forEach(PartitionResource::start); + partitionResources.values().forEach(PartitionResource::startBackgroundFetch); checkPartitionsStarted = false; } try { - List> polledRecords = new ArrayList<>(); + int expectedSize = Math.min(Math.max(records.size(), 1), maxRecordsPerPoll); + + List> polledRecords = new ArrayList<>(expectedSize); + Queues.drain( records, polledRecords, - Math.min(Math.max(records.size(), 1), maxRecordsPerPoll), + expectedSize, timeout, TimeUnit.MILLISECONDS ); @@ -492,6 +498,7 @@ public List> poll(long timeout) } + @Nullable @Override public String getLatestSequenceNumber(StreamPartition partition) { @@ -499,6 +506,7 @@ public String getLatestSequenceNumber(StreamPartition partition) return getSequenceNumberInternal(partition, ShardIteratorType.LATEST); } + @Nullable @Override public String getEarliestSequenceNumber(StreamPartition partition) { @@ -541,7 +549,7 @@ public void close() } } catch (InterruptedException e) { - log.info(e, "InterruptedException while shutting down"); + log.warn(e, "InterruptedException while shutting down"); } this.closed = true; @@ -570,7 +578,7 @@ private void seekInternal(StreamPartition partition, String sequenceNumb checkPartitionsStarted = true; } - private void filterBufferAndResetFetchRunnable(Set> partitions) + private void filterBufferAndResetFetchRunnable(Set> partitions) throws InterruptedException { scheduledExec.shutdown(); @@ -580,7 +588,8 @@ private void filterBufferAndResetFetchRunnable(Set> part } } catch (InterruptedException e) { - log.info(e, "InterruptedException while shutting down"); + log.warn(e, "InterruptedException while shutting down"); + throw e; } scheduledExec = Executors.newScheduledThreadPool( @@ -602,6 +611,7 @@ private void filterBufferAndResetFetchRunnable(Set> part checkPartitionsStarted = true; } + @Nullable private String getSequenceNumberInternal(StreamPartition partition, ShardIteratorType iteratorEnum) { @@ -614,12 +624,13 @@ private String getSequenceNumberInternal(StreamPartition partition, Shar ).getShardIterator(); } catch (ResourceNotFoundException e) { - log.warn("Caught ResourceNotFoundException: %s", e.getMessage()); + log.warn("Caught ResourceNotFoundException: [%s]", e); } return getSequenceNumberInternal(partition, shardIterator); } + @Nullable private String getSequenceNumberInternal(StreamPartition partition, String shardIterator) { long timeoutMillis = System.currentTimeMillis() + fetchSequenceNumberTimeout; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java index 867294912592..0c68bd8a2d20 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java @@ -57,7 +57,7 @@ public static KinesisRegion fromString(String value) .stream() .filter(x -> x.toString().equals(value)) .findFirst() - .orElseThrow(() -> new IAE("Region must be one of: %s", getNames())); + .orElseThrow(() -> new IAE("Invalid region %s, region must be one of: %s", value, getNames())); } private static List getNames() @@ -70,8 +70,8 @@ public String getEndpoint() return StringUtils.format("kinesis.%s.amazonaws.com%s", toString(), toString().startsWith("cn-") ? ".cn" : ""); } - @Override @JsonValue + @Override public String toString() { return StringUtils.toLowerCase(name()).replace('_', '-'); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index f20ac86f40f3..166678cb7eb4 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -347,7 +347,7 @@ public void testSeek() recordSupplier.seek(shard0Partition, shard0Records.get(1).getSequenceNumber()); recordSupplier.start(); - while (recordSupplier.bufferSize() < 9) { + for (int i = 0; i < 10 && recordSupplier.bufferSize() < 9; i++) { Thread.sleep(100); } @@ -413,7 +413,7 @@ public void testSeekToLatest() recordSupplier.seekToLatest(partitions); recordSupplier.start(); - while (recordSupplier.bufferSize() < 2) { + for (int i = 0; i < 10 && recordSupplier.bufferSize() < 2; i++) { Thread.sleep(100); } Assert.assertEquals(Collections.emptyList(), cleanRecords(recordSupplier.poll(poll_timeout_millis))); @@ -422,7 +422,7 @@ public void testSeekToLatest() } @Test(expected = ISE.class) - public void testSeekUnassigned() + public void testSeekUnassigned() throws InterruptedException { StreamPartition shard0 = StreamPartition.of(stream, shardId0); StreamPartition shard1 = StreamPartition.of(stream, shardId1); @@ -500,7 +500,7 @@ public void testPollAfterSeek() recordSupplier.seek(StreamPartition.of(stream, shardId1), "5"); recordSupplier.start(); - while (recordSupplier.bufferSize() < 6) { + for (int i = 0; i < 10 && recordSupplier.bufferSize() < 6; i++) { Thread.sleep(100); } @@ -582,7 +582,7 @@ public void testPollDeaggregate() throws InterruptedException recordSupplier.seekToEarliest(partitions); recordSupplier.start(); - while (recordSupplier.bufferSize() < 12) { + for (int i = 0; i < 10 && recordSupplier.bufferSize() < 12; i++) { Thread.sleep(100); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRegionTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRegionTest.java new file mode 100644 index 000000000000..dec619fdc4dc --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRegionTest.java @@ -0,0 +1,54 @@ +package org.apache.druid.indexing.kinesis; + +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +public class KinesisRegionTest +{ + private ObjectMapper mapper; + + @Before + public void setupTest() + { + mapper = new DefaultObjectMapper(); + } + + @Test + public void testSerde() throws IOException + { + KinesisRegion kinesisRegionUs1 = KinesisRegion.US_EAST_1; + KinesisRegion kinesisRegionAp1 = KinesisRegion.AP_NORTHEAST_1; + + Assert.assertEquals("\"us-east-1\"", mapper.writeValueAsString(kinesisRegionUs1)); + Assert.assertEquals("\"ap-northeast-1\"", mapper.writeValueAsString(kinesisRegionAp1)); + + KinesisRegion kinesisRegion = mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + "\"us-east-1\"", + KinesisRegion.class + ) + ), + KinesisRegion.class + ); + + Assert.assertEquals(kinesisRegion, KinesisRegion.US_EAST_1); + } + + @Test(expected = JsonMappingException.class) + public void testBadSerde() throws IOException + { + mapper.readValue( + "\"us-east-10\"", + KinesisRegion.class + ); + } + +} + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 9f03aa6675b3..b26f1947d2ab 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -175,7 +175,7 @@ protected enum Type private final Condition hasPaused = pauseLock.newCondition(); private final Condition shouldResume = pauseLock.newCondition(); - public final AtomicBoolean stopRequested = new AtomicBoolean(false); + protected final AtomicBoolean stopRequested = new AtomicBoolean(false); private final AtomicBoolean publishOnStop = new AtomicBoolean(false); // [statusLock] is used to synchronize the Jetty thread calling stopGracefully() with the main run thread. It prevents @@ -1080,7 +1080,7 @@ private Set> assignPartitions( private void seekToStartingSequence( RecordSupplier recordSupplier, Set> partitions - ) + ) throws InterruptedException { for (final StreamPartition partition : partitions) { final SequenceOffsetType sequence = currOffsets.get(partition.getPartitionId()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java index 5e97c1fc7a97..8f270915edaa 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java @@ -19,6 +19,11 @@ package org.apache.druid.indexing.seekablestream.common; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import javax.validation.constraints.NotNull; +import java.util.Arrays; import java.util.List; import java.util.Objects; @@ -43,10 +48,13 @@ public OrderedPartitionableRecord( List data ) { + Preconditions.checkNotNull(stream, "stream"); + Preconditions.checkNotNull(partitionId, "partitionId"); + Preconditions.checkNotNull(sequenceNumber, "sequenceNumber"); this.stream = stream; this.partitionId = partitionId; this.sequenceNumber = sequenceNumber; - this.data = data; + this.data = data == null ? ImmutableList.of() : data; } public String getStream() @@ -64,6 +72,7 @@ public SequenceOffsetType getSequenceNumber() return sequenceNumber; } + @NotNull public List getData() { return data; @@ -74,6 +83,7 @@ public StreamPartition getStreamPartition() return StreamPartition.of(stream, partitionId); } + @Override public boolean equals(Object o) { @@ -86,13 +96,14 @@ public boolean equals(Object o) OrderedPartitionableRecord that = (OrderedPartitionableRecord) o; return Objects.equals(stream, that.stream) && Objects.equals(partitionId, that.partitionId) && - Objects.equals(sequenceNumber, that.sequenceNumber); - + Objects.equals(sequenceNumber, that.sequenceNumber) && + data.size() == that.data.size() && + data.stream().allMatch(d -> that.data.stream().anyMatch(d2 -> Arrays.equals(d, d2))); } @Override public int hashCode() { - return Objects.hash(stream, partitionId, sequenceNumber); + return Objects.hash(stream, partitionId, sequenceNumber, data); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java index 77a2c073bc0e..d9e599da0c80 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java @@ -53,21 +53,21 @@ public interface RecordSupplier extends Clo * @param partition partition to seek * @param sequenceNumber sequence number to seek to */ - void seek(StreamPartition partition, SequenceOffsetType sequenceNumber); + void seek(StreamPartition partition, SequenceOffsetType sequenceNumber) throws InterruptedException; /** * seek a set of partitions to the earliest record position available in the stream * * @param partitions partitions to seek */ - void seekToEarliest(Set> partitions); + void seekToEarliest(Set> partitions) throws InterruptedException; /** * seek a set of partitions to the latest/newest record position available in the stream * * @param partitions partitions to seek */ - void seekToLatest(Set> partitions); + void seekToLatest(Set> partitions) throws InterruptedException; /** * get the current assignment diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 5b488cd8395e..5d3b74b394f7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -2529,7 +2529,7 @@ private void updateCurrentOffsets() throws InterruptedException, ExecutionExcept Futures.successfulAsList(futures).get(futureTimeoutInSeconds, TimeUnit.SECONDS); } - protected void updateLatestOffsetsFromStream() + private void updateLatestOffsetsFromStream() throws InterruptedException { synchronized (recordSupplierLock) { Set partitionIds = null; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java index a7864c7672de..b265df1856b3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java @@ -33,7 +33,7 @@ public class TaskReportData private final DateTime startTime; private final Long remainingSeconds; private final TaskType type; - private Map currentOffsets; + private final Map currentOffsets; private final Map lag; public TaskReportData( @@ -100,11 +100,6 @@ public Map getLag() return lag; } - public void setCurrentSequenceNumbers(Map currentOffsets) - { - this.currentOffsets = currentOffsets; - } - @Override public String toString() { From 1516da7a0ce0c7488529d9feacf21f943312c95f Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Tue, 18 Dec 2018 17:00:39 -0800 Subject: [PATCH 74/87] comments part 3 --- ...ementalPublishingKafkaIndexTaskRunner.java | 18 ++++--- .../druid/indexing/kafka/KafkaIndexTask.java | 16 +++--- .../kafka/LegacyKafkaIndexTaskRunner.java | 20 ++++--- .../kafka/KafkaRecordSupplierTest.java | 46 ---------------- .../indexing/kinesis/KinesisIndexTask.java | 12 ++++- .../kinesis/KinesisIndexTaskRunner.java | 18 ++++--- .../kinesis/KinesisRecordSupplier.java | 17 +++--- .../kinesis/KinesisIndexTaskTest.java | 1 - .../indexing/kinesis/KinesisRegionTest.java | 19 +++++++ .../SeekableStreamIndexTask.java | 22 ++++---- .../SeekableStreamIndexTaskRunner.java | 52 +++++++++++-------- 11 files changed, 126 insertions(+), 115 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index c9487f25a348..aee1972b056e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -196,12 +196,6 @@ protected OrderedSequenceNumber createSequenceNumber(Long sequenceNumber) return KafkaSequenceNumber.of(sequenceNumber); } - @Override - protected Type getRunnerType() - { - return Type.KAFKA; - } - @Override protected void possiblyResetDataSourceMetadata( TaskToolbox toolbox, @@ -239,6 +233,18 @@ protected void possiblyResetDataSourceMetadata( } } + @Override + protected boolean isEndSequenceOffsetsExclusive() + { + return true; + } + + @Override + protected boolean isStartingSequenceOffsetsExclusive() + { + return false; + } + @Override protected boolean isEndOfShard(Long seqNum) { diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 81b576e3de1f..38608c288199 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -29,7 +29,6 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; -import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; @@ -47,8 +46,7 @@ public class KafkaIndexTask extends SeekableStreamIndexTask { - - private static final EmittingLogger log = new EmittingLogger(KafkaIndexTask.class); + private static final String TYPE = "index_kafka"; static final long POLL_TIMEOUT_MILLIS = TimeUnit.MILLISECONDS.toMillis(100); private final KafkaIndexTaskIOConfig ioConfig; @@ -72,7 +70,7 @@ public KafkaIndexTask( ) { super( - id, + id == null ? getFormatedId(dataSchema.getDataSource(), TYPE) : id, taskResource, dataSchema, tuningConfig, @@ -81,7 +79,7 @@ public KafkaIndexTask( chatHandlerProvider, authorizerMapper, rowIngestionMetersFactory, - "index_kafka" + getFormattedGroupId(dataSchema.getDataSource(), TYPE) ); this.configMapper = configMapper; this.ioConfig = ioConfig; @@ -105,7 +103,7 @@ KafkaConsumer newConsumer() KafkaRecordSupplier.addConsumerPropertiesFromConfig( props, configMapper, - ((KafkaIndexTaskIOConfig) ioConfig).getConsumerProperties() + ioConfig.getConsumerProperties() ); props.setProperty("enable.auto.commit", "false"); @@ -198,4 +196,10 @@ public KafkaIndexTaskIOConfig getIOConfig() { return (KafkaIndexTaskIOConfig) super.getIOConfig(); } + + @Override + public String getType() + { + return TYPE; + } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index ced7a4b723e1..53320c664da5 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -81,6 +81,7 @@ import org.apache.kafka.common.TopicPartition; import org.joda.time.DateTime; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; @@ -630,6 +631,7 @@ protected boolean isEndOfShard(Long seqNum) return false; } + @Nonnull @Override protected List> getRecords( RecordSupplier recordSupplier, TaskToolbox toolbox @@ -714,6 +716,18 @@ protected void possiblyResetDataSourceMetadata( throw new UnsupportedOperationException(); } + @Override + protected boolean isEndSequenceOffsetsExclusive() + { + return false; + } + + @Override + protected boolean isStartingSequenceOffsetsExclusive() + { + return false; + } + @Override protected SeekableStreamPartitions deserializeSeekableStreamPartitionsFromMetadata( @@ -1216,12 +1230,6 @@ protected OrderedSequenceNumber createSequenceNumber(Long sequenceNumber) throw new UnsupportedOperationException(); } - @Override - protected Type getRunnerType() - { - return Type.KAFKA; - } - @Override @GET @Path("/time/start") diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java index a2c96b204b17..2f445aa99431 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierTest.java @@ -84,52 +84,6 @@ private static List> generateRecords(String topic ); } -// private static List> generateOrderedPartitionableRecords(String topic) -// { -// return ImmutableList.of( -// new OrderedPartitionableRecord<>( -// topic, -// 0, -// 0L, -// ImmutableList.of(JB("2008", "a", "y", "10", "20.0", "1.0")) -// ), -// new OrderedPartitionableRecord<>(topic, 0, 1L, ImmutableList.of(JB("2009", "b", "y", "10", "20.0", "1.0"))), -// new OrderedPartitionableRecord<>(topic, 0, 2L, ImmutableList.of(JB("2010", "c", "y", "10", "20.0", "1.0"))), -// new OrderedPartitionableRecord<>(topic, 0, 3L, ImmutableList.of(JB("2011", "d", "y", "10", "20.0", "1.0"))), -// new OrderedPartitionableRecord<>(topic, 0, 4L, ImmutableList.of(JB("2011", "e", "y", "10", "20.0", "1.0"))), -// new OrderedPartitionableRecord<>( -// topic, -// 0, -// 5L, -// ImmutableList.of(JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")) -// ), -// new OrderedPartitionableRecord<>(topic, 0, 6L, ImmutableList.of(StringUtils.toUtf8("unparseable"))), -// new OrderedPartitionableRecord<>(topic, 0, 7L, ImmutableList.of(StringUtils.toUtf8("unparseable2"))), -// new OrderedPartitionableRecord<>(topic, 0, 8L, ImmutableList.of()), -// new OrderedPartitionableRecord<>(topic, 0, 9L, ImmutableList.of(JB("2013", "f", "y", "10", "20.0", "1.0"))), -// new OrderedPartitionableRecord<>( -// topic, -// 0, -// 10L, -// ImmutableList.of(JB("2049", "f", "y", "notanumber", "20.0", "1.0")) -// ), -// new OrderedPartitionableRecord<>( -// topic, -// 1, -// 0L, -// ImmutableList.of(JB("2049", "f", "y", "10", "notanumber", "1.0")) -// ), -// new OrderedPartitionableRecord<>( -// topic, -// 1, -// 1L, -// ImmutableList.of(JB("2049", "f", "y", "10", "20.0", "notanumber")) -// ), -// new OrderedPartitionableRecord<>(topic, 1, 2L, ImmutableList.of(JB("2012", "g", "y", "10", "20.0", "1.0"))), -// new OrderedPartitionableRecord<>(topic, 1, 3L, ImmutableList.of(JB("2011", "h", "y", "10", "20.0", "1.0"))) -// ); -// } - private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) { try { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 0bd9826ea78e..2a86aa448d62 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -35,6 +35,8 @@ public class KinesisIndexTask extends SeekableStreamIndexTask { + private static final String TYPE = "index_kinesis"; + private final AWSCredentialsConfig awsCredentialsConfig; @JsonCreator @@ -52,7 +54,7 @@ public KinesisIndexTask( ) { super( - id, + id == null ? getFormatedId(dataSchema.getDataSource(), TYPE) : id, taskResource, dataSchema, tuningConfig, @@ -61,7 +63,7 @@ public KinesisIndexTask( chatHandlerProvider, authorizerMapper, rowIngestionMetersFactory, - "index_kinesis" + getFormattedGroupId(dataSchema.getDataSource(), TYPE) ); this.awsCredentialsConfig = awsCredentialsConfig; } @@ -116,4 +118,10 @@ public KinesisIndexTaskIOConfig getIOConfig() { return (KinesisIndexTaskIOConfig) super.getIOConfig(); } + + @Override + public String getType() + { + return TYPE; + } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index 7f218baea418..3e7e5e7aeddd 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -119,12 +119,6 @@ protected OrderedSequenceNumber createSequenceNumber(String sequenceNumb return KinesisSequenceNumber.of(sequenceNumber); } - @Override - protected Type getRunnerType() - { - return Type.KINESIS; - } - @Override protected void possiblyResetDataSourceMetadata( TaskToolbox toolbox, @@ -164,6 +158,18 @@ protected void possiblyResetDataSourceMetadata( } } + @Override + protected boolean isEndSequenceOffsetsExclusive() + { + return false; + } + + @Override + protected boolean isStartingSequenceOffsetsExclusive() + { + return true; + } + @Override protected boolean isEndOfShard(String seqNum) { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 6ed7cb99ab57..82401345c744 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -241,8 +241,7 @@ private Runnable getRecordRunnable() rescheduleRunnable(retryMs); } catch (Throwable e) { - log.error(e, "getRecordRunnable exception, retrying in [%,dms]", EXCEPTION_RETRY_DELAY_MS); - rescheduleRunnable(EXCEPTION_RETRY_DELAY_MS); + log.error(e, "getRecordRunnable exception"); } }; @@ -255,8 +254,9 @@ private void rescheduleRunnable(long delayMillis) scheduledExec.schedule(getRecordRunnable(), delayMillis, TimeUnit.MILLISECONDS); } catch (RejectedExecutionException e) { - log.info( - "Rejecting fetch records runnable submission, worker for partition[%s] is not in a valid state", + log.warn( + "Caught RejectedExecutionException, KinesisRecordSupplier for partition[%s] has likely temporarily shutdown the ExecutorService." + + "This is expected behavior after calling seek(), seekToEarliest() and seekToLatest()", streamPartition.getPartitionId() ); @@ -316,11 +316,10 @@ public KinesisRecordSupplier( this.fetchThreads = fetchThreads; this.recordBufferSize = recordBufferSize; - /** - * the deaggregate function is implemented by the amazon-kinesis-client, whose license is not compatible with Apache. - * The work around here is to use reflection to find the deaggregate function in the classpath. See details on the - * docs page for more information on how to use deaggregation - */ + + // the deaggregate function is implemented by the amazon-kinesis-client, whose license is not compatible with Apache. + // The work around here is to use reflection to find the deaggregate function in the classpath. See details on the + // docs page for more information on how to use deaggregation if (deaggregate) { try { Class kclUserRecordclass = Class.forName("com.amazonaws.services.kinesis.clientlibrary.types.UserRecord"); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 98278e16b765..870b74eb879e 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -349,7 +349,6 @@ public static void tearDownClass() throws Exception emitter.close(); } - @Test(timeout = 120_000L) public void testRunAfterDataInserted() throws Exception { diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRegionTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRegionTest.java index dec619fdc4dc..350c4c087415 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRegionTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRegionTest.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.indexing.kinesis; import com.fasterxml.jackson.databind.JsonMappingException; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 3421f08289f8..ef96fcd00c2d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -75,7 +75,6 @@ public abstract class SeekableStreamIndexTask ioConfig; protected final Optional chatHandlerProvider; - protected final String type; protected final Map context; protected final AuthorizerMapper authorizerMapper; protected final RowIngestionMetersFactory rowIngestionMetersFactory; @@ -92,17 +91,16 @@ public SeekableStreamIndexTask( @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, - String type + String groupId ) { super( - id == null ? makeTaskId(dataSchema.getDataSource(), RANDOM.nextInt(), type) : id, - StringUtils.format("%s_%s", type, dataSchema.getDataSource()), + id, + groupId, taskResource, dataSchema.getDataSource(), context ); - this.type = type; this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); this.parser = Preconditions.checkNotNull((InputRowParser) dataSchema.getParser(), "parser"); this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); @@ -129,17 +127,21 @@ private static String makeTaskId(String dataSource, int randomBits, String type) return Joiner.on("_").join(type, dataSource, suffix); } + protected static String getFormatedId(String dataSource, String type) + { + return makeTaskId(dataSource, RANDOM.nextInt(), type); + } - @Override - public int getPriority() + protected static String getFormattedGroupId(String dataSource, String type) { - return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY); + return StringUtils.format("%s_%s", type, dataSource); } + @Override - public String getType() + public int getPriority() { - return type; + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index b26f1947d2ab..c3d182e07810 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -141,13 +141,6 @@ public enum Status PUBLISHING } - protected enum Type - { - KAFKA, - KINESIS - } - - private final EmittingLogger log = new EmittingLogger(this.getClass()); private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; private static final String METADATA_PUBLISH_PARTITIONS = "publishPartitions"; @@ -511,8 +504,9 @@ public void run() for (OrderedPartitionableRecord record : records) { // for Kafka, the end offsets are exclusive, so skip it - if (getRunnerType() == Type.KAFKA && createSequenceNumber(record.getSequenceNumber()).compareTo( - createSequenceNumber(endOffsets.get(record.getPartitionId()))) == 0) { + if (isEndSequenceOffsetsExclusive() && + createSequenceNumber(record.getSequenceNumber()).compareTo( + createSequenceNumber(endOffsets.get(record.getPartitionId()))) == 0) { continue; } @@ -1763,15 +1757,21 @@ boolean canHandle(OrderedPartitionableRecord= (getExclusiveStartPartitions().contains(record.getPartitionId()) ? 1 : 0) - && recordOffset.compareTo(partitionEndOffset) <= 0; + boolean ret; + if (isStartingSequenceOffsetsExclusive()) { + ret = recordOffset.compareTo(partitionStartOffset) + >= (getExclusiveStartPartitions().contains(record.getPartitionId()) ? 1 : 0); } else { - return recordOffset.compareTo(partitionStartOffset) >= 0 - && recordOffset.compareTo(partitionEndOffset) < 0; + ret = recordOffset.compareTo(partitionStartOffset) >= 0; + } + if (isEndSequenceOffsetsExclusive()) { + ret &= recordOffset.compareTo(partitionEndOffset) < 0; + } else { + ret &= recordOffset.compareTo(partitionEndOffset) <= 0; } + + return ret; } finally { lock.unlock(); @@ -1926,7 +1926,7 @@ private boolean verifyInitialRecordAndSkipExclusivePartition( } // check exclusive starting sequence - if (getRunnerType() == Type.KINESIS && exclusiveStartingPartitions.contains(record.getPartitionId())) { + if (isStartingSequenceOffsetsExclusive() && exclusiveStartingPartitions.contains(record.getPartitionId())) { log.info("Skipping starting sequenceNumber for partition [%s] marked exclusive", record.getPartitionId()); return false; @@ -2014,13 +2014,6 @@ protected abstract SeekableStreamDataSourceMetadata createSequenceNumber(SequenceOffsetType sequenceNumber); - /** - * get the type{Kafka, Kinesis} of the TaskRunner, used mostly for Kafka/Kinesis specifc logic - * - * @return Type - */ - protected abstract Type getRunnerType(); - /** * check if the sequence offsets stored in currOffsets are still valid sequence offsets compared to * earliest sequence offsets fetched from stream @@ -2036,4 +2029,17 @@ protected abstract void possiblyResetDataSourceMetadata( Set> assignment, Map currOffsets ); + + /** + * In Kafka, the endOffsets are exclusive, so skip it. + * In Kinesis the endOffsets are inclusive + */ + protected abstract boolean isEndSequenceOffsetsExclusive(); + + /** + * In Kafka, the startingOffsets are inclusive. + * In Kinesis, the startingOffsets are exclusive, except for the first + * partition we read from stream + */ + protected abstract boolean isStartingSequenceOffsetsExclusive(); } From 7cd756c0b4fad42f31bfc15cbb161edb8196ba48 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 19 Dec 2018 10:52:17 -0800 Subject: [PATCH 75/87] merge #6754 --- .../kafka/KafkaIndexTaskClientTest.java | 221 +++++++++--------- .../kafka/supervisor/KafkaSupervisorTest.java | 5 +- .../kinesis/KinesisIndexTaskClientTest.java | 220 ++++++++--------- .../supervisor/KinesisSupervisorTest.java | 5 +- .../SeekableStreamIndexTaskClient.java | 65 +++--- .../supervisor/SeekableStreamSupervisor.java | 34 ++- 6 files changed, 298 insertions(+), 252 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java index b59ddc58a94d..28cc0bbe9d15 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -66,6 +66,8 @@ import java.util.List; import java.util.Map; +import static org.easymock.EasyMock.expect; + @RunWith(Parameterized.class) public class KafkaIndexTaskClientTest extends EasyMockSupport { @@ -112,20 +114,20 @@ public void setUp() headers = createMock(HttpHeaders.class); client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider); - EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)) - .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) - .anyTimes(); - EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)) - .andReturn(Optional.of(TaskStatus.running(TEST_ID))) - .anyTimes(); + expect(taskInfoProvider.getTaskLocation(TEST_ID)) + .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + expect(taskInfoProvider.getTaskStatus(TEST_ID)) + .andReturn(Optional.of(TaskStatus.running(TEST_ID))) + .anyTimes(); for (String testId : TEST_IDS) { - EasyMock.expect(taskInfoProvider.getTaskLocation(testId)) - .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) - .anyTimes(); - EasyMock.expect(taskInfoProvider.getTaskStatus(testId)) - .andReturn(Optional.of(TaskStatus.running(testId))) - .anyTimes(); + expect(taskInfoProvider.getTaskLocation(testId)) + .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + expect(taskInfoProvider.getTaskStatus(testId)) + .andReturn(Optional.of(TaskStatus.running(testId))) + .anyTimes(); } } @@ -139,10 +141,10 @@ public void tearDown() public void testNoTaskLocation() throws IOException { EasyMock.reset(taskInfoProvider); - EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); - EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)) - .andReturn(Optional.of(TaskStatus.running(TEST_ID))) - .anyTimes(); + expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); + expect(taskInfoProvider.getTaskStatus(TEST_ID)) + .andReturn(Optional.of(TaskStatus.running(TEST_ID))) + .anyTimes(); replayAll(); Assert.assertEquals(false, client.stop(TEST_ID, true)); @@ -166,12 +168,12 @@ public void testTaskNotRunnableException() expectedException.expectMessage("Aborting request because task [test-id] is not runnable"); EasyMock.reset(taskInfoProvider); - EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)) - .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) - .anyTimes(); - EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)) - .andReturn(Optional.of(TaskStatus.failure(TEST_ID))) - .anyTimes(); + expect(taskInfoProvider.getTaskLocation(TEST_ID)) + .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + expect(taskInfoProvider.getTaskStatus(TEST_ID)) + .andReturn(Optional.of(TaskStatus.failure(TEST_ID))) + .anyTimes(); replayAll(); client.getCurrentOffsets(TEST_ID, true); @@ -184,8 +186,8 @@ public void testInternalServerError() expectedException.expect(RuntimeException.class); expectedException.expectMessage("org.apache.druid.java.util.common.IOE: Received status [500]"); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2); - EasyMock.expect( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2); + expect( httpClient.go( EasyMock.anyObject(Request.class), EasyMock.anyObject(FullResponseHandler.class), @@ -206,9 +208,9 @@ public void testBadRequest() expectedException.expect(IAE.class); expectedException.expectMessage("Received 400 Bad Request with body:"); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2); - EasyMock.expect(responseHolder.getContent()).andReturn(""); - EasyMock.expect( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2); + expect(responseHolder.getContent()).andReturn(""); + expect( httpClient.go( EasyMock.anyObject(Request.class), EasyMock.anyObject(FullResponseHandler.class), @@ -226,14 +228,14 @@ public void testBadRequest() @Test public void testTaskLocationMismatch() { - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) - .andReturn(HttpResponseStatus.OK); - EasyMock.expect(responseHolder.getResponse()).andReturn(response); - EasyMock.expect(responseHolder.getContent()).andReturn("") - .andReturn("{}"); - EasyMock.expect(response.headers()).andReturn(headers); - EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id"); - EasyMock.expect( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) + .andReturn(HttpResponseStatus.OK); + expect(responseHolder.getResponse()).andReturn(response); + expect(responseHolder.getContent()).andReturn("") + .andReturn("{}"); + expect(response.headers()).andReturn(headers); + expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id"); + expect( httpClient.go( EasyMock.anyObject(Request.class), EasyMock.anyObject(FullResponseHandler.class), @@ -254,9 +256,9 @@ public void testTaskLocationMismatch() public void testGetCurrentOffsets() throws Exception { Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -287,15 +289,15 @@ public void testGetCurrentOffsetsWithRetry() throws Exception client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 3); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) - .andReturn(HttpResponseStatus.OK).times(1); - EasyMock.expect(responseHolder.getContent()).andReturn("").times(2) - .andReturn("{\"0\":1, \"1\":10}"); - EasyMock.expect(responseHolder.getResponse()).andReturn(response).times(2); - EasyMock.expect(response.headers()).andReturn(headers).times(2); - EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2); - - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) + .andReturn(HttpResponseStatus.OK).times(1); + expect(responseHolder.getContent()).andReturn("").times(2) + .andReturn("{\"0\":1, \"1\":10}"); + expect(responseHolder.getResponse()).andReturn(response).times(2); + expect(response.headers()).andReturn(headers).times(2); + expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2); + + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -331,13 +333,13 @@ public void testGetCurrentOffsetsWithExhaustedRetries() client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); - EasyMock.expect(responseHolder.getContent()).andReturn("").anyTimes(); - EasyMock.expect(responseHolder.getResponse()).andReturn(response).anyTimes(); - EasyMock.expect(response.headers()).andReturn(headers).anyTimes(); - EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).anyTimes(); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); + expect(responseHolder.getContent()).andReturn("").anyTimes(); + expect(responseHolder.getResponse()).andReturn(response).anyTimes(); + expect(response.headers()).andReturn(headers).anyTimes(); + expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).anyTimes(); - EasyMock.expect( + expect( httpClient.go( EasyMock.anyObject(Request.class), EasyMock.anyObject(FullResponseHandler.class), @@ -354,9 +356,9 @@ public void testGetCurrentOffsetsWithExhaustedRetries() public void testGetEndOffsets() throws Exception { Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -388,13 +390,13 @@ public void testGetStartTime() throws Exception DateTime now = DateTimes.nowUtc(); Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) - .andReturn(HttpResponseStatus.OK); - EasyMock.expect(responseHolder.getResponse()).andReturn(response); - EasyMock.expect(response.headers()).andReturn(headers); - EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(null); - EasyMock.expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) + .andReturn(HttpResponseStatus.OK); + expect(responseHolder.getResponse()).andReturn(response); + expect(response.headers()).andReturn(headers); + expect(headers.get("X-Druid-Task-Id")).andReturn(null); + expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -423,9 +425,9 @@ public void testGetStatus() throws Exception Status status = Status.READING; Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - EasyMock.expect(responseHolder.getContent()).andReturn(StringUtils.format("\"%s\"", status.toString())).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + expect(responseHolder.getContent()).andReturn(StringUtils.format("\"%s\"", status.toString())).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -452,9 +454,9 @@ public void testGetStatus() throws Exception public void testPause() throws Exception { Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); - EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); + expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -485,25 +487,26 @@ public void testPauseWithSubsequentGetOffsets() throws Exception Capture captured = Capture.newInstance(); Capture captured2 = Capture.newInstance(); Capture captured3 = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.ACCEPTED).times(2) - .andReturn(HttpResponseStatus.OK).times(2); - EasyMock.expect(responseHolder.getContent()).andReturn("\"PAUSED\"") - .andReturn("{\"0\":1, \"1\":10}").anyTimes(); - EasyMock.expect(httpClient.go( + // one time in IndexTaskClient.submitRequest() and another in KafkaIndexTaskClient.pause() + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.ACCEPTED).times(2) + .andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("\"PAUSED\"").times(2) + .andReturn("{\"0\":1, \"1\":10}").anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) )).andReturn( Futures.immediateFuture(responseHolder) ); - EasyMock.expect(httpClient.go( + expect(httpClient.go( EasyMock.capture(captured2), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) )).andReturn( Futures.immediateFuture(responseHolder) ); - EasyMock.expect(httpClient.go( + expect(httpClient.go( EasyMock.capture(captured3), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -547,8 +550,8 @@ public void testPauseWithSubsequentGetOffsets() throws Exception public void testResume() throws Exception { Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -575,8 +578,8 @@ public void testSetEndOffsets() throws Exception Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -604,8 +607,8 @@ public void testSetEndOffsetsAndResume() throws Exception Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -631,8 +634,8 @@ public void testSetEndOffsetsAndResume() throws Exception public void testStop() throws Exception { Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -657,8 +660,8 @@ public void testStop() throws Exception public void testStopAndPublish() throws Exception { Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -684,8 +687,8 @@ public void testStopAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -720,8 +723,8 @@ public void testResumeAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -756,9 +759,9 @@ public void testPauseAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -793,9 +796,9 @@ public void testGetStatusAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -831,9 +834,9 @@ public void testGetStartTimeAsync() throws Exception final DateTime now = DateTimes.nowUtc(); final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -868,9 +871,9 @@ public void testGetCurrentOffsetsAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -905,9 +908,9 @@ public void testGetEndOffsetsAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -943,8 +946,8 @@ public void testSetEndOffsetsAsync() throws Exception final Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -986,8 +989,8 @@ public void testSetEndOffsetsAsyncWithResume() throws Exception final Map endOffsets = ImmutableMap.of(0, 15L, 1, 120L); final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 533e8a10193e..427b9d1742de 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -1611,8 +1611,9 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception .andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); taskQueue.shutdown( EasyMock.contains("sequenceName-0"), - EasyMock.eq("Task [%s] failed to respond to [pause] in a timely manner, killing task"), - EasyMock.contains("sequenceName-0") + EasyMock.eq("An exception occured while waiting for task [%s] to pause: [%s]"), + EasyMock.contains("sequenceName-0"), + EasyMock.anyString() ); expectLastCall().times(2); expect(taskQueue.add(capture(captured))).andReturn(true).times(2); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java index 82959767750c..72a6ec527ab2 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java @@ -66,6 +66,8 @@ import java.util.List; import java.util.Map; +import static org.easymock.EasyMock.expect; + @RunWith(Parameterized.class) public class KinesisIndexTaskClientTest extends EasyMockSupport @@ -113,20 +115,20 @@ public void setUp() headers = createMock(HttpHeaders.class); client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider); - EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)) - .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) - .anyTimes(); - EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)) - .andReturn(Optional.of(TaskStatus.running(TEST_ID))) - .anyTimes(); + expect(taskInfoProvider.getTaskLocation(TEST_ID)) + .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + expect(taskInfoProvider.getTaskStatus(TEST_ID)) + .andReturn(Optional.of(TaskStatus.running(TEST_ID))) + .anyTimes(); for (String testId : TEST_IDS) { - EasyMock.expect(taskInfoProvider.getTaskLocation(testId)) - .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) - .anyTimes(); - EasyMock.expect(taskInfoProvider.getTaskStatus(testId)) - .andReturn(Optional.of(TaskStatus.running(testId))) - .anyTimes(); + expect(taskInfoProvider.getTaskLocation(testId)) + .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + expect(taskInfoProvider.getTaskStatus(testId)) + .andReturn(Optional.of(TaskStatus.running(testId))) + .anyTimes(); } } @@ -140,10 +142,10 @@ public void tearDown() public void testNoTaskLocation() throws IOException { EasyMock.reset(taskInfoProvider); - EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); - EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)) - .andReturn(Optional.of(TaskStatus.running(TEST_ID))) - .anyTimes(); + expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); + expect(taskInfoProvider.getTaskStatus(TEST_ID)) + .andReturn(Optional.of(TaskStatus.running(TEST_ID))) + .anyTimes(); replayAll(); Assert.assertFalse(client.stop(TEST_ID, true)); @@ -167,12 +169,12 @@ public void testTaskNotRunnableException() expectedException.expectMessage("Aborting request because task [test-id] is not runnable"); EasyMock.reset(taskInfoProvider); - EasyMock.expect(taskInfoProvider.getTaskLocation(TEST_ID)) - .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) - .anyTimes(); - EasyMock.expect(taskInfoProvider.getTaskStatus(TEST_ID)) - .andReturn(Optional.of(TaskStatus.failure(TEST_ID))) - .anyTimes(); + expect(taskInfoProvider.getTaskLocation(TEST_ID)) + .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) + .anyTimes(); + expect(taskInfoProvider.getTaskStatus(TEST_ID)) + .andReturn(Optional.of(TaskStatus.failure(TEST_ID))) + .anyTimes(); replayAll(); client.getCurrentOffsets(TEST_ID, true); @@ -185,8 +187,8 @@ public void testInternalServerError() expectedException.expect(RuntimeException.class); expectedException.expectMessage("org.apache.druid.java.util.common.IOE: Received status [500]"); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2); - EasyMock.expect( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.INTERNAL_SERVER_ERROR).times(2); + expect( httpClient.go( EasyMock.anyObject(Request.class), EasyMock.anyObject(FullResponseHandler.class), @@ -207,9 +209,9 @@ public void testBadRequest() expectedException.expect(IAE.class); expectedException.expectMessage("Received 400 Bad Request with body:"); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2); - EasyMock.expect(responseHolder.getContent()).andReturn(""); - EasyMock.expect( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.BAD_REQUEST).times(2); + expect(responseHolder.getContent()).andReturn(""); + expect( httpClient.go( EasyMock.anyObject(Request.class), EasyMock.anyObject(FullResponseHandler.class), @@ -227,14 +229,14 @@ public void testBadRequest() @Test public void testTaskLocationMismatch() { - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) - .andReturn(HttpResponseStatus.OK); - EasyMock.expect(responseHolder.getResponse()).andReturn(response); - EasyMock.expect(responseHolder.getContent()).andReturn("") - .andReturn("{}"); - EasyMock.expect(response.headers()).andReturn(headers); - EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id"); - EasyMock.expect( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) + .andReturn(HttpResponseStatus.OK); + expect(responseHolder.getResponse()).andReturn(response); + expect(responseHolder.getContent()).andReturn("") + .andReturn("{}"); + expect(response.headers()).andReturn(headers); + expect(headers.get("X-Druid-Task-Id")).andReturn("a-different-task-id"); + expect( httpClient.go( EasyMock.anyObject(Request.class), EasyMock.anyObject(FullResponseHandler.class), @@ -255,9 +257,9 @@ public void testTaskLocationMismatch() public void testGetCurrentOffsets() throws Exception { Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -288,15 +290,15 @@ public void testGetCurrentOffsetsWithRetry() throws Exception client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 3); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) - .andReturn(HttpResponseStatus.OK).times(1); - EasyMock.expect(responseHolder.getContent()).andReturn("").times(2) - .andReturn("{\"0\":1, \"1\":10}"); - EasyMock.expect(responseHolder.getResponse()).andReturn(response).times(2); - EasyMock.expect(response.headers()).andReturn(headers).times(2); - EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2); - - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) + .andReturn(HttpResponseStatus.OK).times(1); + expect(responseHolder.getContent()).andReturn("").times(2) + .andReturn("{\"0\":1, \"1\":10}"); + expect(responseHolder.getResponse()).andReturn(response).times(2); + expect(response.headers()).andReturn(headers).times(2); + expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).times(2); + + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -332,13 +334,13 @@ public void testGetCurrentOffsetsWithExhaustedRetries() client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); - EasyMock.expect(responseHolder.getContent()).andReturn("").anyTimes(); - EasyMock.expect(responseHolder.getResponse()).andReturn(response).anyTimes(); - EasyMock.expect(response.headers()).andReturn(headers).anyTimes(); - EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).anyTimes(); + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); + expect(responseHolder.getContent()).andReturn("").anyTimes(); + expect(responseHolder.getResponse()).andReturn(response).anyTimes(); + expect(response.headers()).andReturn(headers).anyTimes(); + expect(headers.get("X-Druid-Task-Id")).andReturn(TEST_ID).anyTimes(); - EasyMock.expect( + expect( httpClient.go( EasyMock.anyObject(Request.class), EasyMock.anyObject(FullResponseHandler.class), @@ -355,9 +357,9 @@ public void testGetCurrentOffsetsWithExhaustedRetries() public void testGetEndOffsets() throws Exception { Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}"); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -389,13 +391,13 @@ public void testGetStartTime() throws Exception DateTime now = DateTimes.nowUtc(); Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) - .andReturn(HttpResponseStatus.OK); - EasyMock.expect(responseHolder.getResponse()).andReturn(response); - EasyMock.expect(response.headers()).andReturn(headers); - EasyMock.expect(headers.get("X-Druid-Task-Id")).andReturn(null); - EasyMock.expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(3) + .andReturn(HttpResponseStatus.OK); + expect(responseHolder.getResponse()).andReturn(response); + expect(response.headers()).andReturn(headers); + expect(headers.get("X-Druid-Task-Id")).andReturn(null); + expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -424,9 +426,9 @@ public void testGetStatus() throws Exception Status status = Status.READING; Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); - EasyMock.expect(responseHolder.getContent()).andReturn(StringUtils.format("\"%s\"", status.toString())).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK); + expect(responseHolder.getContent()).andReturn(StringUtils.format("\"%s\"", status.toString())).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -453,9 +455,9 @@ public void testGetStatus() throws Exception public void testPause() throws Exception { Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); - EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).times(2); + expect(responseHolder.getContent()).andReturn("{\"0\":1, \"1\":10}").anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -486,25 +488,25 @@ public void testPauseWithSubsequentGetOffsets() throws Exception Capture captured = Capture.newInstance(); Capture captured2 = Capture.newInstance(); Capture captured3 = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.ACCEPTED).times(2) - .andReturn(HttpResponseStatus.OK).times(2); - EasyMock.expect(responseHolder.getContent()).andReturn("\"PAUSED\"") - .andReturn("{\"0\":1, \"1\":10}").anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.ACCEPTED).times(2) + .andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("\"PAUSED\"").times(2) + .andReturn("{\"0\":1, \"1\":10}").anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) )).andReturn( Futures.immediateFuture(responseHolder) ); - EasyMock.expect(httpClient.go( + expect(httpClient.go( EasyMock.capture(captured2), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) )).andReturn( Futures.immediateFuture(responseHolder) ); - EasyMock.expect(httpClient.go( + expect(httpClient.go( EasyMock.capture(captured3), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -548,8 +550,8 @@ public void testPauseWithSubsequentGetOffsets() throws Exception public void testResume() throws Exception { Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -576,8 +578,8 @@ public void testSetEndOffsets() throws Exception Map endOffsets = ImmutableMap.of("0", "15", "1", "120"); Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -605,8 +607,8 @@ public void testSetEndOffsetsAndResume() throws Exception Map endOffsets = ImmutableMap.of("0", "15", "1", "120"); Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -632,8 +634,8 @@ public void testSetEndOffsetsAndResume() throws Exception public void testStop() throws Exception { Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -658,8 +660,8 @@ public void testStop() throws Exception public void testStopAndPublish() throws Exception { Capture captured = Capture.newInstance(); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -685,8 +687,8 @@ public void testStopAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -721,8 +723,8 @@ public void testResumeAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -757,9 +759,9 @@ public void testPauseAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -794,9 +796,9 @@ public void testGetStatusAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("\"READING\"").anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -832,9 +834,9 @@ public void testGetStartTimeAsync() throws Exception final DateTime now = DateTimes.nowUtc(); final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn(String.valueOf(now.getMillis())).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -869,9 +871,9 @@ public void testGetCurrentOffsetsAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -906,9 +908,9 @@ public void testGetEndOffsetsAsync() throws Exception { final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(responseHolder.getContent()).andReturn("{\"0\":\"1\"}").anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -944,8 +946,8 @@ public void testSetEndOffsetsAsync() throws Exception Map endOffsets = ImmutableMap.of("0", "15L", "1", "120L"); final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) @@ -987,8 +989,8 @@ public void testSetEndOffsetsAsyncWithResume() throws Exception Map endOffsets = ImmutableMap.of("0", "15L", "1", "120L"); final int numRequests = TEST_IDS.size(); Capture captured = Capture.newInstance(CaptureType.ALL); - EasyMock.expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); - EasyMock.expect(httpClient.go( + expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.OK).anyTimes(); + expect(httpClient.go( EasyMock.capture(captured), EasyMock.anyObject(FullResponseHandler.class), EasyMock.eq(TEST_HTTP_TIMEOUT) diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 5bd9404c90f8..f9dab5407fd9 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -2040,8 +2040,9 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception .andReturn(Futures.immediateFailedFuture(new RuntimeException())).times(2); taskQueue.shutdown( EasyMock.contains("sequenceName-0"), - EasyMock.eq("Task [%s] failed to respond to [pause] in a timely manner, killing task"), - EasyMock.contains("sequenceName-0") + EasyMock.eq("An exception occured while waiting for task [%s] to pause: [%s]"), + EasyMock.contains("sequenceName-0"), + EasyMock.anyString() ); EasyMock.expectLastCall().times(2); EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 36c9ec60e488..8f582a38283f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -100,7 +100,6 @@ public boolean resume(final String id) } } - public Map pause(final String id) { log.debug("Pause task[%s]", id); @@ -114,29 +113,46 @@ public Map pause(final String id) true ); - if (response.getStatus().equals(HttpResponseStatus.OK)) { - log.info("Task [%s] paused successfully", id); - return deserializeMap(response.getContent(), Map.class, getPartitionType(), getSequenceType()); - } + final HttpResponseStatus responseStatus = response.getStatus(); + final String responseContent = response.getContent(); - while (true) { - if (getStatus(id) == SeekableStreamIndexTaskRunner.Status.PAUSED) { - return getCurrentOffsets(id, true); - } - - final Duration delay = newRetryPolicy().getAndIncrementRetryDelay(); - if (delay == null) { - log.error("Task [%s] failed to pause, aborting", id); - throw new ISE("Task [%s] failed to pause, aborting", id); - } else { - final long sleepTime = delay.getMillis(); - log.info( - "Still waiting for task [%s] to pause; will try again in [%s]", - id, - new Duration(sleepTime).toString() - ); - Thread.sleep(sleepTime); + if (responseStatus.equals(HttpResponseStatus.OK)) { + log.info("Task [%s] paused successfully", id); + return deserializeMap(responseContent, Map.class, getPartitionType(), getSequenceType()); + } else if (responseStatus.equals(HttpResponseStatus.ACCEPTED)) { + // The task received the pause request, but its status hasn't been changed yet. + while (true) { + final SeekableStreamIndexTaskRunner.Status status = getStatus(id); + if (status == SeekableStreamIndexTaskRunner.Status.PAUSED) { + return getCurrentOffsets(id, true); + } + + final Duration delay = newRetryPolicy().getAndIncrementRetryDelay(); + if (delay == null) { + throw new ISE( + "Task [%s] failed to change its status from [%s] to [%s], aborting", + id, + status, + SeekableStreamIndexTaskRunner.Status.PAUSED + ); + } else { + final long sleepTime = delay.getMillis(); + log.info( + "Still waiting for task [%s] to change its status to [%s]; will try again in [%s]", + id, + SeekableStreamIndexTaskRunner.Status.PAUSED, + new Duration(sleepTime).toString() + ); + Thread.sleep(sleepTime); + } } + } else { + throw new ISE( + "Pause request for task [%s] failed with response [%s] : [%s]", + id, + responseStatus, + responseContent + ); } } catch (NoTaskLocationException e) { @@ -144,10 +160,7 @@ public Map pause(final String id) return ImmutableMap.of(); } catch (IOException | InterruptedException e) { - throw new RE( - StringUtils.format("Exception [%s] while pausing Task [%s]", e.getMessage(), id), - e - ); + throw new RE(e, "Exception [%s] while pausing Task [%s]", e.getMessage(), id); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 5d3b74b394f7..d42ceba08bbe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1952,13 +1952,39 @@ public Map apply(List endOffsets = new HashMap<>(); for (int i = 0; i < input.size(); i++) { - Map result = input.get(i); + final Map result = input.get(i); + final String taskId = pauseTaskIds.get(i); - if (result == null || result.isEmpty()) { // kill tasks that didn't return a value - String taskId = pauseTaskIds.get(i); - killTask(taskId, "Task [%s] failed to respond to [pause] in a timely manner, killing task", taskId); + if (result == null) { + // Get the exception + final Throwable pauseException; + try { + // The below get should throw ExecutionException since result is null. + final Map pauseResult = pauseFutures.get(i).get(); + throw new ISE( + "WTH? The pause request for task [%s] is supposed to fail, but returned [%s]", + taskId, + pauseResult + ); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + catch (ExecutionException e) { + pauseException = e.getCause() == null ? e : e.getCause(); + } + + killTask( + taskId, + "An exception occured while waiting for task [%s] to pause: [%s]", + taskId, + pauseException + ); taskGroup.tasks.remove(taskId); + } else if (result.isEmpty()) { + killTask(taskId, "Task [%s] returned empty offsets after pause", taskId); + taskGroup.tasks.remove(taskId); } else { // otherwise build a map of the highest sequences seen for (Entry sequence : result.entrySet()) { if (!endOffsets.containsKey(sequence.getKey()) From 66be31de075e33a55301e5d203e607bd36288042 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 19 Dec 2018 13:03:48 -0800 Subject: [PATCH 76/87] fix injection binding --- .../extensions-core/kafka-ingestion.md | 2 +- .../extensions-core/kinesis-ingestion.md | 17 +++++------------ .../kafka/KafkaIndexTaskTuningConfig.java | 2 ++ .../kinesis/KinesisIndexTaskTuningConfig.java | 4 ++-- .../kinesis/KinesisIndexingServiceModule.java | 3 ++- .../supervisor/KinesisSupervisorSpec.java | 3 ++- 6 files changed, 14 insertions(+), 17 deletions(-) diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index 17a43618b57b..aa771b7a27c1 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -312,7 +312,7 @@ compatible because they have a different ingestion spec or partition allocation, supervisor will create a new set of tasks. In this way, the supervisors are persistent across overlord restarts and fail-overs. -A supervisor is stopped via the `POST /druid/indexer/v1/supervisor//shutdown` endpoint. This places a +A supervisor is stopped via the `POST /druid/indexer/v1/supervisor//terminate` endpoint. This places a tombstone marker in the database (to prevent the supervisor from being reloaded on a restart) and then gracefully shuts down the currently running supervisor. When a supervisor is shut down in this way, it will instruct its managed tasks to stop reading and begin publishing their segments immediately. The call to the shutdown endpoint will diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md index 9dde6ce18739..631857d71b81 100644 --- a/docs/content/development/extensions-core/kinesis-ingestion.md +++ b/docs/content/development/extensions-core/kinesis-ingestion.md @@ -194,7 +194,7 @@ For Roaring bitmaps: |`endpoint`|String|The AWS Kinesis stream endpoint for a region. You can find a list of endpoints [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region).|no (default == kinesis.us-east-1.amazonaws.com)| |`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against node failure.|no (default == 1)| |`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. See 'Capacity Planning' below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKinesisshards}`.|no (default == 1)| -|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment. Note that segments are only pushed to deep storage and loadable by historical nodes when the indexing task completes.|no (default == PT1H)| +|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment.|no (default == PT1H)| |`startDelay`|ISO8601 Period|The period to wait before the supervisor starts managing tasks.|no (default == PT5S)| |`period`|ISO8601 Period|How often the supervisor will execute its management logic. Note that the supervisor will also run in response to certain events (such as tasks succeeding, failing, and reaching their taskDuration) so this value specifies the maximum time between iterations.|no (default == PT30S)| |`useEarliestSequenceNumber`|Boolean|If a supervisor is managing a dataSource for the first time, it will obtain a set of starting sequence numbers from Kinesis. This flag determines whether it retrieves the earliest or latest sequence numbers in Kinesis. Under normal circumstances, subsequent tasks will start from where the previous segments ended so this flag will only be used on first run.|no (default == false)| @@ -213,24 +213,17 @@ This section gives descriptions of how some supervisor APIs work specifically in For all supervisor APIs, please check [Supervisor APIs](../../operations/api-reference.html#supervisors). ### AWS Authentication -To authenticate with AWS, you must provide your AWS access key and AWS secret key via environment properties, for example: +To authenticate with AWS, you must provide your AWS access key and AWS secret key via runtime.properties, for example: ``` -Ddruid.kinesis.accessKey=123 -Ddruid.kinesis.secretKey=456 ``` -- The AWS access key ID is used for Kinesis API requests. If this is not provided, the service will look for credentials set in system properties, in the default profile configuration file, and from the EC2 instance profile provider (in this order). -- The AWS secret access key is used for Kinesis API requests. +The AWS access key ID ad secret access key are used for Kinesis API requests. If this is not provided, the service will look for credentials set in environment variables, in the default profile configuration file, and from the EC2 instance profile provider (in this order). ### Getting Supervisor Status Report `GET /druid/indexer/v1/supervisor//status` returns a snapshot report of the current state of the tasks managed by the given supervisor. This includes the latest sequence numbers as reported by Kinesis. Unlike the Kafka Indexing Service, stats about lag is not yet supported. -### Getting Supervisor Ingestion Stats Report - -`GET /druid/indexer/v1/supervisor//stats` returns a snapshot of the current ingestion row counters for each task being managed by the supervisor, along with moving averages for the row counters. - -See [Task Reports: Row Stats](../../ingestion/reports.html#row-stats) for more information. - ### Updating Existing Supervisors `POST /druid/indexer/v1/supervisor` can be used to update existing supervisor spec. @@ -261,7 +254,7 @@ generated segments to be accepted. If the messages at the expected starting sequ refuse to start and in-flight tasks will fail. This endpoint can be used to clear the stored sequence numbers which will cause the supervisor to start reading from -either the earliest or latest sequence numbers in Kinesis (depending on the value of `useEarliestOffset`). The supervisor must be +either the earliest or latest sequence numbers in Kinesis (depending on the value of `useEarliestSequenceNumber`). The supervisor must be running for this endpoint to be available. After the stored sequence numbers are cleared, the supervisor will automatically kill and re-create any active tasks so that tasks begin reading from valid sequence numbers. @@ -321,7 +314,7 @@ compatible because they have a different ingestion spec or shard allocation, the supervisor will create a new set of tasks. In this way, the supervisors are persistent across overlord restarts and fail-overs. -A supervisor is stopped via the `POST /druid/indexer/v1/supervisor//shutdown` endpoint. This places a +A supervisor is stopped via the `POST /druid/indexer/v1/supervisor//terminate` endpoint. This places a tombstone marker in the database (to prevent the supervisor from being reloaded on a restart) and then gracefully shuts down the currently running supervisor. When a supervisor is shut down in this way, it will instruct its managed tasks to stop reading and begin publishing their segments immediately. The call to the shutdown endpoint will diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java index 9ceb21ed0df2..88095943fe6f 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -29,6 +30,7 @@ import javax.annotation.Nullable; import java.io.File; +@JsonTypeName("KafkaTuningConfig") public class KafkaIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningConfig { @JsonCreator diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java index 98915a4080f4..30474dc06050 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.segment.IndexSpec; @@ -31,16 +32,15 @@ import java.io.File; import java.util.Objects; +@JsonTypeName("KinesisTuningConfig") public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningConfig { - private static final int DEFAULT_RECORD_BUFFER_SIZE = 10000; private static final int DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT = 5000; private static final int DEFAULT_RECORD_BUFFER_FULL_WAIT = 5000; private static final int DEFAULT_FETCH_SEQUENCE_NUMBER_TIMEOUT = 20000; private static final int DEFAULT_MAX_RECORDS_PER_POLL = 100; - private final int recordBufferSize; private final int recordBufferOfferTimeout; private final int recordBufferFullWait; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java index e32f710e98a2..cba5166f268d 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexingServiceModule.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.TypeLiteral; +import com.google.inject.name.Names; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; @@ -61,6 +62,6 @@ public void configure(Binder binder) } ).to(KinesisIndexTaskClientFactory.class).in(LazySingleton.class); - JsonConfigProvider.bind(binder, "druid.kinesis", AWSCredentialsConfig.class); + JsonConfigProvider.bind(binder, "druid.kinesis", AWSCredentialsConfig.class, Names.named("kinesis")); } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java index 97dc2c5f0672..1feb8f63361e 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.name.Named; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -57,7 +58,7 @@ public KinesisSupervisorSpec( @JacksonInject ServiceEmitter emitter, @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, - @JacksonInject AWSCredentialsConfig awsCredentialsConfig + @JacksonInject @Named("kinesis") AWSCredentialsConfig awsCredentialsConfig ) { super( From d27bf0e3ba223d4e7e0535df3f7effb903e01e0b Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 19 Dec 2018 13:47:20 -0800 Subject: [PATCH 77/87] comments --- .../druid/indexing/kafka/KafkaIndexTaskTuningConfig.java | 1 + .../druid/indexing/kafka/supervisor/KafkaSupervisor.java | 8 +++----- .../kafka/supervisor/KafkaSupervisorIOConfig.java | 1 + .../indexing/kinesis/supervisor/KinesisSupervisor.java | 2 +- .../supervisor/SeekableStreamSupervisor.java | 9 ++++----- 5 files changed, 10 insertions(+), 11 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java index 88095943fe6f..7cee87790654 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java @@ -108,6 +108,7 @@ public String toString() return "KafkaIndexTaskTuningConfig{" + "maxRowsInMemory=" + getMaxRowsInMemory() + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + + ", maxTotalRows=" + getMaxTotalRows() + ", maxBytesInMemory=" + getMaxBytesInMemory() + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index c113bcc7a3f0..45cb39a09320 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -191,7 +191,7 @@ protected SeekableStreamSupervisorReportPayload createReportPaylo @Override - protected SeekableStreamIndexTaskIOConfig createIoConfig( + protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( int groupId, Map startPartitions, Map endPartitions, @@ -282,8 +282,7 @@ protected Map getLagPerPartition(Map currentOffset @Override protected KafkaDataSourceMetadata createDataSourceMetaData( - String topic, - Map map + String topic, Map map ) { return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, map)); @@ -291,8 +290,7 @@ protected KafkaDataSourceMetadata createDataSourceMetaData( @Override protected OrderedSequenceNumber makeSequenceNumber( - Long seq, - boolean isExclusive + Long seq, boolean isExclusive ) { return KafkaSequenceNumber.of(seq); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java index 693f4fe6c359..80b842b83496 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java @@ -112,6 +112,7 @@ public String toString() ", period=" + getPeriod() + ", useEarliestOffset=" + isUseEarliestOffset() + ", completionTimeout=" + getCompletionTimeout() + + ", earlyMessageRejectionPeriod=" + getEarlyMessageRejectionPeriod() + ", lateMessageRejectionPeriod=" + getLateMessageRejectionPeriod() + ", skipOffsetGaps=" + skipOffsetGaps + '}'; diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index e6471e476a9f..29a06987f44b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -105,7 +105,7 @@ public KinesisSupervisor( } @Override - protected SeekableStreamIndexTaskIOConfig createIoConfig( + protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( int groupId, Map startPartitions, Map endPartitions, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index d42ceba08bbe..fd0e53f373a3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -2478,7 +2478,7 @@ private void createTasksForGroup(int groupId, int replicas) DateTime minimumMessageTime = activelyReadingTaskGroups.get(groupId).minimumMessageTime.orNull(); DateTime maximumMessageTime = activelyReadingTaskGroups.get(groupId).maximumMessageTime.orNull(); - SeekableStreamIndexTaskIOConfig newIoConfig = createIoConfig( + SeekableStreamIndexTaskIOConfig newIoConfig = createTaskIoConfig( groupId, startPartitions, endPartitions, @@ -2665,11 +2665,11 @@ public void checkpoint( } /** - * creates a specific task IOConfig instance for Kafka/Kinesis* + * creates a specific task IOConfig instance for Kafka/Kinesis * * @return specific instance of Kafka/Kinesis IOConfig */ - protected abstract SeekableStreamIndexTaskIOConfig createIoConfig( + protected abstract SeekableStreamIndexTaskIOConfig createTaskIoConfig( int groupId, Map startPartitions, Map endPartitions, @@ -2737,8 +2737,7 @@ protected abstract List createDataSourceMetaData( - String - stream, Map map + String stream, Map map ); /** From 5b4892818e9c39622b6fdeddf870fae9db37d330 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 19 Dec 2018 14:59:56 -0800 Subject: [PATCH 78/87] KinesisRegion refactor --- .../kinesis/KinesisRecordSupplier.java | 76 +++++++++++++++---- .../druid/indexing/kinesis/KinesisRegion.java | 5 +- .../common/OrderedPartitionableRecord.java | 19 ++++- 3 files changed, 79 insertions(+), 21 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 36fea16af897..317612d7d760 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -25,8 +25,10 @@ import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.services.kinesis.AmazonKinesis; import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder; +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; import com.amazonaws.services.kinesis.model.GetRecordsRequest; import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.InvalidArgumentException; import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; import com.amazonaws.services.kinesis.model.Record; import com.amazonaws.services.kinesis.model.ResourceNotFoundException; @@ -147,6 +149,9 @@ private Runnable getRecordRunnable() return; } + // used for retrying on InterruptedException + GetRecordsResult recordsResult = null; + OrderedPartitionableRecord currRecord = null; try { @@ -154,14 +159,16 @@ private Runnable getRecordRunnable() log.info("shardIterator[%s] has been closed and has no more records", streamPartition.getPartitionId()); // add an end-of-shard marker so caller knows this shard is closed - OrderedPartitionableRecord endOfShardRecord = new OrderedPartitionableRecord<>( + currRecord = new OrderedPartitionableRecord<>( streamPartition.getStream(), streamPartition.getPartitionId(), KinesisSequenceNumber.END_OF_SHARD_MARKER, null ); - if (!records.offer(endOfShardRecord, recordBufferOfferTimeout, TimeUnit.MILLISECONDS)) { + recordsResult = null; + + if (!records.offer(currRecord, recordBufferOfferTimeout, TimeUnit.MILLISECONDS)) { log.warn("OrderedPartitionableRecord buffer full, retrying in [%,dms]", recordBufferFullWait); rescheduleRunnable(recordBufferFullWait); } @@ -169,7 +176,7 @@ private Runnable getRecordRunnable() return; } - GetRecordsResult recordsResult = kinesis.getRecords(new GetRecordsRequest().withShardIterator( + recordsResult = kinesis.getRecords(new GetRecordsRequest().withShardIterator( shardIterator).withLimit(recordsPerFetch)); // list will come back empty if there are no records @@ -195,7 +202,7 @@ private Runnable getRecordRunnable() data = Collections.singletonList(toByteArray(kinesisRecord.getData())); } - final OrderedPartitionableRecord record = new OrderedPartitionableRecord<>( + currRecord = new OrderedPartitionableRecord<>( streamPartition.getStream(), streamPartition.getPartitionId(), kinesisRecord.getSequenceNumber(), @@ -205,26 +212,26 @@ private Runnable getRecordRunnable() log.trace( "Stream[%s] / partition[%s] / sequenceNum[%s] / bufferRemainingCapacity[%d]: %s", - record.getStream(), - record.getPartitionId(), - record.getSequenceNumber(), + currRecord.getStream(), + currRecord.getPartitionId(), + currRecord.getSequenceNumber(), records.remainingCapacity(), - record.getData().stream().map(StringUtils::fromUtf8).collect(Collectors.toList()) + currRecord.getData().stream().map(StringUtils::fromUtf8).collect(Collectors.toList()) ); // If the buffer was full and we weren't able to add the message, grab a new stream iterator starting // from this message and back off for a bit to let the buffer drain before retrying. - if (!records.offer(record, recordBufferOfferTimeout, TimeUnit.MILLISECONDS)) { + if (!records.offer(currRecord, recordBufferOfferTimeout, TimeUnit.MILLISECONDS)) { log.warn( "OrderedPartitionableRecord buffer full, storing iterator and retrying in [%,dms]", recordBufferFullWait ); shardIterator = kinesis.getShardIterator( - record.getStream(), - record.getPartitionId(), + currRecord.getStream(), + currRecord.getPartitionId(), ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), - record.getSequenceNumber() + currRecord.getSequenceNumber() ).getShardIterator(); rescheduleRunnable(recordBufferFullWait); @@ -240,8 +247,47 @@ private Runnable getRecordRunnable() long retryMs = Math.max(PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS, fetchDelayMillis); rescheduleRunnable(retryMs); } + catch (InterruptedException e) { + // may happen if interrupted while BlockingQueue.offer() is waiting + log.warn( + e, + "Interrupted while waiting to add record to buffer, retrying in [%,dms]", + EXCEPTION_RETRY_DELAY_MS + ); + if (currRecord != null) { + shardIterator = kinesis.getShardIterator( + currRecord.getStream(), + currRecord.getPartitionId(), + ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), + currRecord.getSequenceNumber() + ).getShardIterator(); + rescheduleRunnable(EXCEPTION_RETRY_DELAY_MS); + } else { + throw new ISE("can't reschedule fetch records runnable, current record is null??"); + } + } + catch (ExpiredIteratorException e) { + log.warn( + e, + "ShardIterator expired while trying to fetch records, retrying in [%,dms]", + fetchDelayMillis + ); + if (recordsResult != null) { + shardIterator = recordsResult.getNextShardIterator(); // will be null if the shard has been closed + rescheduleRunnable(fetchDelayMillis); + } else { + throw new ISE("can't reschedule fetch records runnable, recordsResult is null??"); + } + } + catch (ResourceNotFoundException | InvalidArgumentException e) { + // aws errors + log.error(e, "encounted AWS error while attempting to fetch records, will not retry"); + throw e; + } catch (Throwable e) { + // non transient errors log.error(e, "getRecordRunnable exception"); + throw new RuntimeException(e); } }; @@ -255,6 +301,7 @@ private void rescheduleRunnable(long delayMillis) } catch (RejectedExecutionException e) { log.warn( + e, "Caught RejectedExecutionException, KinesisRecordSupplier for partition[%s] has likely temporarily shutdown the ExecutorService." + "This is expected behavior after calling seek(), seekToEarliest() and seekToLatest()", streamPartition.getPartitionId() @@ -548,6 +595,7 @@ public void close() } catch (InterruptedException e) { log.warn(e, "InterruptedException while shutting down"); + throw new RuntimeException(e); } this.closed = true; @@ -598,7 +646,7 @@ private void filterBufferAndResetFetchRunnable(Set> part // filter records in buffer and only retain ones whose partition was not seeked BlockingQueue> newQ = new LinkedBlockingQueue<>(recordBufferSize); records - .parallelStream() + .stream() .filter(x -> !partitions.contains(x.getStreamPartition())) .forEachOrdered(newQ::offer); @@ -622,7 +670,7 @@ private String getSequenceNumberInternal(StreamPartition partition, Shar ).getShardIterator(); } catch (ResourceNotFoundException e) { - log.warn("Caught ResourceNotFoundException: [%s]", e); + log.warn(e, "Caught ResourceNotFoundException while getting shardIterator"); } return getSequenceNumberInternal(partition, shardIterator); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java index 0c68bd8a2d20..2b9fbd758e94 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java @@ -55,7 +55,7 @@ public static KinesisRegion fromString(String value) { return EnumSet.allOf(KinesisRegion.class) .stream() - .filter(x -> x.toString().equals(value)) + .filter(x -> x.serialize().equals(value)) .findFirst() .orElseThrow(() -> new IAE("Invalid region %s, region must be one of: %s", value, getNames())); } @@ -71,8 +71,7 @@ public String getEndpoint() } @JsonValue - @Override - public String toString() + public String serialize() { return StringUtils.toLowerCase(name()).replace('_', '-'); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java index 8f270915edaa..4dd653e760ff 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java @@ -26,6 +26,7 @@ import java.util.Arrays; import java.util.List; import java.util.Objects; +import java.util.stream.Collectors; /** * Represents a generic record with a PartitionIdType (partition id) and SequenceOffsetType (sequence number) and data @@ -94,16 +95,26 @@ public boolean equals(Object o) return false; } OrderedPartitionableRecord that = (OrderedPartitionableRecord) o; + + if (data.size() != that.data.size()) { + return false; + } + + for (int i = 0; i < data.size(); i++) { + if (!Arrays.equals(data.get(i), that.data.get(i))) { + return false; + } + } + return Objects.equals(stream, that.stream) && Objects.equals(partitionId, that.partitionId) && - Objects.equals(sequenceNumber, that.sequenceNumber) && - data.size() == that.data.size() && - data.stream().allMatch(d -> that.data.stream().anyMatch(d2 -> Arrays.equals(d, d2))); + Objects.equals(sequenceNumber, that.sequenceNumber); } @Override public int hashCode() { - return Objects.hash(stream, partitionId, sequenceNumber, data); + final int hashOfData = data.stream().map(Arrays::hashCode).collect(Collectors.toList()).hashCode(); + return Objects.hash(stream, partitionId, sequenceNumber, hashOfData); } } From 205668ca169192e94640ae8a5e7c459e3ba53dbc Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 19 Dec 2018 15:18:03 -0800 Subject: [PATCH 79/87] comments part idk lol --- .../seekablestream/supervisor/SeekableStreamSupervisor.java | 3 ++- .../indexing/seekablestream/supervisor/TaskReportData.java | 6 ++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index fd0e53f373a3..d0a872e9f35f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -2737,7 +2737,8 @@ protected abstract List createDataSourceMetaData( - String stream, Map map + String stream, + Map map ); /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java index b265df1856b3..0b18e58ee678 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/TaskReportData.java @@ -113,6 +113,12 @@ public String toString() '}'; } + /** + * Used by the Supervisor to report status of tasks + * ACTIVE - task is waiting to be started, started, or reading + * PUBLISHING - task is publishing or registering handoff + * UNNKNOWN - unknown + */ public enum TaskType { ACTIVE, PUBLISHING, UNKNOWN From f3c7063eb8b244b945bfaba52367231d381b2d0e Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 19 Dec 2018 15:38:32 -0800 Subject: [PATCH 80/87] can't think of a commit msg anymore --- .../druid/indexing/kafka/supervisor/KafkaSupervisor.java | 8 ++------ .../indexing/kinesis/supervisor/KinesisSupervisor.java | 5 +---- .../indexing/seekablestream/SeekableStreamIndexTask.java | 2 +- .../seekablestream/SeekableStreamIndexTaskClient.java | 2 +- .../seekablestream/SeekableStreamIndexTaskRunner.java | 2 +- .../supervisor/SeekableStreamSupervisor.java | 2 +- 6 files changed, 7 insertions(+), 14 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 45cb39a09320..6b52e40a1961 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -281,17 +281,13 @@ protected Map getLagPerPartition(Map currentOffset } @Override - protected KafkaDataSourceMetadata createDataSourceMetaData( - String topic, Map map - ) + protected KafkaDataSourceMetadata createDataSourceMetaData(String topic, Map map) { return new KafkaDataSourceMetadata(new SeekableStreamPartitions<>(topic, map)); } @Override - protected OrderedSequenceNumber makeSequenceNumber( - Long seq, boolean isExclusive - ) + protected OrderedSequenceNumber makeSequenceNumber(Long seq, boolean isExclusive) { return KafkaSequenceNumber.of(seq); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 29a06987f44b..1829a0db6536 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -274,10 +274,7 @@ protected SeekableStreamDataSourceMetadata createDataSourceMetaD } @Override - protected OrderedSequenceNumber makeSequenceNumber( - String seq, - boolean isExclusive - ) + protected OrderedSequenceNumber makeSequenceNumber(String seq, boolean isExclusive) { return KinesisSequenceNumber.of(seq, isExclusive); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index ef96fcd00c2d..a9873239a06d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -67,8 +67,8 @@ public abstract class SeekableStreamIndexTask runner; protected final DataSchema dataSchema; protected final InputRowParser parser; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 8f582a38283f..bd946cc6e9c7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -44,7 +44,7 @@ public abstract class SeekableStreamIndexTaskClient extends IndexTaskClient { - private final EmittingLogger log = new EmittingLogger(this.getClass()); + private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTaskClient.class); public SeekableStreamIndexTaskClient( HttpClient httpClient, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index c3d182e07810..24f68708aa48 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -141,7 +141,7 @@ public enum Status PUBLISHING } - private final EmittingLogger log = new EmittingLogger(this.getClass()); + private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTaskRunner.class); private static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; private static final String METADATA_PUBLISH_PARTITIONS = "publishPartitions"; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index d0a872e9f35f..84c624deb53b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -131,7 +131,7 @@ public abstract class SeekableStreamSupervisor Date: Wed, 19 Dec 2018 15:55:02 -0800 Subject: [PATCH 81/87] remove possiblyResetDataSourceMetadata() for IncrementalPublishingTaskRunner --- ...ementalPublishingKafkaIndexTaskRunner.java | 28 +------------------ 1 file changed, 1 insertion(+), 27 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index aee1972b056e..e8f626208667 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -204,33 +204,7 @@ protected void possiblyResetDataSourceMetadata( Map currOffsets ) { - for (final StreamPartition streamPartition : assignment) { - Long sequence = currOffsets.get(streamPartition.getPartitionId()); - Long earliestSequenceNumber = recordSupplier.getEarliestSequenceNumber(streamPartition); - if (earliestSequenceNumber == null - || createSequenceNumber(earliestSequenceNumber).compareTo(createSequenceNumber(sequence)) > 0) { - if (task.getTuningConfig().isResetOffsetAutomatically()) { - log.info("Attempting to reset sequences automatically for all partitions"); - try { - sendResetRequestAndWait( - assignment.stream() - .collect(Collectors.toMap(x -> x, x -> currOffsets.get(x.getPartitionId()))), - toolbox - ); - } - catch (IOException e) { - throw new ISE(e, "Exception while attempting to automatically reset sequences"); - } - } else { - throw new ISE( - "Starting sequenceNumber [%s] is no longer available for partition [%s] (earliest: [%s]) and resetOffsetAutomatically is not enabled", - sequence, - streamPartition.getPartitionId(), - earliestSequenceNumber - ); - } - } - } + // do nothing } @Override From 9f594887bd669a16b3efd04a21e32063c35ed79b Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 19 Dec 2018 17:37:49 -0800 Subject: [PATCH 82/87] commmmmmmmmmments --- .../KinesisIndexTaskClientFactory.java | 5 +--- .../kinesis/KinesisIndexTaskTuningConfig.java | 12 +++++++-- .../kinesis/KinesisRecordSupplier.java | 17 +++++++++++- .../SeekableStreamIndexTaskRunner.java | 26 +++++++++---------- .../SeekableStreamPartitions.java | 4 +++ 5 files changed, 43 insertions(+), 21 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java index 4f61892f177d..6d443464da6b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientFactory.java @@ -36,10 +36,7 @@ public KinesisIndexTaskClientFactory( @Json ObjectMapper mapper ) { - super( - httpClient, - mapper - ); + super(httpClient, mapper); } @Override diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java index 30474dc06050..95ee278985fe 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java @@ -205,11 +205,14 @@ public boolean equals(Object o) isLogParseExceptions() == that.isLogParseExceptions() && getMaxParseExceptions() == that.getMaxParseExceptions() && getMaxSavedParseExceptions() == that.getMaxSavedParseExceptions() && + getMaxRecordsPerPoll() == that.getMaxRecordsPerPoll() && Objects.equals(getIntermediatePersistPeriod(), that.getIntermediatePersistPeriod()) && Objects.equals(getBasePersistDirectory(), that.getBasePersistDirectory()) && Objects.equals(getIndexSpec(), that.getIndexSpec()) && Objects.equals(getFetchThreads(), that.getFetchThreads()) && - Objects.equals(getSegmentWriteOutMediumFactory(), that.getSegmentWriteOutMediumFactory()); + Objects.equals(getSegmentWriteOutMediumFactory(), that.getSegmentWriteOutMediumFactory()) && + Objects.equals(getMaxTotalRows(), that.getMaxTotalRows()) && + Objects.equals(getIntermediateHandoffPeriod(), that.getIntermediateHandoffPeriod()); } @Override @@ -219,6 +222,7 @@ public int hashCode() getMaxRowsInMemory(), getMaxBytesInMemory(), getMaxRowsPerSegment(), + getMaxTotalRows(), getIntermediatePersistPeriod(), getBasePersistDirectory(), getMaxPendingPersists(), @@ -236,7 +240,9 @@ public int hashCode() getSegmentWriteOutMediumFactory(), isLogParseExceptions(), getMaxParseExceptions(), - getMaxSavedParseExceptions() + getMaxSavedParseExceptions(), + getMaxRecordsPerPoll(), + getIntermediateHandoffPeriod() ); } @@ -247,6 +253,7 @@ public String toString() "maxRowsInMemory=" + getMaxRowsInMemory() + ", maxBytesInMemory=" + getMaxBytesInMemory() + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + + ", maxTotalRows=" + getMaxTotalRows() + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + ", maxPendingPersists=" + getMaxPendingPersists() + @@ -265,6 +272,7 @@ public String toString() ", maxParseExceptions=" + getMaxParseExceptions() + ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + ", maxRecordsPerPoll=" + maxRecordsPerPoll + + ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() + '}'; } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 317612d7d760..11a831c9e9e3 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -244,6 +244,12 @@ private Runnable getRecordRunnable() rescheduleRunnable(fetchDelayMillis); } catch (ProvisionedThroughputExceededException e) { + log.warn( + e, + "encounted ProvisionedThroughputExceededException while fetching records, this means " + + "that the request rate for the stream is too high, or the requested data is too large for " + + "the available throughput. Reduce the frequency or size of your requests." + ); long retryMs = Math.max(PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS, fetchDelayMillis); rescheduleRunnable(retryMs); } @@ -563,7 +569,7 @@ public String getEarliestSequenceNumber(StreamPartition partition) @Override public String getPosition(StreamPartition partition) { - throw new UnsupportedOperationException("getPosition is not supported in Kinesiss"); + throw new UnsupportedOperationException("getPosition() is not supported in Kinesis"); } @Override @@ -690,9 +696,18 @@ private String getSequenceNumberInternal(StreamPartition partition, Stri GetRecordsResult recordsResult; try { + // we call getRecords with limit 1000 to make sure that we can find the first (earliest) record in the shard. + // In the case where the shard is constantly removing records that are past their retention period, it is possible + // that we never find the first record in the shard if we use a limit of 1. recordsResult = kinesis.getRecords(new GetRecordsRequest().withShardIterator(shardIterator).withLimit(1000)); } catch (ProvisionedThroughputExceededException e) { + log.warn( + e, + "encounted ProvisionedThroughputExceededException while fetching records, this means " + + "that the request rate for the stream is too high, or the requested data is too large for " + + "the available throughput. Reduce the frequency or size of your requests." + ); try { Thread.sleep(PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS); continue; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 24f68708aa48..dba7f1072fea 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -492,12 +492,13 @@ public void run() // calling getRecord() ensures that exceptions specific to kafka/kinesis like OffsetOutOfRangeException - // are handled in the subclasses + // are handled in the subclasses. List> records = getRecords( recordSupplier, toolbox ); + // note: getRecords() also updates assignment stillReading = !assignment.isEmpty(); SequenceMetadata sequenceToCheckpoint = null; @@ -516,14 +517,12 @@ public void run() continue; } - if (log.isTraceEnabled()) { - log.trace( - "Got stream[%s] partition[%s] sequence[%s].", - record.getStream(), - record.getPartitionId(), - record.getSequenceNumber() - ); - } + log.trace( + "Got stream[%s] partition[%s] sequence[%s].", + record.getStream(), + record.getPartitionId(), + record.getSequenceNumber() + ); if (isEndOfShard(record.getSequenceNumber())) { // shard is closed, applies to Kinesis only @@ -617,7 +616,7 @@ public void onSuccess(@Nullable Object result) } @Override - public void onFailure(@ParametersAreNonnullByDefault Throwable t) + public void onFailure(Throwable t) { log.error("Persist failed, dying"); backgroundThreadException = t; @@ -1622,15 +1621,14 @@ private class SequenceMetadata private final Set assignments; private final boolean sentinel; private boolean checkpointed; - /** * Lock for accessing {@link #endOffsets} and {@link #checkpointed}. This lock is required because * {@link #setEndOffsets)} can be called by both the main thread and the HTTP thread. */ - protected final ReentrantLock lock = new ReentrantLock(); - protected final Map startOffsets; - protected final Map endOffsets; + private final ReentrantLock lock = new ReentrantLock(); + final Map startOffsets; + final Map endOffsets; @JsonCreator public SequenceMetadata( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java index b2c1f8eeb69c..8c034da41fb4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamPartitions.java @@ -45,6 +45,10 @@ */ public class SeekableStreamPartitions { + // this special marker is used by the KinesisSupervisor to set the endOffsets + // of newly created indexing tasks. This is necessary because streaming tasks do not + // have endPartitionOffsets. This marker signals to the task that it should continue + // to ingest data until taskDuration has elapsed or the task was stopped or paused or killed public static final String NO_END_SEQUENCE_NUMBER = "NO_END_SEQUENCE_NUMBER"; // stream/topic From e16324ccb1bdafb577745004c9590a08e18da80d Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Wed, 19 Dec 2018 17:52:10 -0800 Subject: [PATCH 83/87] extra error handling in KinesisRecordSupplier getRecords --- .../druid/indexing/kinesis/KinesisRecordSupplier.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 11a831c9e9e3..aea9fe398a86 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.kinesis; import com.amazonaws.ClientConfiguration; +import com.amazonaws.SdkClientException; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider; import com.amazonaws.client.builder.AwsClientBuilder; @@ -290,9 +291,13 @@ private Runnable getRecordRunnable() log.error(e, "encounted AWS error while attempting to fetch records, will not retry"); throw e; } + catch (SdkClientException e) { + log.warn(e, "encounted unknown AWS exception, retrying in [%,dms]", EXCEPTION_RETRY_DELAY_MS); + rescheduleRunnable(EXCEPTION_RETRY_DELAY_MS); + } catch (Throwable e) { // non transient errors - log.error(e, "getRecordRunnable exception"); + log.error(e, "unknown getRecordRunnable exception, will not retry"); throw new RuntimeException(e); } From 7439a17d04f2541dc0d3882d08a5f68bc0946215 Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 20 Dec 2018 16:45:20 -0800 Subject: [PATCH 84/87] comments --- .../druid/indexing/kafka/KafkaIndexTask.java | 2 +- .../indexing/kafka/KafkaRecordSupplier.java | 4 +- .../kafka/supervisor/KafkaSupervisor.java | 4 +- .../indexing/kafka/KafkaIndexTaskTest.java | 1 - .../indexing/kinesis/KinesisIndexTask.java | 5 +- .../kinesis/KinesisIndexTaskIOConfig.java | 11 --- .../kinesis/KinesisRecordSupplier.java | 69 +++++++++---------- .../druid/indexing/kinesis/KinesisRegion.java | 4 +- .../ConstructibleAWSCredentialsConfig.java | 61 ---------------- .../kinesis/supervisor/KinesisSupervisor.java | 8 +-- .../supervisor/KinesisSupervisorSpec.java | 1 + .../KinesisSupervisorTuningConfig.java | 8 ++- .../indexing/kinesis/KinesisIOConfigTest.java | 3 - .../kinesis/KinesisIndexTaskClientTest.java | 14 ++-- .../kinesis/KinesisIndexTaskTest.java | 27 -------- .../indexing/kinesis/KinesisRegionTest.java | 7 ++ .../supervisor/KinesisSupervisorTest.java | 27 -------- .../supervisor/SupervisorManager.java | 7 +- .../SeekableStreamIndexTask.java | 8 +-- .../SeekableStreamIndexTaskIOConfig.java | 6 +- .../SeekableStreamIndexTaskRunner.java | 5 +- .../supervisor/SeekableStreamSupervisor.java | 20 ++---- .../SeekableStreamSupervisorIOConfig.java | 3 - ...SeekableStreamSupervisorReportPayload.java | 12 ++-- .../SeekableStreamSupervisorSpec.java | 4 -- .../SeekableStreamSupervisorTuningConfig.java | 3 - .../{RandomId.java => RandomIdUtils.java} | 2 +- 27 files changed, 89 insertions(+), 237 deletions(-) delete mode 100644 extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java rename indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/utils/{RandomId.java => RandomIdUtils.java} (97%) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 38608c288199..22ede084b3a9 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -70,7 +70,7 @@ public KafkaIndexTask( ) { super( - id == null ? getFormatedId(dataSchema.getDataSource(), TYPE) : id, + id == null ? getFormattedId(dataSchema.getDataSource(), TYPE) : id, taskResource, dataSchema, tuningConfig, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java index 844994038a6a..935404cbc7c2 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java @@ -25,7 +25,7 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; -import org.apache.druid.indexing.seekablestream.utils.RandomId; +import org.apache.druid.indexing.seekablestream.utils.RandomIdUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.metadata.PasswordProvider; @@ -195,7 +195,7 @@ private KafkaConsumer getKafkaConsumer() final Properties props = new Properties(); props.setProperty("metadata.max.age.ms", "10000"); - props.setProperty("group.id", StringUtils.format("kafka-supervisor-%s", RandomId.getRandomId())); + props.setProperty("group.id", StringUtils.format("kafka-supervisor-%s", RandomIdUtils.getRandomId())); addConsumerPropertiesFromConfig(props, sortingMapper, consumerProperties); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 6b52e40a1961..6f4434ea3e95 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -49,7 +49,7 @@ import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; -import org.apache.druid.indexing.seekablestream.utils.RandomId; +import org.apache.druid.indexing.seekablestream.utils.RandomIdUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -244,7 +244,7 @@ protected List> createIndexTasks( List> taskList = new ArrayList<>(); for (int i = 0; i < replicas; i++) { - String taskId = Joiner.on("_").join(baseSequenceName, RandomId.getRandomId()); + String taskId = Joiner.on("_").join(baseSequenceName, RandomIdUtils.getRandomId()); taskList.add(new KafkaIndexTask( taskId, new TaskResource(baseSequenceName, 1), diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 2ad94c54239c..72041eb947d5 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -420,7 +420,6 @@ public void testRunAfterDataInserted() throws Exception @Test(timeout = 60_000L) public void testRunBeforeDataInserted() throws Exception { - Map consumerProps = kafkaServer.consumerProperties(); final KafkaIndexTask task = createTask( null, new KafkaIndexTaskIOConfig( diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 2a86aa448d62..3467481762af 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -54,7 +54,7 @@ public KinesisIndexTask( ) { super( - id == null ? getFormatedId(dataSchema.getDataSource(), TYPE) : id, + id == null ? getFormattedId(dataSchema.getDataSource(), TYPE) : id, taskResource, dataSchema, tuningConfig, @@ -95,8 +95,7 @@ protected KinesisRecordSupplier newTaskRecordSupplier() return new KinesisRecordSupplier( KinesisRecordSupplier.getAmazonKinesisClient( ioConfig.getEndpoint(), - awsCredentialsConfig.getAccessKey().getPassword(), - awsCredentialsConfig.getSecretKey().getPassword(), + awsCredentialsConfig, ioConfig.getAwsAssumedRoleArn(), ioConfig.getAwsExternalId() ), diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java index 044ee0bfc653..307e971bcfd3 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java @@ -31,11 +31,9 @@ public class KinesisIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig { - private static final boolean DEFAULT_PAUSE_AFTER_READ = true; public static final int DEFAULT_RECORDS_PER_FETCH = 4000; public static final int DEFAULT_FETCH_DELAY_MILLIS = 0; - private final boolean pauseAfterRead; private final String endpoint; private final Integer recordsPerFetch; private final Integer fetchDelayMillis; @@ -51,7 +49,6 @@ public KinesisIndexTaskIOConfig( @JsonProperty("startPartitions") SeekableStreamPartitions startPartitions, @JsonProperty("endPartitions") SeekableStreamPartitions endPartitions, @JsonProperty("useTransaction") Boolean useTransaction, - @JsonProperty("pauseAfterRead") Boolean pauseAfterRead, @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, @JsonProperty("endpoint") String endpoint, @@ -79,7 +76,6 @@ public KinesisIndexTaskIOConfig( .stream() .noneMatch(x -> x.equals(KinesisSequenceNumber.END_OF_SHARD_MARKER))); - this.pauseAfterRead = pauseAfterRead != null ? pauseAfterRead : DEFAULT_PAUSE_AFTER_READ; this.endpoint = Preconditions.checkNotNull(endpoint, "endpoint"); this.recordsPerFetch = recordsPerFetch != null ? recordsPerFetch : DEFAULT_RECORDS_PER_FETCH; this.fetchDelayMillis = fetchDelayMillis != null ? fetchDelayMillis : DEFAULT_FETCH_DELAY_MILLIS; @@ -88,12 +84,6 @@ public KinesisIndexTaskIOConfig( this.deaggregate = deaggregate; } - @JsonProperty - public boolean isPauseAfterRead() - { - return pauseAfterRead; - } - @JsonProperty public String getEndpoint() { @@ -138,7 +128,6 @@ public String toString() ", startPartitions=" + getStartPartitions() + ", endPartitions=" + getEndPartitions() + ", useTransaction=" + isUseTransaction() + - ", pauseAfterRead=" + pauseAfterRead + ", minimumMessageTime=" + getMinimumMessageTime() + ", maximumMessageTime=" + getMaximumMessageTime() + ", endpoint='" + endpoint + '\'' + diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index aea9fe398a86..70c5719a745f 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -19,8 +19,8 @@ package org.apache.druid.indexing.kinesis; +import com.amazonaws.AmazonServiceException; import com.amazonaws.ClientConfiguration; -import com.amazonaws.SdkClientException; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider; import com.amazonaws.client.builder.AwsClientBuilder; @@ -41,8 +41,8 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Queues; +import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.common.aws.AWSCredentialsUtils; -import org.apache.druid.indexing.kinesis.aws.ConstructibleAWSCredentialsConfig; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; @@ -53,6 +53,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.io.IOException; import java.lang.invoke.MethodHandle; import java.lang.invoke.MethodHandles; import java.lang.reflect.Method; @@ -84,13 +85,17 @@ public class KinesisRecordSupplier implements RecordSupplier private static final long PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS = 3000; private static final long EXCEPTION_RETRY_DELAY_MS = 10000; + private static boolean isServiceExceptionRecoverable(AmazonServiceException ex) + { + final boolean isIOException = ex.getCause() instanceof IOException; + final boolean isTimeout = "RequestTimeout".equals(ex.getErrorCode()); + return isIOException || isTimeout; + } + private class PartitionResource { private final StreamPartition streamPartition; - // lock for cooradinating startBackground fetch, guards started - private final Object startLock = new Object(); - // shardIterator points to the record that will be polled next by recordRunnable // can be null when shard is closed due to the user shard splitting or changing the number // of shards in the stream, in which case a 'EOS' marker is used by the KinesisRecordSupplier @@ -109,22 +114,20 @@ private class PartitionResource void startBackgroundFetch() { - synchronized (startLock) { - if (started) { - return; - } + if (started) { + return; + } - log.info( - "Starting scheduled fetch runnable for stream[%s] partition[%s]", - streamPartition.getStream(), - streamPartition.getPartitionId() - ); + log.info( + "Starting scheduled fetch runnable for stream[%s] partition[%s]", + streamPartition.getStream(), + streamPartition.getPartitionId() + ); - stopRequested = false; - started = true; + stopRequested = false; + started = true; - rescheduleRunnable(fetchDelayMillis); - } + rescheduleRunnable(fetchDelayMillis); } void stopBackgroundFetch() @@ -152,7 +155,7 @@ private Runnable getRecordRunnable() // used for retrying on InterruptedException GetRecordsResult recordsResult = null; - OrderedPartitionableRecord currRecord = null; + OrderedPartitionableRecord currRecord; try { @@ -261,17 +264,7 @@ private Runnable getRecordRunnable() "Interrupted while waiting to add record to buffer, retrying in [%,dms]", EXCEPTION_RETRY_DELAY_MS ); - if (currRecord != null) { - shardIterator = kinesis.getShardIterator( - currRecord.getStream(), - currRecord.getPartitionId(), - ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), - currRecord.getSequenceNumber() - ).getShardIterator(); - rescheduleRunnable(EXCEPTION_RETRY_DELAY_MS); - } else { - throw new ISE("can't reschedule fetch records runnable, current record is null??"); - } + rescheduleRunnable(EXCEPTION_RETRY_DELAY_MS); } catch (ExpiredIteratorException e) { log.warn( @@ -291,9 +284,14 @@ private Runnable getRecordRunnable() log.error(e, "encounted AWS error while attempting to fetch records, will not retry"); throw e; } - catch (SdkClientException e) { - log.warn(e, "encounted unknown AWS exception, retrying in [%,dms]", EXCEPTION_RETRY_DELAY_MS); - rescheduleRunnable(EXCEPTION_RETRY_DELAY_MS); + catch (AmazonServiceException e) { + if (isServiceExceptionRecoverable(e)) { + log.warn(e, "encounted unknown recoverable AWS exception, retrying in [%,dms]", EXCEPTION_RETRY_DELAY_MS); + rescheduleRunnable(EXCEPTION_RETRY_DELAY_MS); + } else { + log.warn(e, "encounted unknown unrecoverable AWS exception, will not retry"); + throw new RuntimeException(e); + } } catch (Throwable e) { // non transient errors @@ -416,14 +414,13 @@ public KinesisRecordSupplier( public static AmazonKinesis getAmazonKinesisClient( String endpoint, - String awsAccessKeyId, - String awsSecretAccessKey, + AWSCredentialsConfig awsCredentialsConfig, String awsAssumedRoleArn, String awsExternalId ) { AWSCredentialsProvider awsCredentialsProvider = AWSCredentialsUtils.defaultAWSCredentialsProviderChain( - new ConstructibleAWSCredentialsConfig(awsAccessKeyId, awsSecretAccessKey) + awsCredentialsConfig ); if (awsAssumedRoleArn != null) { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java index 2b9fbd758e94..a5e1195616f9 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRegion.java @@ -62,12 +62,12 @@ public static KinesisRegion fromString(String value) private static List getNames() { - return EnumSet.allOf(KinesisRegion.class).stream().map(KinesisRegion::toString).collect(Collectors.toList()); + return EnumSet.allOf(KinesisRegion.class).stream().map(KinesisRegion::serialize).collect(Collectors.toList()); } public String getEndpoint() { - return StringUtils.format("kinesis.%s.amazonaws.com%s", toString(), toString().startsWith("cn-") ? ".cn" : ""); + return StringUtils.format("kinesis.%s.amazonaws.com%s", serialize(), serialize().startsWith("cn-") ? ".cn" : ""); } @JsonValue diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java deleted file mode 100644 index 3a244abde962..000000000000 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/aws/ConstructibleAWSCredentialsConfig.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.kinesis.aws; - -import org.apache.druid.common.aws.AWSCredentialsConfig; -import org.apache.druid.metadata.DefaultPasswordProvider; -import org.apache.druid.metadata.PasswordProvider; - -public class ConstructibleAWSCredentialsConfig extends AWSCredentialsConfig -{ - private final String accessKey; - private final String secretKey; - private final String fileSessionCredentials; - - public ConstructibleAWSCredentialsConfig(String accessKey, String secretKey) - { - this(accessKey, secretKey, null); - } - - public ConstructibleAWSCredentialsConfig(String accessKey, String secretKey, String fileSessionCredentials) - { - this.accessKey = accessKey != null ? accessKey : ""; - this.secretKey = secretKey != null ? secretKey : ""; - this.fileSessionCredentials = fileSessionCredentials != null ? fileSessionCredentials : ""; - } - - @Override - public PasswordProvider getAccessKey() - { - return DefaultPasswordProvider.fromString(accessKey); - } - - @Override - public PasswordProvider getSecretKey() - { - return DefaultPasswordProvider.fromString(secretKey); - } - - @Override - public String getFileSessionCredentials() - { - return fileSessionCredentials; - } -} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 1829a0db6536..f53bdf2ad129 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -50,7 +50,7 @@ import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; -import org.apache.druid.indexing.seekablestream.utils.RandomId; +import org.apache.druid.indexing.seekablestream.utils.RandomIdUtils; import org.apache.druid.java.util.common.StringUtils; import org.joda.time.DateTime; @@ -123,7 +123,6 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( new SeekableStreamPartitions<>(ioConfig.getStream(), startPartitions), new SeekableStreamPartitions<>(ioConfig.getStream(), endPartitions), true, - true, // should pause after reading otherwise the task may complete early which will confuse the supervisor minimumMessageTime, maximumMessageTime, ioConfig.getEndpoint(), @@ -163,7 +162,7 @@ protected List> createIndexTasks( .build(); List> taskList = new ArrayList<>(); for (int i = 0; i < replicas; i++) { - String taskId = Joiner.on("_").join(baseSequenceName, RandomId.getRandomId()); + String taskId = Joiner.on("_").join(baseSequenceName, RandomIdUtils.getRandomId()); taskList.add(new KinesisIndexTask( taskId, new TaskResource(baseSequenceName, 1), @@ -191,8 +190,7 @@ protected RecordSupplier setupRecordSupplier() return new KinesisRecordSupplier( KinesisRecordSupplier.getAmazonKinesisClient( ioConfig.getEndpoint(), - awsCredentialsConfig.getAccessKey().getPassword(), - awsCredentialsConfig.getSecretKey().getPassword(), + awsCredentialsConfig, ioConfig.getAwsAssumedRoleArn(), ioConfig.getAwsExternalId() ), diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java index 1feb8f63361e..479851f8eb8c 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -135,6 +135,7 @@ public String toString() ", tuningConfig=" + getTuningConfig() + ", ioConfig=" + getIoConfig() + ", suspended=" + isSuspended() + + ", context=" + getContext() + '}'; } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 7a5abb31f7f7..3c749be5cffc 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -149,12 +149,13 @@ public String toString() { return "KinesisSupervisorTuningConfig{" + "maxRowsInMemory=" + getMaxRowsInMemory() + + ", maxBytesInMemory=" + getMaxBytesInMemory() + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + + ", maxTotalRows=" + getMaxTotalRows() + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + ", maxPendingPersists=" + getMaxPendingPersists() + ", indexSpec=" + getIndexSpec() + - ", buildV9Directly=" + true + ", reportParseExceptions=" + isReportParseExceptions() + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + @@ -170,6 +171,11 @@ public String toString() ", fetchSequenceNumberTimeout=" + getFetchSequenceNumberTimeout() + ", fetchThreads=" + getFetchThreads() + ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() + + ", logParseExceptions=" + isLogParseExceptions() + + ", maxParseExceptions=" + getMaxParseExceptions() + + ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + + ", maxRecordsPerPoll=" + getMaxRecordsPerPoll() + + ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() + '}'; } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index 6ddd1c1a098b..fff34f946b26 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -81,7 +81,6 @@ public void testSerdeWithDefaults() throws Exception config.getEndPartitions().getPartitionSequenceNumberMap() ); Assert.assertTrue(config.isUseTransaction()); - Assert.assertTrue(config.isPauseAfterRead()); Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent()); Assert.assertEquals(config.getEndpoint(), "kinesis.us-east-1.amazonaws.com"); Assert.assertEquals(config.getRecordsPerFetch(), 4000); @@ -103,7 +102,6 @@ public void testSerdeWithNonDefaults() throws Exception + " \"startPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"1\", \"1\":\"10\"}},\n" + " \"endPartitions\": {\"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"0\":\"15\", \"1\":\"200\"}},\n" + " \"useTransaction\": false,\n" - + " \"pauseAfterRead\": false,\n" + " \"minimumMessageTime\": \"2016-05-31T12:00Z\",\n" + " \"maximumMessageTime\": \"2016-05-31T14:00Z\",\n" + " \"endpoint\": \"kinesis.us-east-2.amazonaws.com\",\n" @@ -137,7 +135,6 @@ public void testSerdeWithNonDefaults() throws Exception config.getEndPartitions().getPartitionSequenceNumberMap() ); Assert.assertFalse(config.isUseTransaction()); - Assert.assertFalse(config.isPauseAfterRead()); Assert.assertTrue("maximumMessageTime", config.getMaximumMessageTime().isPresent()); Assert.assertTrue("minimumMessageTime", config.getMinimumMessageTime().isPresent()); Assert.assertEquals(DateTimes.of("2016-05-31T12:00Z"), config.getMinimumMessageTime().get()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java index 72a6ec527ab2..9d7fafbb4c73 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskClientTest.java @@ -114,7 +114,7 @@ public void setUp() response = createMock(HttpResponse.class); headers = createMock(HttpHeaders.class); - client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider); + client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider); expect(taskInfoProvider.getTaskLocation(TEST_ID)) .andReturn(new TaskLocation(TEST_HOST, TEST_PORT, TEST_TLS_PORT)) .anyTimes(); @@ -287,7 +287,7 @@ public void testGetCurrentOffsets() throws Exception @Test public void testGetCurrentOffsetsWithRetry() throws Exception { - client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 3); + client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 3); Capture captured = Capture.newInstance(CaptureType.ALL); expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).times(6) @@ -332,7 +332,7 @@ public void testGetCurrentOffsetsWithExhaustedRetries() expectedException.expect(RuntimeException.class); expectedException.expectMessage("org.apache.druid.java.util.common.IOE: Received status [404]"); - client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); + client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); expect(responseHolder.getStatus()).andReturn(HttpResponseStatus.NOT_FOUND).anyTimes(); expect(responseHolder.getContent()).andReturn("").anyTimes(); @@ -387,7 +387,7 @@ public void testGetEndOffsets() throws Exception @Test public void testGetStartTime() throws Exception { - client = new TestableKafkaIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); + client = new TestableKinesisIndexTaskClient(httpClient, objectMapper, taskInfoProvider, 2); DateTime now = DateTimes.nowUtc(); Capture captured = Capture.newInstance(); @@ -1030,9 +1030,9 @@ public void testSetEndOffsetsAsyncWithResume() throws Exception } } - private class TestableKafkaIndexTaskClient extends KinesisIndexTaskClient + private class TestableKinesisIndexTaskClient extends KinesisIndexTaskClient { - public TestableKafkaIndexTaskClient( + TestableKinesisIndexTaskClient( HttpClient httpClient, ObjectMapper jsonMapper, TaskInfoProvider taskInfoProvider @@ -1041,7 +1041,7 @@ public TestableKafkaIndexTaskClient( this(httpClient, jsonMapper, taskInfoProvider, TEST_NUM_RETRIES); } - public TestableKafkaIndexTaskClient( + TestableKinesisIndexTaskClient( HttpClient httpClient, ObjectMapper jsonMapper, TaskInfoProvider taskInfoProvider, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 870b74eb879e..cef97955db2e 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -195,7 +195,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport { private static final Logger log = new Logger(KinesisIndexTaskTest.class); private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); - // private static final long POLL_RETRY_MS = 100; private static String stream = "stream"; private static String shardId1 = "1"; private static String shardId0 = "0"; @@ -383,7 +382,6 @@ public void testRunAfterDataInserted() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -465,7 +463,6 @@ public void testRunBeforeDataInserted() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -575,7 +572,6 @@ public void testIncrementalHandOff() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -715,7 +711,6 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -834,7 +829,6 @@ public void testRunWithMinimumMessageTime() throws Exception "4" )), true, - null, DateTimes.of("2010"), null, "awsEndpoint", @@ -919,7 +913,6 @@ public void testRunWithMaximumMessageTime() throws Exception )), true, null, - null, DateTimes.of("2010"), "awsEndpoint", null, @@ -1014,7 +1007,6 @@ public void testRunWithTransformSpec() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -1095,7 +1087,6 @@ public void testRunOnNothing() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -1159,7 +1150,6 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -1237,7 +1227,6 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio true, null, null, - null, "awsEndpoint", null, null, @@ -1315,7 +1304,6 @@ public void testReportParseExceptions() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -1382,7 +1370,6 @@ public void testMultipleParseExceptionsSuccess() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -1493,7 +1480,6 @@ public void testMultipleParseExceptionsFailure() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -1583,7 +1569,6 @@ public void testRunReplicas() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -1609,7 +1594,6 @@ public void testRunReplicas() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -1696,7 +1680,6 @@ public void testRunConflicting() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -1722,7 +1705,6 @@ public void testRunConflicting() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -1809,7 +1791,6 @@ public void testRunConflictingWithoutTransactions() throws Exception false, null, null, - null, "awsEndpoint", null, null, @@ -1835,7 +1816,6 @@ public void testRunConflictingWithoutTransactions() throws Exception false, null, null, - null, "awsEndpoint", null, null, @@ -1923,7 +1903,6 @@ public void testRunOneTaskTwoPartitions() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -2014,7 +1993,6 @@ public void testRunTwoTasksTwoPartitions() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -2040,7 +2018,6 @@ public void testRunTwoTasksTwoPartitions() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -2130,7 +2107,6 @@ public void testRestore() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -2190,7 +2166,6 @@ public void testRestore() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -2275,7 +2250,6 @@ public void testRunWithPauseAndResume() throws Exception true, null, null, - null, "awsEndpoint", null, null, @@ -2414,7 +2388,6 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception true, null, null, - null, "awsEndpoint", null, null, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRegionTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRegionTest.java index 350c4c087415..b20cef3fc333 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRegionTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRegionTest.java @@ -60,6 +60,13 @@ public void testSerde() throws IOException Assert.assertEquals(kinesisRegion, KinesisRegion.US_EAST_1); } + @Test + public void testGetEndpoint() + { + Assert.assertEquals("kinesis.cn-north-1.amazonaws.com.cn", KinesisRegion.CN_NORTH_1.getEndpoint()); + Assert.assertEquals("kinesis.us-east-1.amazonaws.com", KinesisRegion.US_EAST_1.getEndpoint()); + } + @Test(expected = JsonMappingException.class) public void testBadSerde() throws IOException { diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index f9dab5407fd9..750b9247d792 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -57,7 +57,6 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; import org.apache.druid.indexing.seekablestream.SeekableStreamPartitions; -import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; @@ -114,7 +113,6 @@ public class KinesisSupervisorTest extends EasyMockSupport private static final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); - private static final String TOPIC_PREFIX = "testTopic"; private static final String DATASOURCE = "testDS"; private static final int TEST_CHAT_THREADS = 3; private static final long TEST_CHAT_RETRIES = 9L; @@ -125,31 +123,8 @@ public class KinesisSupervisorTest extends EasyMockSupport private static String shardId0 = "0"; private static StreamPartition shard1Partition = StreamPartition.of(stream, shardId1); private static StreamPartition shard0Partition = StreamPartition.of(stream, shardId0); - private static List> records = ImmutableList.of( - new OrderedPartitionableRecord<>(stream, "1", "0", JB("2008", "a", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "1", JB("2009", "b", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "2", JB("2010", "c", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "3", JB("2011", "d", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "4", JB("2011", "e", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>( - stream, - "1", - "5", - JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0") - ), - new OrderedPartitionableRecord<>(stream, "1", "6", Collections.singletonList(StringUtils.toUtf8("unparseable"))), - new OrderedPartitionableRecord<>(stream, "1", "7", Collections.singletonList(StringUtils.toUtf8("unparseable2"))), - new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(StringUtils.toUtf8("{}"))), - new OrderedPartitionableRecord<>(stream, "1", "9", JB("2013", "f", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "10", JB("2049", "f", "y", "notanumber", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "11", JB("2049", "f", "y", "10", "notanumber", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "12", JB("2049", "f", "y", "10", "20.0", "notanumber")), - new OrderedPartitionableRecord<>(stream, "0", "0", JB("2012", "g", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "0", "1", JB("2011", "h", "y", "10", "20.0", "1.0")) - ); private static DataSchema dataSchema; - private KinesisRecordSupplier taskRecordSupplier; private KinesisRecordSupplier supervisorRecordSupplier; private final int numThreads; @@ -185,7 +160,6 @@ public void setupTest() taskClient = createMock(KinesisIndexTaskClient.class); taskQueue = createMock(TaskQueue.class); supervisorRecordSupplier = createMock(KinesisRecordSupplier.class); - taskRecordSupplier = mock(KinesisRecordSupplier.class); tuningConfig = new KinesisSupervisorTuningConfig( 1000, @@ -3633,7 +3607,6 @@ private KinesisIndexTask createKinesisIndexTask( startPartitions, endPartitions, true, - null, minimumMessageTime, maximumMessageTime, "awsEndpoint", diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 57686aa6a31d..b47ec8901693 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -70,7 +70,7 @@ public boolean createOrUpdateAndStartSupervisor(SupervisorSpec spec) { Preconditions.checkState(started, "SupervisorManager not started"); Preconditions.checkNotNull(spec, "spec"); - Preconditions.checkNotNull(spec.getId(), "spec.getStream()"); + Preconditions.checkNotNull(spec.getId(), "spec.getId()"); Preconditions.checkNotNull(spec.getDataSources(), "spec.getDatasources()"); synchronized (lock) { @@ -241,7 +241,10 @@ private boolean possiblyStopAndRemoveSupervisorInternal(String id, boolean write } if (writeTombstone) { - metadataSupervisorManager.insert(id, new NoopSupervisorSpec(null, pair.rhs.getDataSources())); // where NoopSupervisorSpec is a tombstone + metadataSupervisorManager.insert( + id, + new NoopSupervisorSpec(null, pair.rhs.getDataSources()) + ); // where NoopSupervisorSpec is a tombstone } pair.lhs.stop(true); supervisors.remove(id); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index a9873239a06d..2e6ffa522fe6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -39,6 +39,7 @@ import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.utils.RandomIdUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; @@ -120,14 +121,11 @@ public SeekableStreamIndexTask( private static String makeTaskId(String dataSource, int randomBits, String type) { - final StringBuilder suffix = new StringBuilder(8); - for (int i = 0; i < Integer.BYTES * 2; ++i) { - suffix.append((char) ('a' + ((randomBits >>> (i * 4)) & 0x0F))); - } + final String suffix = RandomIdUtils.getRandomId(); return Joiner.on("_").join(type, dataSource, suffix); } - protected static String getFormatedId(String dataSource, String type) + protected static String getFormattedId(String dataSource, String type) { return makeTaskId(dataSource, RANDOM.nextInt(), type); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java index c4a79898ec1b..6c469c7d0123 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java @@ -56,7 +56,8 @@ public SeekableStreamIndexTaskIOConfig( @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, @JsonProperty("skipOffsetGaps") Boolean skipOffsetGaps, - @JsonProperty("exclusiveStartSequenceNumberPartitions") Set exclusiveStartSequenceNumberPartitions + @JsonProperty("exclusiveStartSequenceNumberPartitions") + Set exclusiveStartSequenceNumberPartitions ) { this.taskGroupId = taskGroupId; @@ -141,7 +142,4 @@ public boolean isSkipOffsetGaps() { return skipOffsetGaps; } - - @Override - public abstract String toString(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index dba7f1072fea..e85de9d8db68 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -86,7 +86,6 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; -import javax.annotation.ParametersAreNonnullByDefault; import javax.servlet.http.HttpServletRequest; import javax.validation.constraints.NotNull; import javax.ws.rs.Consumes; @@ -938,7 +937,7 @@ public Void apply(@Nullable SegmentsAndMetadata handoffSegmentsAndMetadata) } @Override - public void onFailure(@ParametersAreNonnullByDefault Throwable t) + public void onFailure(Throwable t) { log.error(t, "Error while publishing segments for sequence[%s]", sequenceMetadata); handoffFuture.setException(t); @@ -2008,7 +2007,7 @@ protected abstract SeekableStreamDataSourceMetadata createSequenceNumber(SequenceOffsetType sequenceNumber); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 84c624deb53b..ddd854d3bdad 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -108,9 +108,6 @@ * this class is the parent class of both the Kafka and Kinesis supervisor. All the main run loop * logic are similar enough so they're grouped together into this class. *

      - * incremental handoff & checkpointing are not yet supported by Kinesis, but the logic is left in here - * so in the future it's easier to implement - *

      * Supervisor responsible for managing the SeekableStreamIndexTasks (Kafka/Kinesis) for a single dataSource. At a high level, the class accepts a * {@link SeekableStreamSupervisorSpec} which includes the stream name (topic / stream) and configuration as well as an ingestion spec which will * be used to generate the indexing tasks. The run loop periodically refreshes its view of the stream's partitions @@ -427,13 +424,13 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) } - // Map<{group RandomId}, {actively reading task group}>; see documentation for TaskGroup class + // Map<{group RandomIdUtils}, {actively reading task group}>; see documentation for TaskGroup class private final ConcurrentHashMap activelyReadingTaskGroups = new ConcurrentHashMap<>(); // After telling a taskGroup to stop reading and begin publishing a segment, it is moved from [activelyReadingTaskGroups] to here so // we can monitor its status while we queue new tasks to read the next range of sequences. This is a list since we could // have multiple sets of tasks publishing at once if time-to-publish > taskDuration. - // Map<{group RandomId}, List<{pending completion task groups}>> + // Map<{group RandomIdUtils}, List<{pending completion task groups}>> private final ConcurrentHashMap> pendingCompletionTaskGroups = new ConcurrentHashMap<>(); // The starting sequence for a new partition in [partitionGroups] is initially set to getNotSetMarker(). When a new task group @@ -445,7 +442,7 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) // cause successive tasks to again grab their starting sequence from metadata store. This mechanism allows us to // start up successive tasks without waiting for the previous tasks to succeed and still be able to handle task // failures during publishing. - // Map<{group RandomId}, Map<{partition RandomId}, {startingOffset}>> + // Map<{group RandomIdUtils}, Map<{partition RandomIdUtils}, {startingOffset}>> private final ConcurrentHashMap> partitionGroups = new ConcurrentHashMap<>(); protected final ObjectMapper sortingMapper; @@ -970,7 +967,7 @@ public void addTaskGroupToActivelyReadingTaskGroup( group.tasks.putAll(tasks.stream().collect(Collectors.toMap(x -> x, x -> new TaskData()))); if (activelyReadingTaskGroups.putIfAbsent(taskGroupId, group) != null) { throw new ISE( - "trying to add taskGroup with RandomId [%s] to actively reading task groups, but group already exists.", + "trying to add taskGroup with RandomIdUtils [%s] to actively reading task groups, but group already exists.", taskGroupId ); } @@ -1840,15 +1837,6 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } } - // earlyPublishTime is only supported in Kinesis - // for now it is not implemented - // boolean doEarlyPublish = false; - // if (earlyPublishTime != null && (earlyPublishTime.isBeforeNow() || earlyPublishTime.isEqualNow())) { - // log.info("Early publish requested - signalling tasks to publish"); - // earlyPublishTime = null; - // doEarlyPublish = true; - // } - // if this task has run longer than the configured duration, signal all tasks in the group to persist if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { log.info("Task group [%d] has run for [%s]", groupId, ioConfig.getTaskDuration()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java index f758f68b00de..e62f675d7fc5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java @@ -131,7 +131,4 @@ public Optional getLateMessageRejectionPeriod() { return lateMessageRejectionPeriod; } - - @Override - public abstract String toString(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java index f221d7ea60b4..29547c335a66 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorReportPayload.java @@ -33,7 +33,7 @@ public abstract class SeekableStreamSupervisorReportPayload { private final String dataSource; - private final String id; + private final String stream; private final int partitions; private final int replicas; private final long durationSeconds; @@ -47,7 +47,7 @@ public abstract class SeekableStreamSupervisorReportPayload Date: Thu, 20 Dec 2018 17:08:31 -0800 Subject: [PATCH 85/87] quickfix --- docs/content/development/extensions-core/kinesis-ingestion.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/development/extensions-core/kinesis-ingestion.md b/docs/content/development/extensions-core/kinesis-ingestion.md index 631857d71b81..8b3583b0d496 100644 --- a/docs/content/development/extensions-core/kinesis-ingestion.md +++ b/docs/content/development/extensions-core/kinesis-ingestion.md @@ -217,7 +217,7 @@ To authenticate with AWS, you must provide your AWS access key and AWS secret ke ``` -Ddruid.kinesis.accessKey=123 -Ddruid.kinesis.secretKey=456 ``` -The AWS access key ID ad secret access key are used for Kinesis API requests. If this is not provided, the service will look for credentials set in environment variables, in the default profile configuration file, and from the EC2 instance profile provider (in this order). +The AWS access key ID and secret access key are used for Kinesis API requests. If this is not provided, the service will look for credentials set in environment variables, in the default profile configuration file, and from the EC2 instance profile provider (in this order). ### Getting Supervisor Status Report From 913ec668942eada7389d53fec59960d4f5043d7a Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 20 Dec 2018 17:14:45 -0800 Subject: [PATCH 86/87] typo --- .../apache/druid/indexing/kinesis/KinesisRecordSupplier.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index 70c5719a745f..622c1f37bbc3 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -706,9 +706,10 @@ private String getSequenceNumberInternal(StreamPartition partition, Stri catch (ProvisionedThroughputExceededException e) { log.warn( e, - "encounted ProvisionedThroughputExceededException while fetching records, this means " + "encountered ProvisionedThroughputExceededException while fetching records, this means " + "that the request rate for the stream is too high, or the requested data is too large for " - + "the available throughput. Reduce the frequency or size of your requests." + + "the available throughput. Reduce the frequency or size of your requests. Consider increasing " + + "the number of shards to increase throughput." ); try { Thread.sleep(PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS); From 6df49e4b67bcf34eeb2395895e860053b3d963cf Mon Sep 17 00:00:00 2001 From: Joshua Sun Date: Thu, 20 Dec 2018 17:26:23 -0800 Subject: [PATCH 87/87] oof --- .../indexing/seekablestream/SeekableStreamIndexTask.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 2e6ffa522fe6..7f279917a3f1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -119,7 +119,7 @@ public SeekableStreamIndexTask( } - private static String makeTaskId(String dataSource, int randomBits, String type) + private static String makeTaskId(String dataSource, String type) { final String suffix = RandomIdUtils.getRandomId(); return Joiner.on("_").join(type, dataSource, suffix); @@ -127,7 +127,7 @@ private static String makeTaskId(String dataSource, int randomBits, String type) protected static String getFormattedId(String dataSource, String type) { - return makeTaskId(dataSource, RANDOM.nextInt(), type); + return makeTaskId(dataSource, type); } protected static String getFormattedGroupId(String dataSource, String type)