From aa58950ee3b8487e9892d9638e893764cf3cc345 Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Thu, 6 May 2021 14:44:46 -0700 Subject: [PATCH 01/20] SAMZA-2591: Async Commit [1/3]: Checkpoint v2 migration (#1489) Introduce a new checkpoint message format (CheckpointV2) that includes state markers (kafka changelog offset / ambry blob id) in them. Renamed previous checkpoint messages as CheckpointV1 --- .../apache/samza/checkpoint/Checkpoint.java | 57 ++----- .../apache/samza/checkpoint/CheckpointId.java | 51 ++++-- .../apache/samza/checkpoint/CheckpointV1.java | 83 ++++++++++ .../apache/samza/checkpoint/CheckpointV2.java | 123 +++++++++++++++ .../serializers/JsonCheckpointIdMixin.java | 42 +++++ .../samza/checkpoint/TestCheckpointId.java | 53 +++++++ .../azure/AzureCheckpointManager.java | 10 +- .../azure/ITestAzureCheckpointManager.java | 15 +- .../kafka/KafkaChangelogSSPOffset.java | 40 +++-- .../kafka/KafkaStateCheckpointMarker.java | 145 +++++++++++++++++ .../org/apache/samza/config/JobConfig.java | 28 ++++ .../apache/samza/config/StorageConfig.java | 44 ++++++ .../org/apache/samza/config/TaskConfig.java | 37 +++++ .../samza/serializers/CheckpointV2Serde.java | 59 +++++++ .../samza/serializers/JsonCheckpoint.java | 57 +++++++ .../model/JsonCheckpointV2Mixin.java | 49 ++++++ .../KafkaStateCheckpointMarkerMixin.java | 48 ++++++ .../serializers/model/SamzaObjectMapper.java | 23 +++ .../samza/checkpoint/CheckpointTool.scala | 5 +- .../samza/checkpoint/OffsetManager.scala | 30 ++-- .../file/FileSystemCheckpointManager.scala | 10 +- ...intSerde.scala => CheckpointV1Serde.scala} | 69 +++----- .../org/apache/samza/util/FileUtil.scala | 9 +- .../kafka/TestKafkaChangelogSSPOffset.java | 73 +++++++++ .../kafka/TestKafkaStateCheckpointMarker.java | 114 ++++++++++++++ .../serializers/TestCheckpointV2Serde.java | 96 +++++++++++ .../samza/checkpoint/TestCheckpointTool.scala | 12 +- .../samza/checkpoint/TestOffsetManager.scala | 30 ++-- .../TestFileSystemCheckpointManager.scala | 10 +- ...erde.scala => TestCheckpointV1Serde.scala} | 26 +-- .../org/apache/samza/util/TestFileUtil.scala | 49 +++++- .../kafka/KafkaCheckpointLogKey.java | 8 +- .../kafka/KafkaCheckpointLogKeySerde.java | 12 +- .../kafka/KafkaCheckpointManager.scala | 105 ++++++++---- .../kafka/TestKafkaCheckpointLogKeySerde.java | 14 +- .../kafka/TestKafkaCheckpointManagerJava.java | 29 ++-- .../kafka/TestKafkaCheckpointManager.scala | 129 +++++++++++++-- .../CheckpointVersionIntegrationTest.java | 149 ++++++++++++++++++ .../kv/TransactionalStateIntegrationTest.java | 1 + ...ctionalStateMultiStoreIntegrationTest.java | 1 + 40 files changed, 1675 insertions(+), 270 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointV1.java create mode 100644 samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointV2.java create mode 100644 samza-api/src/main/java/org/apache/samza/serializers/JsonCheckpointIdMixin.java create mode 100644 samza-api/src/test/java/org/apache/samza/checkpoint/TestCheckpointId.java rename samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointedChangelogOffset.java => samza-core/src/main/java/org/apache/samza/checkpoint/kafka/KafkaChangelogSSPOffset.java (57%) create mode 100644 samza-core/src/main/java/org/apache/samza/checkpoint/kafka/KafkaStateCheckpointMarker.java create mode 100644 samza-core/src/main/java/org/apache/samza/serializers/CheckpointV2Serde.java create mode 100644 samza-core/src/main/java/org/apache/samza/serializers/JsonCheckpoint.java create mode 100644 samza-core/src/main/java/org/apache/samza/serializers/model/JsonCheckpointV2Mixin.java create mode 100644 samza-core/src/main/java/org/apache/samza/serializers/model/KafkaStateCheckpointMarkerMixin.java rename samza-core/src/main/scala/org/apache/samza/serializers/{CheckpointSerde.scala => CheckpointV1Serde.scala} (54%) create mode 100644 samza-core/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaChangelogSSPOffset.java create mode 100644 samza-core/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaStateCheckpointMarker.java create mode 100644 samza-core/src/test/java/org/apache/samza/serializers/TestCheckpointV2Serde.java rename samza-core/src/test/scala/org/apache/samza/serializers/{TestCheckpointSerde.scala => TestCheckpointV1Serde.scala} (69%) create mode 100644 samza-test/src/test/java/org/apache/samza/checkpoint/CheckpointVersionIntegrationTest.java diff --git a/samza-api/src/main/java/org/apache/samza/checkpoint/Checkpoint.java b/samza-api/src/main/java/org/apache/samza/checkpoint/Checkpoint.java index 593d118724..5f9276f4e9 100644 --- a/samza-api/src/main/java/org/apache/samza/checkpoint/Checkpoint.java +++ b/samza-api/src/main/java/org/apache/samza/checkpoint/Checkpoint.java @@ -19,54 +19,27 @@ package org.apache.samza.checkpoint; +import java.util.Map; import org.apache.samza.system.SystemStreamPartition; -import java.util.Collections; -import java.util.Map; - -/** - * A checkpoint is a mapping of all the streams a job is consuming and the most recent current offset for each. - * It is used to restore a {@link org.apache.samza.task.StreamTask}, either as part of a job restart or as part - * of restarting a failed container within a running job. - */ -public class Checkpoint { - private final Map offsets; +public interface Checkpoint { /** - * Constructs a new checkpoint based off a map of Samza stream offsets. - * @param offsets Map of Samza streams to their current offset. + * Gets the version number of the Checkpoint + * @return Short indicating the version number */ - public Checkpoint(Map offsets) { - this.offsets = offsets; - } + short getVersion(); /** - * Gets a unmodifiable view of the current Samza stream offsets. - * @return A unmodifiable view of a Map of Samza streams to their recorded offsets. + * Gets a unmodifiable view of the last processed offsets for {@link SystemStreamPartition}s. + * The returned value differs based on the Checkpoint version: + *
    + *
  1. For {@link CheckpointV1}, returns the input {@link SystemStreamPartition} offsets, as well + * as the latest KafkaStateChangelogOffset for any store changelog {@link SystemStreamPartition}
  2. + *
  3. For {@link CheckpointV2} returns the input offsets only.
  4. + *
+ * + * @return a unmodifiable view of last processed offsets for {@link SystemStreamPartition}s. */ - public Map getOffsets() { - return Collections.unmodifiableMap(offsets); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (!(o instanceof Checkpoint)) return false; - - Checkpoint that = (Checkpoint) o; - - if (offsets != null ? !offsets.equals(that.offsets) : that.offsets != null) return false; - - return true; - } - - @Override - public int hashCode() { - return offsets != null ? offsets.hashCode() : 0; - } - - @Override - public String toString() { - return "Checkpoint [offsets=" + offsets + "]"; - } + Map getOffsets(); } diff --git a/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointId.java b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointId.java index 95dfd2422f..f2aa8e1f30 100644 --- a/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointId.java +++ b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointId.java @@ -16,54 +16,64 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.samza.checkpoint; import java.util.Objects; import org.apache.commons.lang3.StringUtils; import org.apache.samza.annotation.InterfaceStability; + /** * Checkpoint ID has the format: [currentTimeMillis, last 6 digits of nanotime], separated by a dash. * This is to avoid conflicts, e.g when requesting frequent manual commits. * - * It is expected that persistent stores use the {@link #toString()} representation of the checkpoint id + * It is expected that persistent stores use the {@link #serialize()} representation of the checkpoint id * as the store checkpoint directory name. */ @InterfaceStability.Unstable -public class CheckpointId { +public class CheckpointId implements Comparable { public static final String SEPARATOR = "-"; private final long millis; - private final long nanos; + private final long nanoId; - public CheckpointId(long millis, long nanos) { + private CheckpointId(long millis, long nanoId) { this.millis = millis; - this.nanos = nanos; + this.nanoId = nanoId; } public static CheckpointId create() { return new CheckpointId(System.currentTimeMillis(), System.nanoTime() % 1000000); } - public static CheckpointId fromString(String checkpointId) { + public static CheckpointId deserialize(String checkpointId) { if (StringUtils.isBlank(checkpointId)) { throw new IllegalArgumentException("Invalid checkpoint id: " + checkpointId); } - String[] parts = checkpointId.split(SEPARATOR); - return new CheckpointId(Long.parseLong(parts[0]), Long.parseLong(parts[1])); + try { + String[] parts = checkpointId.split(SEPARATOR); + return new CheckpointId(Long.parseLong(parts[0]), Long.parseLong(parts[1])); + } catch (NumberFormatException ex) { + throw new IllegalArgumentException(String.format( + "Could not deserialize CheckpointId: %s", checkpointId), ex); + } } public long getMillis() { return millis; } - public long getNanos() { - return nanos; + public long getNanoId() { + return nanoId; } - @Override - public String toString() { - return String.format("%s%s%s", millis, SEPARATOR, nanos); + /** + * Serialization of {@link CheckpointId} as part of task checkpoints, in conjunction with {@link #deserialize(String)}. + * @return the String representation of this {@link CheckpointId}. + */ + public String serialize() { + return String.format("%s%s%s", getMillis(), SEPARATOR, getNanoId()); } @Override @@ -72,11 +82,22 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; CheckpointId that = (CheckpointId) o; return millis == that.millis && - nanos == that.nanos; + nanoId == that.nanoId; } @Override public int hashCode() { - return Objects.hash(millis, nanos); + return Objects.hash(millis, nanoId); + } + + @Override + public int compareTo(CheckpointId that) { + if(this.millis != that.millis) return Long.compare(this.millis, that.millis); + else return Long.compare(this.nanoId, that.nanoId); + } + + @Override + public String toString() { + return String.format("%s%s%s", millis, SEPARATOR, nanoId); } } \ No newline at end of file diff --git a/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointV1.java b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointV1.java new file mode 100644 index 0000000000..3e2cce3a32 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointV1.java @@ -0,0 +1,83 @@ +/* + * 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.samza.checkpoint; + +import org.apache.samza.system.SystemStreamPartition; + +import java.util.Collections; +import java.util.Map; +import java.util.Objects; + +/** + * A checkpoint is a mapping of all the streams a job is consuming and the most recent current offset for each. + * It is used to restore a {@link org.apache.samza.task.StreamTask}, either as part of a job restart or as part + * of restarting a failed container within a running job. + */ +public class CheckpointV1 implements Checkpoint { + public static final short CHECKPOINT_VERSION = 1; + + private final Map offsets; + + /** + * Constructs a new checkpoint based off a map of Samza stream offsets. + * + * @param offsets Map of Samza streams to their current offset. + */ + public CheckpointV1(Map offsets) { + this.offsets = offsets; + } + + public short getVersion() { + return CHECKPOINT_VERSION; + } + + /** + * Gets a unmodifiable view of the current Samza input stream offsets. + * + * @return a unmodifiable view of a Map of Samza streams to their recorded offsets. + */ + @Override + public Map getOffsets() { + return Collections.unmodifiableMap(offsets); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CheckpointV1 that = (CheckpointV1) o; + + return Objects.equals(offsets, that.offsets); + } + + @Override + public int hashCode() { + return Objects.hash(offsets); + } + + @Override + public String toString() { + return "CheckpointV1 [CHECKPOINT_VERSION=" + CHECKPOINT_VERSION + ", offsets=" + offsets + "]"; + } +} diff --git a/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointV2.java b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointV2.java new file mode 100644 index 0000000000..ac553f44ec --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointV2.java @@ -0,0 +1,123 @@ +/* + * 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.samza.checkpoint; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import java.util.Objects; +import org.apache.samza.system.SystemStreamPartition; + +import java.util.Map; + +/** + * A checkpoint is a mapping of all the streams a job is consuming and the most recent current offset for each. + * It is used to restore a {@link org.apache.samza.task.StreamTask}, either as part of a job restart or as part + * of restarting a failed container within a running job. + */ + +public class CheckpointV2 implements Checkpoint { + public static final short CHECKPOINT_VERSION = 2; + + private final CheckpointId checkpointId; + private final Map inputOffsets; + private final Map> stateCheckpointMarkers; + + /** + * Constructs the checkpoint with separated input and state offsets + * + * @param checkpointId {@link CheckpointId} associated with this checkpoint + * @param inputOffsets Map of Samza system stream partition to offset of the checkpoint + * @param stateCheckpoints Map of state backend factory name to map of local state store names + * to state checkpoints + */ + public CheckpointV2(CheckpointId checkpointId, + Map inputOffsets, + Map> stateCheckpoints) { + Preconditions.checkArgument(inputOffsets != null, + "inputOffsets for CheckpointV2 must not be null"); + Preconditions.checkArgument(stateCheckpoints != null, + "stateCheckpoints for CheckpointV2 must not be null"); + this.checkpointId = checkpointId; + this.inputOffsets = ImmutableMap.copyOf(inputOffsets); + this.stateCheckpointMarkers = ImmutableMap.copyOf(stateCheckpoints); + } + + public short getVersion() { + return CHECKPOINT_VERSION; + } + + /** + * Gets the checkpoint id for the checkpoint + * @return The timestamp based checkpoint identifier associated with the checkpoint + */ + public CheckpointId getCheckpointId() { + return checkpointId; + } + + /** + * Gets a unmodifiable view of the current input {@link SystemStreamPartition} offsets. + * @return An unmodifiable map of input {@link SystemStreamPartition}s to their recorded offsets. + */ + @Override + public Map getOffsets() { + return inputOffsets; + } + + /** + * Gets the state checkpoint markers for all stores for each configured state backend. + * + * Note: We don't add this method to the {@link Checkpoint} interface since it is difficult + * to implement it for {@link CheckpointV1} without changing the underlying serialization format - + * the changelog SSP offsets are serialized in the same way as input offsets, and at + * deserialization time we don't have enough information (e.g. configs) to decide whether a + * particular entry is for an input SSP or a changelog SSP. + * + * @return Map of state backend factory name to map of local state store names to state checkpoint markers + */ + public Map> getStateCheckpointMarkers() { + return stateCheckpointMarkers; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CheckpointV2 that = (CheckpointV2) o; + + return checkpointId.equals(that.checkpointId) && + Objects.equals(inputOffsets, that.inputOffsets) && + Objects.equals(stateCheckpointMarkers, that.stateCheckpointMarkers); + } + + @Override + public int hashCode() { + return Objects.hash(checkpointId, inputOffsets, stateCheckpointMarkers); + } + + @Override + public String toString() { + return "CheckpointV2 [CHECKPOINT_VERSION=" + CHECKPOINT_VERSION + ", checkpointId=" + checkpointId + + ", inputOffsets=" + inputOffsets + ", stateCheckpointMarkers=" + stateCheckpointMarkers + "]"; + } +} \ No newline at end of file diff --git a/samza-api/src/main/java/org/apache/samza/serializers/JsonCheckpointIdMixin.java b/samza-api/src/main/java/org/apache/samza/serializers/JsonCheckpointIdMixin.java new file mode 100644 index 0000000000..752e329c1c --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/serializers/JsonCheckpointIdMixin.java @@ -0,0 +1,42 @@ +/* + * 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.samza.serializers; + +import org.apache.samza.checkpoint.CheckpointId; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + + +/** + * A mix-in Jackson class to convert {@link CheckpointId} to/from JSON. + */ +public abstract class JsonCheckpointIdMixin { + @JsonCreator + private JsonCheckpointIdMixin( + @JsonProperty("millis") long millis, + @JsonProperty("nanos") long nanos) { + } + + @JsonProperty("millis") + abstract long getMillis(); + + @JsonProperty("nanos") + abstract long getNanos(); +} diff --git a/samza-api/src/test/java/org/apache/samza/checkpoint/TestCheckpointId.java b/samza-api/src/test/java/org/apache/samza/checkpoint/TestCheckpointId.java new file mode 100644 index 0000000000..2ac1b88244 --- /dev/null +++ b/samza-api/src/test/java/org/apache/samza/checkpoint/TestCheckpointId.java @@ -0,0 +1,53 @@ +/* + * 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.samza.checkpoint; + +import org.junit.Test; + +import static org.apache.samza.checkpoint.CheckpointId.*; +import static org.junit.Assert.assertEquals; + + +public class TestCheckpointId { + @Test + public void testSerializationDeserialization() { + CheckpointId checkpointId = CheckpointId.create(); + CheckpointId deserializedCheckpointId = CheckpointId.deserialize(checkpointId.serialize()); + + assertEquals(checkpointId.getMillis(), deserializedCheckpointId.getMillis()); + assertEquals(checkpointId.getNanoId(), deserializedCheckpointId.getNanoId()); + assertEquals(checkpointId, deserializedCheckpointId); + } + + @Test + public void testSerializationFormatForBackwardsCompatibility() { + CheckpointId checkpointId = CheckpointId.create(); + String serializedCheckpointId = checkpointId.serialize(); + + // WARNING: This format is written to persisted remotes stores and local files, making a change in the format + // would be backwards incompatible + String legacySerializedFormat = serializeLegacy(checkpointId); + assertEquals(checkpointId, CheckpointId.deserialize(legacySerializedFormat)); + } + + public String serializeLegacy(CheckpointId id) { + return String.format("%s%s%s", id.getMillis(), SEPARATOR, id.getNanoId()); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/checkpoint/azure/AzureCheckpointManager.java b/samza-azure/src/main/java/org/apache/samza/checkpoint/azure/AzureCheckpointManager.java index 901319634e..1d49c2c852 100644 --- a/samza-azure/src/main/java/org/apache/samza/checkpoint/azure/AzureCheckpointManager.java +++ b/samza-azure/src/main/java/org/apache/samza/checkpoint/azure/AzureCheckpointManager.java @@ -19,6 +19,7 @@ package org.apache.samza.checkpoint.azure; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.microsoft.azure.storage.StorageException; import com.microsoft.azure.storage.table.*; @@ -28,6 +29,7 @@ import org.apache.samza.SamzaException; import org.apache.samza.checkpoint.Checkpoint; import org.apache.samza.checkpoint.CheckpointManager; +import org.apache.samza.checkpoint.CheckpointV1; import org.apache.samza.config.AzureConfig; import org.apache.samza.container.TaskName; import org.apache.samza.serializers.JsonSerdeV2; @@ -113,13 +115,17 @@ public void register(TaskName taskName) { @Override public void writeCheckpoint(TaskName taskName, Checkpoint checkpoint) { + Preconditions.checkArgument(checkpoint instanceof CheckpointV1, + "Only CheckpointV1 could be written to Azure"); + if (!taskNames.contains(taskName)) { throw new SamzaException("writing checkpoint of unregistered task"); } TableBatchOperation batchOperation = new TableBatchOperation(); - Iterator> iterator = checkpoint.getOffsets().entrySet().iterator(); + Iterator> iterator = + checkpoint.getOffsets().entrySet().iterator(); while (iterator.hasNext()) { Map.Entry entry = iterator.next(); SystemStreamPartition ssp = entry.getKey(); @@ -205,7 +211,7 @@ public Checkpoint readLastCheckpoint(TaskName taskName) { return null; } LOG.debug("Received checkpoint state for taskName=%s", taskName); - return new Checkpoint(builder.build()); + return new CheckpointV1(builder.build()); } @Override diff --git a/samza-azure/src/test/java/org/apache/samza/checkpoint/azure/ITestAzureCheckpointManager.java b/samza-azure/src/test/java/org/apache/samza/checkpoint/azure/ITestAzureCheckpointManager.java index 7cf01c0baf..2240e65434 100644 --- a/samza-azure/src/test/java/org/apache/samza/checkpoint/azure/ITestAzureCheckpointManager.java +++ b/samza-azure/src/test/java/org/apache/samza/checkpoint/azure/ITestAzureCheckpointManager.java @@ -22,6 +22,7 @@ import org.apache.samza.Partition; import org.apache.samza.checkpoint.Checkpoint; import org.apache.samza.checkpoint.CheckpointManager; +import org.apache.samza.checkpoint.CheckpointV1; import org.apache.samza.config.AzureConfig; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; @@ -69,10 +70,10 @@ public void testStoringAndReadingCheckpointsSamePartition() { Map sspMap = new HashMap<>(); sspMap.put(ssp, "12345"); - Checkpoint cp0 = new Checkpoint(sspMap); + Checkpoint cp0 = new CheckpointV1(sspMap); sspMap.put(ssp, "54321"); - Checkpoint cp1 = new Checkpoint(sspMap); + Checkpoint cp1 = new CheckpointV1(sspMap); checkpointManager.register(taskName); @@ -96,12 +97,12 @@ public void testStoringAndReadingCheckpointsMultiPartitions() { Map sspMap = new HashMap<>(); sspMap.put(ssp, "12345"); sspMap.put(ssp1, "54321"); - Checkpoint cp1 = new Checkpoint(sspMap); + Checkpoint cp1 = new CheckpointV1(sspMap); Map sspMap2 = new HashMap<>(); sspMap2.put(ssp, "12347"); sspMap2.put(ssp1, "54323"); - Checkpoint cp2 = new Checkpoint(sspMap2); + Checkpoint cp2 = new CheckpointV1(sspMap2); checkpointManager.register(taskName); @@ -126,12 +127,12 @@ public void testStoringAndReadingCheckpointsMultiTasks() { Map sspMap = new HashMap<>(); sspMap.put(ssp, "12345"); sspMap.put(ssp1, "54321"); - Checkpoint cp1 = new Checkpoint(sspMap); + Checkpoint cp1 = new CheckpointV1(sspMap); Map sspMap2 = new HashMap<>(); sspMap2.put(ssp, "12347"); sspMap2.put(ssp1, "54323"); - Checkpoint cp2 = new Checkpoint(sspMap2); + Checkpoint cp2 = new CheckpointV1(sspMap2); checkpointManager.register(taskName); checkpointManager.register(taskName1); @@ -171,7 +172,7 @@ public void testMultipleBatchWrites() { sspMap.put(ssp, String.valueOf(i)); } - Checkpoint cp0 = new Checkpoint(sspMap); + Checkpoint cp0 = new CheckpointV1(sspMap); checkpointManager.register(taskName); checkpointManager.writeCheckpoint(taskName, cp0); Checkpoint readCp = checkpointManager.readLastCheckpoint(taskName); diff --git a/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointedChangelogOffset.java b/samza-core/src/main/java/org/apache/samza/checkpoint/kafka/KafkaChangelogSSPOffset.java similarity index 57% rename from samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointedChangelogOffset.java rename to samza-core/src/main/java/org/apache/samza/checkpoint/kafka/KafkaChangelogSSPOffset.java index 407ce7a444..09ea4975f2 100644 --- a/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointedChangelogOffset.java +++ b/samza-core/src/main/java/org/apache/samza/checkpoint/kafka/KafkaChangelogSSPOffset.java @@ -16,67 +16,75 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.checkpoint; +package org.apache.samza.checkpoint.kafka; import java.util.Objects; import org.apache.commons.lang3.StringUtils; import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.checkpoint.CheckpointId; /** - * Checkpointed changelog offset has the format: [checkpointId, offset], separated by a colon. + * Used in {@link org.apache.samza.checkpoint.CheckpointV1} for tracking the latest offset for store changelogs at + * the time of commit. Checkpointed changelog offset has the format: [checkpointId, offset], separated by a colon. */ @InterfaceStability.Unstable -public class CheckpointedChangelogOffset { +public class KafkaChangelogSSPOffset { public static final String SEPARATOR = ":"; private final CheckpointId checkpointId; - private final String offset; + private final String changelogOffset; - public CheckpointedChangelogOffset(CheckpointId checkpointId, String offset) { + public KafkaChangelogSSPOffset(CheckpointId checkpointId, String changelogOffset) { this.checkpointId = checkpointId; - this.offset = offset; + this.changelogOffset = changelogOffset; } - public static CheckpointedChangelogOffset fromString(String message) { + public static KafkaChangelogSSPOffset fromString(String message) { if (StringUtils.isBlank(message)) { throw new IllegalArgumentException("Invalid checkpointed changelog message: " + message); } - String[] checkpointIdAndOffset = message.split(":"); + String[] checkpointIdAndOffset = message.split(SEPARATOR); if (checkpointIdAndOffset.length != 2) { throw new IllegalArgumentException("Invalid checkpointed changelog offset: " + message); } - CheckpointId checkpointId = CheckpointId.fromString(checkpointIdAndOffset[0]); + CheckpointId checkpointId = CheckpointId.deserialize(checkpointIdAndOffset[0]); String offset = null; if (!"null".equals(checkpointIdAndOffset[1])) { offset = checkpointIdAndOffset[1]; } - return new CheckpointedChangelogOffset(checkpointId, offset); + + return new KafkaChangelogSSPOffset(checkpointId, offset); } public CheckpointId getCheckpointId() { return checkpointId; } - public String getOffset() { - return offset; + public String getChangelogOffset() { + return changelogOffset; } + /** + * WARNING: Do not change the toString() representation. It is used for serde'ing the store changelog offsets + * as part of task checkpoints, in conjunction with {@link #fromString(String)}. + * @return the String representation of this {@link KafkaChangelogSSPOffset} + */ @Override public String toString() { - return String.format("%s%s%s", checkpointId, SEPARATOR, offset); + return String.format("%s%s%s", checkpointId, SEPARATOR, changelogOffset); } @Override public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - CheckpointedChangelogOffset that = (CheckpointedChangelogOffset) o; + KafkaChangelogSSPOffset that = (KafkaChangelogSSPOffset) o; return Objects.equals(checkpointId, that.checkpointId) && - Objects.equals(offset, that.offset); + Objects.equals(changelogOffset, that.changelogOffset); } @Override public int hashCode() { - return Objects.hash(checkpointId, offset); + return Objects.hash(checkpointId, changelogOffset); } } diff --git a/samza-core/src/main/java/org/apache/samza/checkpoint/kafka/KafkaStateCheckpointMarker.java b/samza-core/src/main/java/org/apache/samza/checkpoint/kafka/KafkaStateCheckpointMarker.java new file mode 100644 index 0000000000..ccd06d1df1 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/checkpoint/kafka/KafkaStateCheckpointMarker.java @@ -0,0 +1,145 @@ +/* + * 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.samza.checkpoint.kafka; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import org.apache.samza.SamzaException; +import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.serializers.model.SamzaObjectMapper; +import org.apache.samza.storage.KafkaChangelogStateBackendFactory; +import org.apache.samza.system.SystemStreamPartition; +import scala.Option; + + +/** + * Used as the serialization format for the state checkpoints of {@link org.apache.samza.checkpoint.CheckpointV2} + * for a store using {@link org.apache.samza.storage.KafkaTransactionalStateTaskBackupManager} or + * {@link org.apache.samza.storage.KafkaNonTransactionalStateTaskBackupManager} for tracking the latest committed + * store changelog offsets. + * + * Kafka state checkpoint marker has the format: [system, stream, partition, offset], separated by a semi-colon. + */ +@InterfaceStability.Unstable +public class KafkaStateCheckpointMarker { + public static final String KAFKA_STATE_BACKEND_FACTORY_NAME = KafkaChangelogStateBackendFactory.class.getName(); + public static final short MARKER_VERSION = 1; + private static final ObjectMapper MAPPER = SamzaObjectMapper.getObjectMapper(); + + // Required for Jackson Serde + private final short version; + private final SystemStreamPartition changelogSSP; + private final String changelogOffset; + + public KafkaStateCheckpointMarker(SystemStreamPartition changelogSSP, String changelogOffset) { + this(MARKER_VERSION, changelogSSP, changelogOffset); + } + + public KafkaStateCheckpointMarker(short version, SystemStreamPartition changelogSSP, String changelogOffset) { + this.version = version; + this.changelogSSP = changelogSSP; + this.changelogOffset = changelogOffset; + } + + public static KafkaStateCheckpointMarker deserialize(String stateCheckpointMarker) { + try { + return MAPPER.readValue(stateCheckpointMarker, KafkaStateCheckpointMarker.class); + } catch (JsonProcessingException e) { + throw new IllegalArgumentException("Could not deserialize KafkaStateCheckpointMarker: " + stateCheckpointMarker); + } + } + + public short getVersion() { + return version; + } + + public SystemStreamPartition getChangelogSSP() { + return changelogSSP; + } + + public String getChangelogOffset() { + return changelogOffset; + } + + /** + * Builds a map of store changelog SSPs to their offset for Kafka changelog backed stores from the provided + * map of state backend factory name to map of store name to serialized state checkpoint markers. + * + * @param stateBackendToStoreSCMs Map of state backend factory name to map of store name to serialized + * state checkpoint markers + * @return Map of store changelog SSPss to their optional offset, or an empty map if there is no mapping for + * {@link #KAFKA_STATE_BACKEND_FACTORY_NAME} in the input map. Optional offset may be empty if the + * changelog SSP was empty. + */ + public static Map> scmsToSSPOffsetMap( + Map> stateBackendToStoreSCMs) { + Map> sspToOffsetOptions = new HashMap<>(); + if (stateBackendToStoreSCMs.containsKey(KAFKA_STATE_BACKEND_FACTORY_NAME)) { + Map storeToKafkaSCMs = stateBackendToStoreSCMs.get(KAFKA_STATE_BACKEND_FACTORY_NAME); + storeToKafkaSCMs.forEach((key, value) -> { + KafkaStateCheckpointMarker stateMarker = KafkaStateCheckpointMarker.deserialize(value); + Option offsetOption = Option.apply(stateMarker.getChangelogOffset()); + sspToOffsetOptions.put(new SystemStreamPartition(stateMarker.getChangelogSSP()), offsetOption); + }); + } + return sspToOffsetOptions; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KafkaStateCheckpointMarker that = (KafkaStateCheckpointMarker) o; + return Objects.equals(changelogSSP, that.changelogSSP) && + Objects.equals(changelogOffset, that.changelogOffset); + } + + @Override + public int hashCode() { + return Objects.hash(changelogSSP, changelogOffset); + } + + /** + * It is used for serde'ing {@link KafkaStateCheckpointMarker}s, in conjunction with {@link #deserialize(String)}. + * @return the String representation of this {@link KafkaStateCheckpointMarker} + */ + public static String serialize(KafkaStateCheckpointMarker marker) { + try { + return MAPPER.writeValueAsString(marker); + } catch (JsonProcessingException e) { + throw new SamzaException(String.format("Error serializing KafkaCheckpointMarker %s", marker), e); + } + } + + @Override + public String toString() { + String separator = ","; + return String.format("%s%s%s%s%s%s%s", + changelogSSP.getSystem(), separator, changelogSSP.getStream(), separator, + changelogSSP.getPartition().getPartitionId(), separator, changelogOffset); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java index 021c67de26..12257e03d7 100644 --- a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java @@ -63,6 +63,18 @@ public class JobConfig extends MapConfig { public static final String JOB_CONTAINER_COUNT = "job.container.count"; static final int DEFAULT_JOB_CONTAINER_COUNT = 1; public static final String JOB_CONTAINER_THREAD_POOL_SIZE = "job.container.thread.pool.size"; + // num commit threads == min(max(2 * num tasks in container, thread pool size), max thread pool size) + public static final String COMMIT_THREAD_POOL_SIZE = "job.container.commit.thread.pool.size"; + static final int DEFAULT_COMMIT_THREAD_POOL_SIZE = 2; + public static final String COMMIT_THREAD_POOL_MAX_SIZE = "job.container.commit.thread.pool.max.size"; + static final int DEFAULT_COMMIT_THREAD_POOL_MAX_SIZE = 64; + + // num commit threads == min(max(2 * num tasks in container, thread pool size), max thread pool size) + public static final String RESTORE_THREAD_POOL_SIZE = "job.container.restore.thread.pool.size"; + static final int DEFAULT_RESTORE_THREAD_POOL_SIZE = 2; + public static final String RESTORE_THREAD_POOL_MAX_SIZE = "job.container.restore.thread.pool.max.size"; + static final int DEFAULT_RESTORE_THREAD_POOL_MAX_SIZE = 64; + public static final String JOB_INTERMEDIATE_STREAM_PARTITIONS = "job.intermediate.stream.partitions"; public static final String JOB_DEBOUNCE_TIME_MS = "job.debounce.time.ms"; @@ -336,6 +348,22 @@ public int getThreadPoolSize() { } } + public int getCommitThreadPoolSize() { + return getInt(COMMIT_THREAD_POOL_SIZE, DEFAULT_COMMIT_THREAD_POOL_SIZE); + } + + public int getCommitThreadPoolMaxSize() { + return getInt(COMMIT_THREAD_POOL_MAX_SIZE, DEFAULT_COMMIT_THREAD_POOL_MAX_SIZE); + } + + public int getRestoreThreadPoolSize() { + return getInt(RESTORE_THREAD_POOL_SIZE, DEFAULT_RESTORE_THREAD_POOL_SIZE); + } + + public int getRestoreThreadPoolMaxSize() { + return getInt(RESTORE_THREAD_POOL_MAX_SIZE, DEFAULT_RESTORE_THREAD_POOL_MAX_SIZE); + } + public int getDebounceTimeMs() { return getInt(JOB_DEBOUNCE_TIME_MS, DEFAULT_DEBOUNCE_TIME_MS); } diff --git a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java index a8b870268f..f5e2055e3c 100644 --- a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java @@ -19,16 +19,22 @@ package org.apache.samza.config; +import com.google.common.collect.ImmutableList; import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; import org.apache.samza.execution.StreamManager; +import org.apache.samza.system.SystemStream; import org.apache.samza.util.StreamUtil; import static com.google.common.base.Preconditions.*; @@ -61,6 +67,12 @@ public class StorageConfig extends MapConfig { public static final String CHANGELOG_MIN_COMPACTION_LAG_MS = STORE_PREFIX + "%s.changelog." + MIN_COMPACTION_LAG_MS; public static final long DEFAULT_CHANGELOG_MIN_COMPACTION_LAG_MS = TimeUnit.HOURS.toMillis(4); + public static final String DEFAULT_STATE_BACKEND_FACTORY = "org.apache.samza.storage.KafkaChangelogStateBackendFactory"; + public static final String STORE_BACKEND_BACKUP_FACTORIES = STORE_PREFIX + "%s.state.backend.backup.factories"; + public static final List DEFAULT_STATE_BACKEND_BACKUP_FACTORIES = ImmutableList.of( + DEFAULT_STATE_BACKEND_FACTORY); + public static final String STATE_BACKEND_RESTORE_FACTORY = STORE_PREFIX + "state.restore.backend"; + static final String CHANGELOG_SYSTEM = "job.changelog.system"; static final String CHANGELOG_DELETE_RETENTION_MS = STORE_PREFIX + "%s.changelog.delete.retention.ms"; static final long DEFAULT_CHANGELOG_DELETE_RETENTION_MS = TimeUnit.DAYS.toMillis(1); @@ -94,6 +106,11 @@ public List getStoreNames() { return storeNames; } + public Map getStoreChangelogs() { + return getStoreNames().stream().filter(store -> getChangelogStream(store).isPresent()) + .collect(Collectors.toMap(Function.identity(), n -> StreamUtil.getSystemStreamFromNames(getChangelogStream(n).get()))); + } + /** * If the config specifies 'stores.<storename>.changelog' as '<system>.<stream>' combination - it will take * precedence. @@ -122,6 +139,15 @@ public Optional getChangelogStream(String storeName) { return Optional.ofNullable(systemStreamRes); } + public List getStoreBackupManagerClassName(String storeName) { + List storeBackupManagers = getList(String.format(STORE_BACKEND_BACKUP_FACTORIES, storeName), new ArrayList<>()); + // For backwards compatibility if the changelog is enabled, we use default kafka backup factory + if (storeBackupManagers.isEmpty() && getChangelogStream(storeName).isPresent()) { + storeBackupManagers = DEFAULT_STATE_BACKEND_BACKUP_FACTORIES; + } + return storeBackupManagers; + } + public boolean getAccessLogEnabled(String storeName) { return getBoolean(String.format(ACCESSLOG_ENABLED, storeName), false); } @@ -240,6 +266,24 @@ public long getChangelogMinCompactionLagMs(String storeName) { return getLong(minCompactLagConfigName, getDefaultChangelogMinCompactionLagMs()); } + + public Set getStateBackendBackupFactories() { + return getStoreNames().stream() + .flatMap((storeName) -> getStoreBackupManagerClassName(storeName).stream()) + .collect(Collectors.toSet()); + } + + public List getBackupStoreNamesForStateBackupFactory(String backendFactoryName) { + return getStoreNames().stream() + .filter((storeName) -> getStoreBackupManagerClassName(storeName) + .contains(backendFactoryName)) + .collect(Collectors.toList()); + } + + public String getStateBackendRestoreFactory() { + return get(STATE_BACKEND_RESTORE_FACTORY, DEFAULT_STATE_BACKEND_FACTORY); + } + /** * Helper method to check if a system has a changelog attached to it. */ diff --git a/samza-core/src/main/java/org/apache/samza/config/TaskConfig.java b/samza-core/src/main/java/org/apache/samza/config/TaskConfig.java index 461b647062..8db1d2a5b6 100644 --- a/samza-core/src/main/java/org/apache/samza/config/TaskConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/TaskConfig.java @@ -19,6 +19,9 @@ package org.apache.samza.config; +import com.google.common.collect.ImmutableList; + +import java.time.Duration; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -53,6 +56,15 @@ public class TaskConfig extends MapConfig { // commit period in milliseconds public static final String COMMIT_MS = "task.commit.ms"; static final long DEFAULT_COMMIT_MS = 60000L; + // maximum amount of time a task may continue processing while a previous commit is pending + public static final String COMMIT_MAX_DELAY_MS = "task.commit.max.delay.ms"; + static final long DEFAULT_COMMIT_MAX_DELAY_MS = Duration.ofMinutes(1).toMillis(); + // maximum amount of time to block for a pending task commit to complete *after* + // COMMIT_MAX_DELAY_MS have passed since the pending commit start. if the pending commit + // does not complete within this timeout, the container will shut down. + public static final String COMMIT_TIMEOUT_MS = "task.commit.timeout.ms"; + static final long DEFAULT_COMMIT_TIMEOUT_MS = Duration.ofMinutes(1).toMillis(); + // how long to wait for a clean shutdown public static final String TASK_SHUTDOWN_MS = "task.shutdown.ms"; static final long DEFAULT_TASK_SHUTDOWN_MS = 30000L; @@ -108,6 +120,14 @@ public class TaskConfig extends MapConfig { // standby containers use this flag to indicate that checkpoints will be polled continually, rather than only once at startup like in an active container public static final String INTERNAL_CHECKPOINT_MANAGER_CONSUMER_STOP_AFTER_FIRST_READ = "samza.internal.task.checkpoint.consumer.stop.after.first.read"; + // list of checkpoint versions to write during processing + public static final String CHECKPOINT_WRITE_VERSIONS = "task.checkpoint.write.versions"; + public static final List DEFAULT_CHECKPOINT_WRITE_VERSIONS = ImmutableList.of("1", "2"); + + // checkpoint version to read during container startup + public static final String CHECKPOINT_READ_VERSION = "task.checkpoint.read.version"; + public static final short DEFAULT_CHECKPOINT_READ_VERSION = 1; + public static final String TRANSACTIONAL_STATE_CHECKPOINT_ENABLED = "task.transactional.state.checkpoint.enabled"; private static final boolean DEFAULT_TRANSACTIONAL_STATE_CHECKPOINT_ENABLED = true; public static final String TRANSACTIONAL_STATE_RESTORE_ENABLED = "task.transactional.state.restore.enabled"; @@ -143,6 +163,14 @@ public long getCommitMs() { return getLong(COMMIT_MS, DEFAULT_COMMIT_MS); } + public long getCommitMaxDelayMs() { + return getLong(COMMIT_MAX_DELAY_MS, DEFAULT_COMMIT_MAX_DELAY_MS); + } + + public long getCommitTimeoutMs() { + return getLong(COMMIT_TIMEOUT_MS, DEFAULT_COMMIT_TIMEOUT_MS); + } + public Optional getTaskClass() { return Optional.ofNullable(get(TASK_CLASS)); } @@ -315,6 +343,15 @@ public long getShutdownMs() { } } + public List getCheckpointWriteVersions() { + return getList(CHECKPOINT_WRITE_VERSIONS, DEFAULT_CHECKPOINT_WRITE_VERSIONS) + .stream().map(Short::valueOf).collect(Collectors.toList()); + } + + public short getCheckpointReadVersion() { + return getShort(CHECKPOINT_READ_VERSION, DEFAULT_CHECKPOINT_READ_VERSION); + } + public boolean getTransactionalStateCheckpointEnabled() { return getBoolean(TRANSACTIONAL_STATE_CHECKPOINT_ENABLED, DEFAULT_TRANSACTIONAL_STATE_CHECKPOINT_ENABLED); } diff --git a/samza-core/src/main/java/org/apache/samza/serializers/CheckpointV2Serde.java b/samza-core/src/main/java/org/apache/samza/serializers/CheckpointV2Serde.java new file mode 100644 index 0000000000..10e6b3d0b9 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/serializers/CheckpointV2Serde.java @@ -0,0 +1,59 @@ +/* + * 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.samza.serializers; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.samza.SamzaException; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.checkpoint.CheckpointV2; +import org.apache.samza.serializers.model.SamzaObjectMapper; +import org.apache.samza.system.SystemStreamPartition; + + +/** + * The {@link Serde} for {@link CheckpointV2} which includes {@link CheckpointId}s, state checkpoint markers + * and the input {@link SystemStreamPartition} offsets. + * + * The overall payload is serde'd as JSON using {@link SamzaObjectMapper}. Since the Samza classes cannot be directly + * serialized by Jackson using {@link org.apache.samza.serializers.model.JsonCheckpointV2Mixin}. + */ +public class CheckpointV2Serde implements Serde { + private static final ObjectMapper OBJECT_MAPPER = SamzaObjectMapper.getObjectMapper(); + + public CheckpointV2Serde() { } + + @Override + public CheckpointV2 fromBytes(byte[] bytes) { + try { + return OBJECT_MAPPER.readValue(bytes, CheckpointV2.class); + } catch (Exception e) { + throw new SamzaException(String.format("Exception while deserializing checkpoint: %s", new String(bytes)), e); + } + } + + @Override + public byte[] toBytes(CheckpointV2 checkpoint) { + try { + return OBJECT_MAPPER.writeValueAsBytes(checkpoint); + } catch (Exception e) { + throw new SamzaException(String.format("Exception while serializing checkpoint: %s", checkpoint.toString()), e); + } + } +} diff --git a/samza-core/src/main/java/org/apache/samza/serializers/JsonCheckpoint.java b/samza-core/src/main/java/org/apache/samza/serializers/JsonCheckpoint.java new file mode 100644 index 0000000000..47a8f7309d --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/serializers/JsonCheckpoint.java @@ -0,0 +1,57 @@ +/* + * 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.samza.serializers; + +import java.util.Map; + +/** + * Used for Json serialization of the {@link org.apache.samza.checkpoint.Checkpoint} class by the + * {@link CheckpointV2Serde} + * This cannot be an internal class as required by Jackson Object mapper + */ +public class JsonCheckpoint { + private String checkpointId; + private Map> inputOffsets; + // Map> + private Map> stateCheckpointMarkers; + + // Default constructor required for Jackson ObjectMapper + public JsonCheckpoint() {} + + public JsonCheckpoint(String checkpointId, + Map> inputOffsets, + Map> stateCheckpointMakers) { + this.checkpointId = checkpointId; + this.inputOffsets = inputOffsets; + this.stateCheckpointMarkers = stateCheckpointMakers; + } + + public String getCheckpointId() { + return checkpointId; + } + + public Map> getInputOffsets() { + return inputOffsets; + } + + public Map> getStateCheckpointMarkers() { + return stateCheckpointMarkers; + } +} diff --git a/samza-core/src/main/java/org/apache/samza/serializers/model/JsonCheckpointV2Mixin.java b/samza-core/src/main/java/org/apache/samza/serializers/model/JsonCheckpointV2Mixin.java new file mode 100644 index 0000000000..8e26745a82 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/serializers/model/JsonCheckpointV2Mixin.java @@ -0,0 +1,49 @@ +/* + * 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.samza.serializers.model; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Map; +import java.util.Set; +import org.apache.samza.Partition; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.container.TaskName; +import org.apache.samza.system.SystemStreamPartition; + +@JsonIgnoreProperties(ignoreUnknown = true) +public abstract class JsonCheckpointV2Mixin { + @JsonCreator + public JsonCheckpointV2Mixin( + @JsonProperty("checkpoint-id") CheckpointId checkpointId, + @JsonProperty("input-offsets") Map inputOffsets, + @JsonProperty("state-checkpoint-markers") Map> stateCheckpointMarkers) { + } + + @JsonProperty("checkpoint-id") + abstract CheckpointId getCheckpointId(); + + @JsonProperty("input-offsets") + abstract Map getOffsets(); + + @JsonProperty("state-checkpoint-markers") + abstract Map> getStateCheckpointMarkers(); +} diff --git a/samza-core/src/main/java/org/apache/samza/serializers/model/KafkaStateCheckpointMarkerMixin.java b/samza-core/src/main/java/org/apache/samza/serializers/model/KafkaStateCheckpointMarkerMixin.java new file mode 100644 index 0000000000..6194877995 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/serializers/model/KafkaStateCheckpointMarkerMixin.java @@ -0,0 +1,48 @@ +/* + * 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.samza.serializers.model; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.samza.system.SystemStreamPartition; + + +/** + * A mix-in Jackson class to convert {@link org.apache.samza.checkpoint.kafka.KafkaStateCheckpointMarker} to/from JSON + */ +@JsonIgnoreProperties(ignoreUnknown = true) +abstract public class KafkaStateCheckpointMarkerMixin { + @JsonCreator + public KafkaStateCheckpointMarkerMixin( + @JsonProperty("version") short version, + @JsonProperty("changelog-ssp") SystemStreamPartition changelogSSP, + @JsonProperty("changelog-offset") String changelogOffset) { + } + + @JsonProperty("version") + abstract short getVersion(); + + @JsonProperty("changelog-ssp") + abstract SystemStreamPartition getChangelogSSP(); + + @JsonProperty("changelog-offset") + abstract String getChangelogOffset(); +} diff --git a/samza-core/src/main/java/org/apache/samza/serializers/model/SamzaObjectMapper.java b/samza-core/src/main/java/org/apache/samza/serializers/model/SamzaObjectMapper.java index 5c5078f47b..7888f1fbef 100644 --- a/samza-core/src/main/java/org/apache/samza/serializers/model/SamzaObjectMapper.java +++ b/samza-core/src/main/java/org/apache/samza/serializers/model/SamzaObjectMapper.java @@ -41,6 +41,9 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import org.apache.samza.Partition; import org.apache.samza.SamzaException; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.checkpoint.CheckpointV2; +import org.apache.samza.checkpoint.kafka.KafkaStateCheckpointMarker; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.container.TaskName; @@ -95,11 +98,15 @@ public static ObjectMapper getObjectMapper() { module.addKeyDeserializer(SystemStreamPartition.class, new SystemStreamPartitionKeyDeserializer()); module.addDeserializer(Config.class, new ConfigDeserializer()); module.addDeserializer(TaskMode.class, new TaskModeDeserializer()); + module.addSerializer(CheckpointId.class, new CheckpointIdSerializer()); + module.addDeserializer(CheckpointId.class, new CheckpointIdDeserializer()); // Setup mixins for data models. mapper.addMixIn(TaskModel.class, JsonTaskModelMixIn.class); mapper.addMixIn(ContainerModel.class, JsonContainerModelMixIn.class); mapper.addMixIn(JobModel.class, JsonJobModelMixIn.class); + mapper.addMixIn(CheckpointV2.class, JsonCheckpointV2Mixin.class); + mapper.addMixIn(KafkaStateCheckpointMarker.class, KafkaStateCheckpointMarkerMixin.class); module.addDeserializer(ContainerModel.class, new JsonDeserializer() { @Override @@ -255,6 +262,22 @@ public SystemStreamPartition deserialize(JsonParser jsonParser, DeserializationC } } + public static class CheckpointIdSerializer extends JsonSerializer { + @Override + public void serialize(CheckpointId checkpointId, JsonGenerator gen, SerializerProvider serializers) throws IOException { + gen.writeString(checkpointId.serialize()); + } + } + + public static class CheckpointIdDeserializer extends JsonDeserializer { + @Override + public CheckpointId deserialize(JsonParser jsonParser, DeserializationContext context) throws IOException { + ObjectCodec oc = jsonParser.getCodec(); + JsonNode node = oc.readTree(jsonParser); + return CheckpointId.deserialize(node.textValue()); + } + } + /** * A Jackson property naming strategy that converts camel case JSON fields to * hyphenated names. For example, myVariableName would be converted to diff --git a/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala b/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala index 7936c0a63d..92b4e593cd 100644 --- a/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala +++ b/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala @@ -185,9 +185,10 @@ class CheckpointTool(newOffsets: TaskNameToCheckpointMap, coordinatorStreamStore taskNames.foreach(checkpointManager.register) checkpointManager.start() + // TODO dchen make add support for checkpointv2 val lastCheckpoints = taskNames.map(taskName => { taskName -> Option(checkpointManager.readLastCheckpoint(taskName)) - .getOrElse(new Checkpoint(new java.util.HashMap[SystemStreamPartition, String]())) + .getOrElse(new CheckpointV1(new java.util.HashMap[SystemStreamPartition, String]())) .getOffsets .asScala .toMap @@ -199,7 +200,7 @@ class CheckpointTool(newOffsets: TaskNameToCheckpointMap, coordinatorStreamStore newOffsets.foreach { case (taskName: TaskName, offsets: Map[SystemStreamPartition, String]) => logCheckpoint(taskName, offsets, "New offset to be written for task: " + taskName) - val checkpoint = new Checkpoint(offsets.asJava) + val checkpoint = new CheckpointV1(offsets.asJava) checkpointManager.writeCheckpoint(taskName, checkpoint) info(s"Updated the checkpoint of the task: $taskName to: $offsets") } diff --git a/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala b/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala index 442d83f035..7491eaaaf9 100644 --- a/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala +++ b/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala @@ -19,6 +19,7 @@ package org.apache.samza.checkpoint +import java.util import java.util.HashMap import java.util.concurrent.ConcurrentHashMap @@ -219,6 +220,16 @@ class OffsetManager( Option(lastProcessedOffsets.get(taskName)).map(_.get(systemStreamPartition)) } + /** + * Get the last checkpoint saved in the checkpoint manager or null if there is no recorded checkpoints for the task + */ + def getLastTaskCheckpoint(taskName: TaskName): Checkpoint = { + if (checkpointManager != null) { + checkpointManager.readLastCheckpoint(taskName) + } + null + } + /** * Get the starting offset for a SystemStreamPartition. This is the offset * where a SamzaContainer begins reading from when it starts up. @@ -267,9 +278,9 @@ class OffsetManager( * ensure there are no concurrent updates to the offsets between when this method is * invoked and the corresponding call to [[OffsetManager.writeCheckpoint()]] */ - def buildCheckpoint(taskName: TaskName): Checkpoint = { + def getLastProcessedOffsets(taskName: TaskName): util.Map[SystemStreamPartition, String] = { if (checkpointManager != null || checkpointListeners.nonEmpty) { - debug("Getting checkpoint offsets for taskName %s." format taskName) + debug("Getting last processed offsets to checkpoint for taskName %s." format taskName) val taskStartingOffsets = startingOffsets.getOrElse(taskName, throw new SamzaException("Couldn't find starting offsets for task: " + taskName)) @@ -283,10 +294,10 @@ class OffsetManager( .filterKeys(taskSSPs.contains) val modifiedTaskOffsets = getModifiedOffsets(taskStartingOffsets, taskLastProcessedOffsets) - new Checkpoint(new HashMap(modifiedTaskOffsets)) // Copy into new Map to prevent mutation + new util.HashMap(modifiedTaskOffsets) } else { - debug("Returning null checkpoint for taskName %s because no checkpoint manager/callback is defined." format taskName) - null + debug("Returning empty offsets for taskName %s because no checkpoint manager/callback is defined." format taskName) + new util.HashMap() } } @@ -336,11 +347,13 @@ class OffsetManager( */ def writeCheckpoint(taskName: TaskName, checkpoint: Checkpoint) { if (checkpoint != null && (checkpointManager != null || checkpointListeners.nonEmpty)) { - debug("Writing checkpoint for taskName %s with offsets %s." format (taskName, checkpoint)) + debug("Writing checkpoint for taskName: %s as: %s." format (taskName, checkpoint)) + + val sspToOffsets = checkpoint.getOffsets if(checkpointManager != null) { checkpointManager.writeCheckpoint(taskName, checkpoint) - val sspToOffsets = checkpoint.getOffsets + if(sspToOffsets != null) { sspToOffsets.asScala.foreach { case (ssp, cp) => { @@ -357,7 +370,7 @@ class OffsetManager( // changelog SSPs are not registered but may be present in the Checkpoint if transactional state checkpointing // is enabled. val registeredSSPs = systemStreamPartitions.getOrElse(taskName, Set[SystemStreamPartition]()) - checkpoint.getOffsets.asScala + sspToOffsets.asScala .filterKeys(registeredSSPs.contains) .groupBy { case (ssp, _) => ssp.getSystem }.foreach { case (systemName:String, offsets: Map[SystemStreamPartition, String]) => { @@ -452,7 +465,6 @@ class OffsetManager( Map(taskName -> checkpoint.getOffsets.asScala.toMap) } else { info("Did not receive a checkpoint for taskName %s. Proceeding without a checkpoint." format taskName) - Map(taskName -> Map()) } } diff --git a/samza-core/src/main/scala/org/apache/samza/checkpoint/file/FileSystemCheckpointManager.scala b/samza-core/src/main/scala/org/apache/samza/checkpoint/file/FileSystemCheckpointManager.scala index a9780b6e93..b740a130e9 100644 --- a/samza-core/src/main/scala/org/apache/samza/checkpoint/file/FileSystemCheckpointManager.scala +++ b/samza-core/src/main/scala/org/apache/samza/checkpoint/file/FileSystemCheckpointManager.scala @@ -24,13 +24,11 @@ import java.io.FileNotFoundException import java.io.FileOutputStream import org.apache.samza.SamzaException -import org.apache.samza.checkpoint.Checkpoint -import org.apache.samza.checkpoint.CheckpointManager -import org.apache.samza.checkpoint.CheckpointManagerFactory +import org.apache.samza.checkpoint.{Checkpoint, CheckpointManager, CheckpointManagerFactory, CheckpointV1} import org.apache.samza.config.{Config, FileSystemCheckpointManagerConfig, JobConfig} import org.apache.samza.container.TaskName import org.apache.samza.metrics.MetricsRegistry -import org.apache.samza.serializers.CheckpointSerde +import org.apache.samza.serializers.CheckpointV1Serde import org.apache.samza.util.ScalaJavaUtil.JavaOptionals import scala.io.Source @@ -38,14 +36,14 @@ import scala.io.Source class FileSystemCheckpointManager( jobName: String, root: File, - serde: CheckpointSerde = new CheckpointSerde) extends CheckpointManager { + serde: CheckpointV1Serde = new CheckpointV1Serde) extends CheckpointManager { override def register(taskName: TaskName):Unit = Unit def getCheckpointFile(taskName: TaskName) = getFile(jobName, taskName, "checkpoints") def writeCheckpoint(taskName: TaskName, checkpoint: Checkpoint) { - val bytes = serde.toBytes(checkpoint) + val bytes = serde.toBytes(checkpoint.asInstanceOf[CheckpointV1]) val fos = new FileOutputStream(getCheckpointFile(taskName)) fos.write(bytes) diff --git a/samza-core/src/main/scala/org/apache/samza/serializers/CheckpointSerde.scala b/samza-core/src/main/scala/org/apache/samza/serializers/CheckpointV1Serde.scala similarity index 54% rename from samza-core/src/main/scala/org/apache/samza/serializers/CheckpointSerde.scala rename to samza-core/src/main/scala/org/apache/samza/serializers/CheckpointV1Serde.scala index 02246675e2..7db709e06f 100644 --- a/samza-core/src/main/scala/org/apache/samza/serializers/CheckpointSerde.scala +++ b/samza-core/src/main/scala/org/apache/samza/serializers/CheckpointV1Serde.scala @@ -19,14 +19,13 @@ package org.apache.samza.serializers -import com.fasterxml.jackson.core.`type`.TypeReference import com.fasterxml.jackson.databind.ObjectMapper -import org.apache.samza.util.Logging import java.util -import org.apache.samza.checkpoint.Checkpoint -import org.apache.samza.container.TaskName +import org.apache.samza.Partition +import org.apache.samza.checkpoint.CheckpointV1 import org.apache.samza.system.SystemStreamPartition -import org.apache.samza.{SamzaException, Partition} +import org.apache.samza.util.Logging + import scala.collection.JavaConverters._ /** @@ -35,34 +34,26 @@ import scala.collection.JavaConverters._ * then interfere with JSON's decoding of the overall map. We'll sidestep the whole issue by turning the * map into a list[String] of (System, Stream, Partition, Offset) serializing that. */ -class CheckpointSerde extends Serde[Checkpoint] with Logging { - import CheckpointSerde._ - // TODO: Elucidate the CheckpointSerde relationshiop to Serde. Should Serde also have keyTo/FromBytes? Should - // we just take CheckpointSerde here as interface and have this be JSONCheckpointSerde? - // TODO: Add more tests. This class currently only has direct test and is mainly tested by the other checkpoint managers +class CheckpointV1Serde extends Serde[CheckpointV1] with Logging { val jsonMapper = new ObjectMapper() - // Jackson absolutely hates Scala types and hidden conversions hate you, so we're going to be very, very - // explicit about the Java (not Scala) types used here and never let Scala get its grubby little hands - // on any instance. - - // Store checkpoint as maps keyed of the SSP.toString to the another map of the constituent SSP components - // and offset. Jackson can't automatically serialize the SSP since it's not a POJO and this avoids - // having to wrap it another class while maintaing readability. - - def fromBytes(bytes: Array[Byte]): Checkpoint = { + // Serialize checkpoint as maps keyed by the SSP.toString() to the another map of the constituent SSP components + // and offset. Jackson can't automatically serialize the SSP since it's not a POJO and this avoids + // having to wrap it another class while maintaining readability. + // { "SSP.toString()" -> {"system": system, "stream": stream, "partition": partition, "offset": offset)} + def fromBytes(bytes: Array[Byte]): CheckpointV1 = { try { val jMap = jsonMapper.readValue(bytes, classOf[util.HashMap[String, util.HashMap[String, String]]]) - def deserializeJSONMap(m:util.HashMap[String, String]) = { - require(m.size() == 4, "All JSON-encoded SystemStreamPartitions must have four keys") - val system = m.get("system") + def deserializeJSONMap(sspInfo:util.HashMap[String, String]) = { + require(sspInfo.size() == 4, "All JSON-encoded SystemStreamPartitions must have four keys") + val system = sspInfo.get("system") require(system != null, "System must be present in JSON-encoded SystemStreamPartition") - val stream = m.get("stream") + val stream = sspInfo.get("stream") require(stream != null, "Stream must be present in JSON-encoded SystemStreamPartition") - val partition = m.get("partition") + val partition = sspInfo.get("partition") require(partition != null, "Partition must be present in JSON-encoded SystemStreamPartition") - val offset = m.get("offset") + val offset = sspInfo.get("offset") // allow null offsets, e.g. for changelog ssps new SystemStreamPartition(system, stream, new Partition(partition.toInt)) -> offset @@ -70,16 +61,15 @@ class CheckpointSerde extends Serde[Checkpoint] with Logging { val cpMap = jMap.values.asScala.map(deserializeJSONMap).toMap - new Checkpoint(cpMap.asJava) - }catch { + new CheckpointV1(cpMap.asJava) + } catch { case e : Exception => - warn("Exception while deserializing checkpoint: " + e) - debug("Exception detail:", e) + warn("Exception while deserializing checkpoint: {}", util.Arrays.toString(bytes), e) null } } - def toBytes(checkpoint: Checkpoint): Array[Byte] = { + def toBytes(checkpoint: CheckpointV1): Array[Byte] = { val offsets = checkpoint.getOffsets val asMap = new util.HashMap[String, util.HashMap[String, String]](offsets.size()) @@ -96,21 +86,4 @@ class CheckpointSerde extends Serde[Checkpoint] with Logging { jsonMapper.writeValueAsBytes(asMap) } - - def changelogPartitionMappingFromBytes(bytes: Array[Byte]): util.Map[TaskName, java.lang.Integer] = { - try { - jsonMapper.readValue(bytes, PARTITION_MAPPING_TYPEREFERENCE) - } catch { - case e : Exception => - throw new SamzaException("Exception while deserializing changelog partition mapping", e) - } - } - - def changelogPartitionMappingToBytes(mapping: util.Map[TaskName, java.lang.Integer]) = { - jsonMapper.writeValueAsBytes(new util.HashMap[TaskName, java.lang.Integer](mapping)) - } -} - -object CheckpointSerde { - val PARTITION_MAPPING_TYPEREFERENCE = new TypeReference[util.HashMap[TaskName, java.lang.Integer]]() {} -} +} \ No newline at end of file diff --git a/samza-core/src/main/scala/org/apache/samza/util/FileUtil.scala b/samza-core/src/main/scala/org/apache/samza/util/FileUtil.scala index d416340686..029868722b 100644 --- a/samza-core/src/main/scala/org/apache/samza/util/FileUtil.scala +++ b/samza-core/src/main/scala/org/apache/samza/util/FileUtil.scala @@ -146,7 +146,14 @@ class FileUtil extends Logging { } def createDirectories(path: Path): Path = { - Files.createDirectories(path) + // Files.createDirectories throws FileAlreadyExistsException if the path already exists + // but the last dir in the path is a symlink to another dir. Check explicitly if the path + // already exists to avoid this behavior. + if (!Files.exists(path)) { + Files.createDirectories(path) + } else { + path + } } /** diff --git a/samza-core/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaChangelogSSPOffset.java b/samza-core/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaChangelogSSPOffset.java new file mode 100644 index 0000000000..324a349daf --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaChangelogSSPOffset.java @@ -0,0 +1,73 @@ +/* + * 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.samza.checkpoint.kafka; + +import org.apache.samza.checkpoint.CheckpointId; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + + +public class TestKafkaChangelogSSPOffset { + @Test + public void testSerializeDeserialize() { + KafkaChangelogSSPOffset kafkaChangelogSSPOffset = new KafkaChangelogSSPOffset(CheckpointId.create(), "offset"); + KafkaChangelogSSPOffset deserializedKafkaChangelogSSPOffset = KafkaChangelogSSPOffset.fromString(kafkaChangelogSSPOffset.toString()); + + assertEquals(kafkaChangelogSSPOffset.getCheckpointId(), deserializedKafkaChangelogSSPOffset.getCheckpointId()); + assertEquals("offset", deserializedKafkaChangelogSSPOffset.getChangelogOffset()); + assertEquals(kafkaChangelogSSPOffset, deserializedKafkaChangelogSSPOffset); + } + + @Test + public void testSerializeDeserializeNullOffsets() { + KafkaChangelogSSPOffset kafkaChangelogSSPOffset = new KafkaChangelogSSPOffset(CheckpointId.create(), null); + KafkaChangelogSSPOffset deserializedKafkaChangelogSSPOffset = KafkaChangelogSSPOffset.fromString(kafkaChangelogSSPOffset.toString()); + + assertEquals(kafkaChangelogSSPOffset.getCheckpointId(), deserializedKafkaChangelogSSPOffset.getCheckpointId()); + assertNull(deserializedKafkaChangelogSSPOffset.getChangelogOffset()); + assertEquals(kafkaChangelogSSPOffset, deserializedKafkaChangelogSSPOffset); + } + + @Test + public void testSerializationFormatForBackwardsCompatibility() { + KafkaChangelogSSPOffset kafkaChangelogSSPOffset = new KafkaChangelogSSPOffset(CheckpointId.create(), "offset"); + + // WARNING: This format is written to persisted remotes stores and local files, making a change in the format + // would be backwards incompatible + String expectedSerializationFormat = kafkaChangelogSSPOffset.getCheckpointId() + KafkaChangelogSSPOffset.SEPARATOR + + kafkaChangelogSSPOffset.getChangelogOffset(); + assertEquals(expectedSerializationFormat, kafkaChangelogSSPOffset.toString()); + assertEquals(kafkaChangelogSSPOffset, KafkaChangelogSSPOffset.fromString(expectedSerializationFormat)); + } + + @Test + public void testNullSerializationFormatForBackwardsCompatibility() { + KafkaChangelogSSPOffset kafkaChangelogSSPOffset = new KafkaChangelogSSPOffset(CheckpointId.create(), null); + + // WARNING: This format is written to persisted remotes stores and local files, making a change in the format + // would be backwards incompatible + String expectedSerializationFormat = kafkaChangelogSSPOffset.getCheckpointId() + KafkaChangelogSSPOffset.SEPARATOR + + "null"; + assertEquals(expectedSerializationFormat, kafkaChangelogSSPOffset.toString()); + assertEquals(kafkaChangelogSSPOffset, KafkaChangelogSSPOffset.fromString(expectedSerializationFormat)); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaStateCheckpointMarker.java b/samza-core/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaStateCheckpointMarker.java new file mode 100644 index 0000000000..75eacff252 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaStateCheckpointMarker.java @@ -0,0 +1,114 @@ +/* + * 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.samza.checkpoint.kafka; + +import com.google.common.collect.ImmutableMap; +import java.util.Collections; +import java.util.Map; +import org.apache.samza.Partition; +import org.apache.samza.system.SystemStreamPartition; +import org.junit.Test; +import scala.Option; + +import static org.apache.samza.checkpoint.kafka.KafkaStateCheckpointMarker.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + + +public class TestKafkaStateCheckpointMarker { + @Test + public void testSerializeDeserialize() { + SystemStreamPartition ssp = new SystemStreamPartition("system", "stream", new Partition(1)); + KafkaStateCheckpointMarker marker = new KafkaStateCheckpointMarker(ssp, "offset"); + KafkaStateCheckpointMarker deserializedMarker = KafkaStateCheckpointMarker + .deserialize(KafkaStateCheckpointMarker.serialize(marker)); + + assertEquals(MARKER_VERSION, deserializedMarker.getVersion()); + assertEquals(marker.getChangelogOffset(), deserializedMarker.getChangelogOffset()); + assertEquals(marker.getChangelogSSP(), deserializedMarker.getChangelogSSP()); + assertEquals(marker, deserializedMarker); + } + + @Test + public void testSerializeDeserializeNullOffsets() { + SystemStreamPartition ssp = new SystemStreamPartition("system", "stream", new Partition(1)); + KafkaStateCheckpointMarker marker = new KafkaStateCheckpointMarker(ssp, null); + KafkaStateCheckpointMarker deserializedMarker = KafkaStateCheckpointMarker + .deserialize(KafkaStateCheckpointMarker.serialize(marker)); + + assertEquals(MARKER_VERSION, deserializedMarker.getVersion()); + assertNull(deserializedMarker.getChangelogOffset()); + assertEquals(marker.getChangelogSSP(), deserializedMarker.getChangelogSSP()); + assertEquals(marker, deserializedMarker); + } + + @Test + public void testStateCheckpointMarkerToSSPOffsetMap() { + SystemStreamPartition ssp1 = new SystemStreamPartition("system1", "stream1", new Partition(1)); + KafkaStateCheckpointMarker marker1 = new KafkaStateCheckpointMarker(ssp1, "offset1"); + SystemStreamPartition ssp2 = new SystemStreamPartition("system2", "stream2", new Partition(2)); + KafkaStateCheckpointMarker marker2 = new KafkaStateCheckpointMarker(ssp2, null); + Map storesToKSCM = ImmutableMap.of( + "store1", KafkaStateCheckpointMarker.serialize(marker1), + "store2", KafkaStateCheckpointMarker.serialize(marker2) + ); + Map> factoryToSCMs = ImmutableMap.of( + KAFKA_STATE_BACKEND_FACTORY_NAME, storesToKSCM, + "factory2", Collections.EMPTY_MAP // factory2 should be ignored + ); + + Map> sspToOffsetOption = KafkaStateCheckpointMarker + .scmsToSSPOffsetMap(factoryToSCMs); + + assertEquals(2, sspToOffsetOption.size()); + assertTrue(sspToOffsetOption.containsKey(ssp1)); + assertEquals(sspToOffsetOption.get(ssp1).get(), marker1.getChangelogOffset()); + assertEquals(ssp1, marker1.getChangelogSSP()); + assertTrue(sspToOffsetOption.containsKey(ssp2)); + assertTrue(sspToOffsetOption.get(ssp2).isEmpty()); + } + + @Test + public void testStateCheckpointMarkerToSSPOffsetMapNoFactoryFound() { + Map> factoryToSCMs = ImmutableMap.of( + "factory1", Collections.EMPTY_MAP, // factory1 should be ignored + "factory2", Collections.EMPTY_MAP // factory2 should be ignored + ); + + Map> sspToOffsetOption = KafkaStateCheckpointMarker + .scmsToSSPOffsetMap(factoryToSCMs); + + assertEquals(0, sspToOffsetOption.size()); + } + + @Test(expected = IllegalArgumentException.class) + public void testStateCheckpointMarkerToSSPOffsetMapDeserializationError() { + Map storesToSCM = ImmutableMap.of( + "store1", "blobId-1234" + ); + Map> factoryToSCMs = ImmutableMap.of( + "factory2", Collections.EMPTY_MAP, // factory2 should be ignored + KAFKA_STATE_BACKEND_FACTORY_NAME, storesToSCM + ); + + KafkaStateCheckpointMarker.scmsToSSPOffsetMap(factoryToSCMs); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/serializers/TestCheckpointV2Serde.java b/samza-core/src/test/java/org/apache/samza/serializers/TestCheckpointV2Serde.java new file mode 100644 index 0000000000..d3b2d40079 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/serializers/TestCheckpointV2Serde.java @@ -0,0 +1,96 @@ +/* + * 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.samza.serializers; + +import java.util.HashMap; +import java.util.Map; +import org.apache.samza.Partition; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.checkpoint.CheckpointV2; +import org.apache.samza.system.SystemStreamPartition; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + + +public class TestCheckpointV2Serde { + + @Test + public void testCheckpointV2Serde() { + CheckpointV2Serde serde = new CheckpointV2Serde(); + Map offsets = new HashMap<>(); + SystemStreamPartition systemStreamPartition = new SystemStreamPartition("test-system", "test-stream", new Partition(777)); + offsets.put(systemStreamPartition, "1"); + + // State Checkpoint marker + Map> factoryStateCheckpointMarkersMap = new HashMap<>(); + Map stateCheckpointMarkersMap = new HashMap<>(); + stateCheckpointMarkersMap.put("store1", "marker1"); + stateCheckpointMarkersMap.put("store2", "marker2"); + + Map stateCheckpointMarkersMap2 = new HashMap<>(); + stateCheckpointMarkersMap2.put("store1", "marker3"); + stateCheckpointMarkersMap2.put("store2", "marker4"); + + factoryStateCheckpointMarkersMap.put("factory1", stateCheckpointMarkersMap); + factoryStateCheckpointMarkersMap.put("factory2", stateCheckpointMarkersMap2); + + CheckpointId checkpointId = CheckpointId.create(); + + CheckpointV2 checkpoint = new CheckpointV2(checkpointId, offsets, factoryStateCheckpointMarkersMap); + CheckpointV2 deserializedCheckpoint = serde.fromBytes(serde.toBytes(checkpoint)); + + // Validate input checkpoints + assertEquals(checkpointId, deserializedCheckpoint.getCheckpointId()); + assertEquals("1", deserializedCheckpoint.getOffsets().get(systemStreamPartition)); + assertEquals(1, deserializedCheckpoint.getOffsets().size()); + + // Validate state checkpoints + assertEquals(2, deserializedCheckpoint.getStateCheckpointMarkers().size()); + assertTrue(deserializedCheckpoint.getStateCheckpointMarkers().containsKey("factory1")); + assertEquals(stateCheckpointMarkersMap, deserializedCheckpoint.getStateCheckpointMarkers().get("factory1")); + assertTrue(deserializedCheckpoint.getStateCheckpointMarkers().containsKey("factory2")); + assertEquals(stateCheckpointMarkersMap2, deserializedCheckpoint.getStateCheckpointMarkers().get("factory2")); + } + + @Test + public void testCheckpointV2SerdeStatelessJob() { + CheckpointV2Serde serde = new CheckpointV2Serde(); + Map offsets = new HashMap<>(); + SystemStreamPartition systemStreamPartition = new SystemStreamPartition("test-system", "test-stream", new Partition(777)); + offsets.put(systemStreamPartition, "1"); + + // State Checkpoint marker + CheckpointId checkpointId = CheckpointId.create(); + + + CheckpointV2 checkpoint = new CheckpointV2(checkpointId, offsets, new HashMap<>()); + CheckpointV2 deserializedCheckpoint = serde.fromBytes(serde.toBytes(checkpoint)); + + // Validate input checkpoints + assertEquals(checkpointId, deserializedCheckpoint.getCheckpointId()); + assertEquals("1", deserializedCheckpoint.getOffsets().get(systemStreamPartition)); + assertEquals(1, deserializedCheckpoint.getOffsets().size()); + + // No state checkpoints, but a map is still created + assertEquals(0, deserializedCheckpoint.getStateCheckpointMarkers().size()); + } +} diff --git a/samza-core/src/test/scala/org/apache/samza/checkpoint/TestCheckpointTool.scala b/samza-core/src/test/scala/org/apache/samza/checkpoint/TestCheckpointTool.scala index 777a6a06d5..5bcbdcd2a2 100644 --- a/samza-core/src/test/scala/org/apache/samza/checkpoint/TestCheckpointTool.scala +++ b/samza-core/src/test/scala/org/apache/samza/checkpoint/TestCheckpointTool.scala @@ -91,9 +91,9 @@ class TestCheckpointTool extends AssertionsForJUnit with MockitoSugar { when(TestCheckpointTool.systemAdmin.getSystemStreamPartitionCounts(Set("foo").asJava, 0)) .thenReturn(Map("foo" -> metadata).asJava) when(TestCheckpointTool.checkpointManager.readLastCheckpoint(tn0)) - .thenReturn(new Checkpoint(Map(new SystemStreamPartition("test", "foo", p0) -> "1234").asJava)) + .thenReturn(new CheckpointV1(Map(new SystemStreamPartition("test", "foo", p0) -> "1234").asJava)) when(TestCheckpointTool.checkpointManager.readLastCheckpoint(tn1)) - .thenReturn(new Checkpoint(Map(new SystemStreamPartition("test", "foo", p1) -> "4321").asJava)) + .thenReturn(new CheckpointV1(Map(new SystemStreamPartition("test", "foo", p1) -> "4321").asJava)) } @Test @@ -113,9 +113,9 @@ class TestCheckpointTool extends AssertionsForJUnit with MockitoSugar { val checkpointTool = CheckpointTool(config, toOverwrite) checkpointTool.run() verify(TestCheckpointTool.checkpointManager) - .writeCheckpoint(tn0, new Checkpoint(Map(new SystemStreamPartition("test", "foo", p0) -> "42").asJava)) + .writeCheckpoint(tn0, new CheckpointV1(Map(new SystemStreamPartition("test", "foo", p0) -> "42").asJava)) verify(TestCheckpointTool.checkpointManager) - .writeCheckpoint(tn1, new Checkpoint(Map(new SystemStreamPartition("test", "foo", p1) -> "43").asJava)) + .writeCheckpoint(tn1, new CheckpointV1(Map(new SystemStreamPartition("test", "foo", p1) -> "43").asJava)) } @Test @@ -157,9 +157,9 @@ class TestCheckpointTool extends AssertionsForJUnit with MockitoSugar { checkpointTool.run() verify(TestCheckpointTool.checkpointManager) - .writeCheckpoint(tn0, new Checkpoint(Map(new SystemStreamPartition("test", "foo", p0) -> "42").asJava)) + .writeCheckpoint(tn0, new CheckpointV1(Map(new SystemStreamPartition("test", "foo", p0) -> "42").asJava)) verify(TestCheckpointTool.checkpointManager) - .writeCheckpoint(tn1, new Checkpoint(Map(new SystemStreamPartition("test", "foo", p1) -> "43").asJava)) + .writeCheckpoint(tn1, new CheckpointV1(Map(new SystemStreamPartition("test", "foo", p1) -> "43").asJava)) // Two configurations job.id, job.name are populated in the coordinator config by SamzaRuntime and it is not present in generated config. assert(generatedConfigs.entrySet().containsAll(TestCheckpointTool.coordinatorConfig.entrySet())) diff --git a/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala b/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala index 677504d8c6..3949ecf186 100644 --- a/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala +++ b/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala @@ -100,7 +100,7 @@ class TestOffsetManager { } startpointManagerUtil.getStartpointManager.start assertFalse(startpointManagerUtil.getStartpointManager.getFanOutForTask(taskName).containsKey(systemStreamPartition)) // Startpoint should delete after checkpoint commit - val expectedCheckpoint = new Checkpoint(Map(systemStreamPartition -> "47").asJava) + val expectedCheckpoint = new CheckpointV1(Map(systemStreamPartition -> "47").asJava) assertEquals(expectedCheckpoint, checkpointManager.readLastCheckpoint(taskName)) startpointManagerUtil.stop } @@ -260,7 +260,7 @@ class TestOffsetManager { val systemStreamPartition = new SystemStreamPartition(systemStream, partition) val testStreamMetadata = new SystemStreamMetadata(systemStream.getStream, Map(partition -> new SystemStreamPartitionMetadata("0", "1", "2")).asJava) val systemStreamMetadata = Map(systemStream -> testStreamMetadata) - val checkpoint = new Checkpoint(Map(systemStreamPartition -> "45").asJava) + val checkpoint = new CheckpointV1(Map(systemStreamPartition -> "45").asJava) val checkpointManager = getCheckpointManager(systemStreamPartition, taskName) val config = new MapConfig(Map( "systems.test-system.samza.offset.default" -> "oldest", @@ -289,7 +289,7 @@ class TestOffsetManager { partition1 -> new SystemStreamPartitionMetadata("0", "1", "2"), partition2 -> new SystemStreamPartitionMetadata("3", "4", "5")).asJava) val systemStreamMetadata = Map(systemStream -> testStreamMetadata) - val checkpoint = new Checkpoint(Map(systemStreamPartition1 -> "45").asJava) + val checkpoint = new CheckpointV1(Map(systemStreamPartition1 -> "45").asJava) // Checkpoint manager only has partition 1. val checkpointManager = getCheckpointManager(systemStreamPartition1, taskName1) val startpointManagerUtil = getStartpointManagerUtil() @@ -397,8 +397,7 @@ class TestOffsetManager { val testStreamMetadata2 = new SystemStreamMetadata(systemStream2.getStream, Map(partition -> new SystemStreamPartitionMetadata("0", "1", "2")).asJava) val systemStreamMetadata = Map(systemStream -> testStreamMetadata, systemStream2->testStreamMetadata2) val config = new MapConfig - val checkpointManager = getCheckpointManager1(systemStreamPartition, - new Checkpoint(Map(systemStreamPartition -> "45", systemStreamPartition2 -> "100").asJava), + val checkpointManager = getCheckpointManager1(new CheckpointV1(Map(systemStreamPartition -> "45", systemStreamPartition2 -> "100").asJava), taskName) val startpointManagerUtil = getStartpointManagerUtil() val consumer = new SystemConsumerWithCheckpointCallback @@ -423,9 +422,9 @@ class TestOffsetManager { // Should get offset 45 back from the checkpoint manager, which is last processed, and system admin should return 46 as starting offset. assertTrue(startpointManagerUtil.getStartpointManager.getFanOutForTask(taskName).containsKey(systemStreamPartition)) val offsetsToCheckpoint = new java.util.HashMap[SystemStreamPartition, String]() - offsetsToCheckpoint.putAll(offsetManager.buildCheckpoint(taskName).getOffsets) + offsetsToCheckpoint.putAll(offsetManager.getLastProcessedOffsets(taskName)) offsetsToCheckpoint.put(unregisteredSystemStreamPartition, "50") - offsetManager.writeCheckpoint(taskName, new Checkpoint(offsetsToCheckpoint)) + offsetManager.writeCheckpoint(taskName, new CheckpointV1(offsetsToCheckpoint)) intercept[IllegalStateException] { // StartpointManager should stop after last fan out is removed @@ -499,15 +498,15 @@ class TestOffsetManager { offsetManager.update(taskName, systemStreamPartition, "46") // Get checkpoint snapshot like we do at the beginning of TaskInstance.commit() - val checkpoint46 = offsetManager.buildCheckpoint(taskName) + val checkpoint46 = offsetManager.getLastProcessedOffsets(taskName) offsetManager.update(taskName, systemStreamPartition, "47") // Offset updated before checkpoint - offsetManager.writeCheckpoint(taskName, checkpoint46) + offsetManager.writeCheckpoint(taskName, new CheckpointV1(checkpoint46)) assertEquals(Some("47"), offsetManager.getLastProcessedOffset(taskName, systemStreamPartition)) assertEquals("46", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition).getValue) // Now write the checkpoint for the latest offset - val checkpoint47 = offsetManager.buildCheckpoint(taskName) - offsetManager.writeCheckpoint(taskName, checkpoint47) + val checkpoint47 = offsetManager.getLastProcessedOffsets(taskName) + offsetManager.writeCheckpoint(taskName, new CheckpointV1(checkpoint47)) startpointManagerUtil.stop assertEquals(Some("47"), offsetManager.getLastProcessedOffset(taskName, systemStreamPartition)) assertEquals("47", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition).getValue) @@ -582,7 +581,7 @@ class TestOffsetManager { // Utility method to create and write checkpoint in one statement def checkpoint(offsetManager: OffsetManager, taskName: TaskName): Unit = { - offsetManager.writeCheckpoint(taskName, offsetManager.buildCheckpoint(taskName)) + offsetManager.writeCheckpoint(taskName, new CheckpointV1(offsetManager.getLastProcessedOffsets(taskName))) } class SystemConsumerWithCheckpointCallback extends SystemConsumer with CheckpointListener{ @@ -602,10 +601,10 @@ class TestOffsetManager { } private def getCheckpointManager(systemStreamPartition: SystemStreamPartition, taskName:TaskName = new TaskName("taskName")) = { - getCheckpointManager1(systemStreamPartition, new Checkpoint(Map(systemStreamPartition -> "45").asJava), taskName) + getCheckpointManager1(new CheckpointV1(Map(systemStreamPartition -> "45").asJava), taskName) } - private def getCheckpointManager1(systemStreamPartition: SystemStreamPartition, checkpoint: Checkpoint, taskName:TaskName = new TaskName("taskName")) = { + private def getCheckpointManager1(checkpoint: Checkpoint, taskName:TaskName = new TaskName("taskName")) = { new CheckpointManager { var isStarted = false var isStopped = false @@ -617,9 +616,6 @@ class TestOffsetManager { def writeCheckpoint(taskName: TaskName, checkpoint: Checkpoint) { checkpoints += taskName -> checkpoint } def readLastCheckpoint(taskName: TaskName) = checkpoints.getOrElse(taskName, null) def stop { isStopped = true } - - // Only for testing purposes - not present in actual checkpoint manager - def getOffets = Map(taskName -> checkpoint.getOffsets.asScala.toMap) } } diff --git a/samza-core/src/test/scala/org/apache/samza/checkpoint/file/TestFileSystemCheckpointManager.scala b/samza-core/src/test/scala/org/apache/samza/checkpoint/file/TestFileSystemCheckpointManager.scala index f7839af876..8cc6770b6a 100644 --- a/samza-core/src/test/scala/org/apache/samza/checkpoint/file/TestFileSystemCheckpointManager.scala +++ b/samza-core/src/test/scala/org/apache/samza/checkpoint/file/TestFileSystemCheckpointManager.scala @@ -20,13 +20,15 @@ package org.apache.samza.checkpoint.file import java.io.File + import scala.collection.JavaConverters._ import java.util.Random + import org.junit.Assert._ import org.junit.{After, Before, Test} import org.apache.samza.SamzaException import org.apache.samza.Partition -import org.apache.samza.checkpoint.Checkpoint +import org.apache.samza.checkpoint.{Checkpoint, CheckpointV1} import org.apache.samza.system.SystemStreamPartition import org.apache.samza.container.TaskName import org.junit.rules.TemporaryFolder @@ -52,17 +54,17 @@ class TestFileSystemCheckpointManager { @Test def testReadForCheckpointFileThatDoesExistShouldReturnProperCheckpoint { - val cp = new Checkpoint(Map( + val cp = new CheckpointV1(Map( new SystemStreamPartition("a", "b", new Partition(0)) -> "c", new SystemStreamPartition("a", "c", new Partition(1)) -> "d", new SystemStreamPartition("b", "d", new Partition(2)) -> "e").asJava) - var readCp:Checkpoint = null + var readCp:CheckpointV1 = null val cpm = new FileSystemCheckpointManager("some-job-name", tempFolder.getRoot) cpm.start cpm.writeCheckpoint(taskName, cp) - readCp = cpm.readLastCheckpoint(taskName) + readCp = cpm.readLastCheckpoint(taskName).asInstanceOf[CheckpointV1] cpm.stop assertNotNull(readCp) diff --git a/samza-core/src/test/scala/org/apache/samza/serializers/TestCheckpointSerde.scala b/samza-core/src/test/scala/org/apache/samza/serializers/TestCheckpointV1Serde.scala similarity index 69% rename from samza-core/src/test/scala/org/apache/samza/serializers/TestCheckpointSerde.scala rename to samza-core/src/test/scala/org/apache/samza/serializers/TestCheckpointV1Serde.scala index c2060e0d03..02f2e59474 100644 --- a/samza-core/src/test/scala/org/apache/samza/serializers/TestCheckpointSerde.scala +++ b/samza-core/src/test/scala/org/apache/samza/serializers/TestCheckpointV1Serde.scala @@ -22,46 +22,30 @@ package org.apache.samza.serializers import java.util import org.apache.samza.Partition -import org.apache.samza.checkpoint.Checkpoint +import org.apache.samza.checkpoint.{CheckpointV1} import org.apache.samza.container.TaskName import org.apache.samza.system.SystemStreamPartition import org.junit.Assert._ import org.junit.Test import scala.collection.JavaConverters._ -import scala.collection.mutable -class TestCheckpointSerde { +class TestCheckpointV1Serde { @Test def testExactlyOneOffset { - val serde = new CheckpointSerde + val serde = new CheckpointV1Serde var offsets = Map[SystemStreamPartition, String]() val systemStreamPartition = new SystemStreamPartition("test-system", "test-stream", new Partition(777)) offsets += systemStreamPartition -> "1" - val deserializedOffsets = serde.fromBytes(serde.toBytes(new Checkpoint(offsets.asJava))) + val deserializedOffsets = serde.fromBytes(serde.toBytes(new CheckpointV1(offsets.asJava))) assertEquals("1", deserializedOffsets.getOffsets.get(systemStreamPartition)) assertEquals(1, deserializedOffsets.getOffsets.size) } - @Test - def testChangelogPartitionMappingRoundTrip { - val mapping = new util.HashMap[TaskName, java.lang.Integer]() - mapping.put(new TaskName("Ted"), 0) - mapping.put(new TaskName("Dougal"), 1) - mapping.put(new TaskName("Jack"), 2) - - val checkpointSerde = new CheckpointSerde - val asBytes = checkpointSerde.changelogPartitionMappingToBytes(mapping) - val backToMap = checkpointSerde.changelogPartitionMappingFromBytes(asBytes) - - assertEquals(mapping, backToMap) - assertNotSame(mapping, backToMap) - } - @Test def testNullCheckpointSerde: Unit = { val checkpointBytes = null.asInstanceOf[Array[Byte]] - val checkpointSerde = new CheckpointSerde + val checkpointSerde = new CheckpointV1Serde val checkpoint = checkpointSerde.fromBytes(checkpointBytes) assertNull(checkpoint) } diff --git a/samza-core/src/test/scala/org/apache/samza/util/TestFileUtil.scala b/samza-core/src/test/scala/org/apache/samza/util/TestFileUtil.scala index 6604ae0262..104f983d9a 100644 --- a/samza-core/src/test/scala/org/apache/samza/util/TestFileUtil.scala +++ b/samza-core/src/test/scala/org/apache/samza/util/TestFileUtil.scala @@ -21,11 +21,15 @@ package org.apache.samza.util -import java.io.{File, FileInputStream, FileOutputStream, ObjectInputStream, ObjectOutputStream} +import org.apache.samza.testUtils.FileUtil -import org.junit.Assert.{assertEquals, assertNull, assertTrue} +import java.io.{File, FileInputStream, FileOutputStream, ObjectInputStream, ObjectOutputStream} +import org.junit.Assert.{assertEquals, assertNull, assertTrue, fail} import org.junit.Test +import java.nio.file.{FileAlreadyExistsException, Files, Paths} +import scala.util.Random + class TestFileUtil { val data = "100" val fileUtil = new FileUtil() @@ -104,4 +108,45 @@ class TestFileUtil { // Check data returned assertNull(result) } + + /** + * Files.createDirectories fails with a FileAlreadyExistsException if the last directory + * in the path already exists but is a symlink to another directory. It works correctly + * if one of the intermediate directory is a symlink. Verify this behavior and + * test that the util method handles this correctly. + */ + @Test + def testCreateDirectoriesWithSymlinks(): Unit = { + /** + * Directory structure: + * /tmp/samza-file-util-RANDOM + * /tmp/samza-file-util-RANDOM-symlink (symlink to dir above) + * /tmp/samza-file-util-RANDOM/subdir (created via the symlink above) + */ + val tmpDirPath = Paths.get(FileUtil.TMP_DIR) + val tmpSubDirName = "samza-file-util-" + Random.nextInt() + val tmpSubDirSymlinkName = tmpSubDirName + "-symlink" + + val tmpSubDirPath = Paths.get(FileUtil.TMP_DIR, tmpSubDirName); + fileUtil.createDirectories(tmpSubDirPath) + + val tmpSymlinkPath = Paths.get(FileUtil.TMP_DIR, tmpSubDirSymlinkName) + Files.createSymbolicLink(tmpSymlinkPath, tmpDirPath); + + try { + Files.createDirectories(tmpSymlinkPath) + fail("Should have thrown a FileAlreadyExistsException since last dir in path already " + + "exists and is a symlink") + } catch { + case e: FileAlreadyExistsException => + // ignore and continue + } + + // test that the util method handles this correctly and does not throw an exception + fileUtil.createDirectories(tmpSymlinkPath) + + // verify that subdirs can be created via symlinks correctly. + val tmpSubSubDirPath = Paths.get(FileUtil.TMP_DIR, tmpSubDirName + "-symlink", "subdir") + fileUtil.createDirectories(tmpSubSubDirPath) + } } diff --git a/samza-kafka/src/main/java/org/apache/samza/checkpoint/kafka/KafkaCheckpointLogKey.java b/samza-kafka/src/main/java/org/apache/samza/checkpoint/kafka/KafkaCheckpointLogKey.java index 05114f97b1..0f0b071c5c 100644 --- a/samza-kafka/src/main/java/org/apache/samza/checkpoint/kafka/KafkaCheckpointLogKey.java +++ b/samza-kafka/src/main/java/org/apache/samza/checkpoint/kafka/KafkaCheckpointLogKey.java @@ -26,7 +26,8 @@ */ public class KafkaCheckpointLogKey { - public static final String CHECKPOINT_KEY_TYPE = "checkpoint"; + public static final String CHECKPOINT_V1_KEY_TYPE = "checkpoint"; + public static final String CHECKPOINT_V2_KEY_TYPE = "checkpoint-v2"; /** * The SystemStreamPartitionGrouperFactory configured for this job run. Since, checkpoints of different * groupers are not compatible, we persist and validate them across job runs. @@ -48,8 +49,9 @@ public KafkaCheckpointLogKey(String type, TaskName taskName, String grouperFacto Preconditions.checkNotNull(type); Preconditions.checkState(!grouperFactoryClassName.isEmpty(), "Empty grouper factory class provided"); - Preconditions.checkState(type.equals(CHECKPOINT_KEY_TYPE), String.format("Invalid type provided for checkpoint key. " + - "Expected: (%s) Actual: (%s)", CHECKPOINT_KEY_TYPE, type)); + Preconditions.checkState(CHECKPOINT_V1_KEY_TYPE.equals(type) || CHECKPOINT_V2_KEY_TYPE.equals(type), + String.format("Invalid type provided for checkpoint key. Expected: (%s or %s) Actual: (%s)", + CHECKPOINT_V1_KEY_TYPE, CHECKPOINT_V2_KEY_TYPE, type)); this.grouperFactoryClassName = grouperFactoryClassName; this.taskName = taskName; diff --git a/samza-kafka/src/main/java/org/apache/samza/checkpoint/kafka/KafkaCheckpointLogKeySerde.java b/samza-kafka/src/main/java/org/apache/samza/checkpoint/kafka/KafkaCheckpointLogKeySerde.java index b00f12e779..c5c431b624 100644 --- a/samza-kafka/src/main/java/org/apache/samza/checkpoint/kafka/KafkaCheckpointLogKeySerde.java +++ b/samza-kafka/src/main/java/org/apache/samza/checkpoint/kafka/KafkaCheckpointLogKeySerde.java @@ -24,7 +24,6 @@ import org.apache.samza.container.TaskName; import org.apache.samza.serializers.Serde; -import java.util.Arrays; import java.util.LinkedHashMap; /** @@ -58,16 +57,13 @@ public byte[] toBytes(KafkaCheckpointLogKey key) { public KafkaCheckpointLogKey fromBytes(byte[] bytes) { try { LinkedHashMap deserializedKey = MAPPER.readValue(bytes, LinkedHashMap.class); + String key = deserializedKey.get(TYPE_FIELD); - if (!KafkaCheckpointLogKey.CHECKPOINT_KEY_TYPE.equals(deserializedKey.get(TYPE_FIELD))) { - throw new IllegalArgumentException(String.format("Invalid key detected. Type of the key is %s", deserializedKey.get(TYPE_FIELD))); - } - - return new KafkaCheckpointLogKey(deserializedKey.get(TYPE_FIELD), new TaskName(deserializedKey.get(TASK_NAME_FIELD)), deserializedKey.get(SSP_GROUPER_FACTORY_FIELD) - ); + return new KafkaCheckpointLogKey(key, new TaskName(deserializedKey.get(TASK_NAME_FIELD)), + deserializedKey.get(SSP_GROUPER_FACTORY_FIELD)); } catch (Exception e) { throw new SamzaException(String.format("Exception in de-serializing checkpoint bytes: %s", - Arrays.toString(bytes)), e); + new String(bytes)), e); } } } diff --git a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala index 757e7ae975..a83a34ba2c 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala @@ -22,18 +22,16 @@ package org.apache.samza.checkpoint.kafka import java.util.Collections import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicReference - import com.google.common.annotations.VisibleForTesting import com.google.common.base.Preconditions -import org.apache.samza.checkpoint.{Checkpoint, CheckpointManager} +import org.apache.samza.checkpoint.{Checkpoint, CheckpointManager, CheckpointV1, CheckpointV2} import org.apache.samza.config.{Config, JobConfig, TaskConfig} import org.apache.samza.container.TaskName -import org.apache.samza.serializers.Serde +import org.apache.samza.serializers.{CheckpointV1Serde, CheckpointV2Serde, Serde} import org.apache.samza.metrics.MetricsRegistry -import org.apache.samza.serializers.CheckpointSerde import org.apache.samza.system._ import org.apache.samza.system.kafka.KafkaStreamSpec -import org.apache.samza.util.{ExponentialSleepStrategy, Logging} +import org.apache.samza.util.Logging import org.apache.samza.{Partition, SamzaException} import scala.collection.mutable @@ -53,7 +51,8 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, validateCheckpoint: Boolean, config: Config, metricsRegistry: MetricsRegistry, - checkpointMsgSerde: Serde[Checkpoint] = new CheckpointSerde, + checkpointV1MsgSerde: Serde[CheckpointV1] = new CheckpointV1Serde, + checkpointV2MsgSerde: Serde[CheckpointV2] = new CheckpointV2Serde, checkpointKeySerde: Serde[KafkaCheckpointLogKey] = new KafkaCheckpointLogKeySerde) extends CheckpointManager with Logging { var MaxRetryDurationInMillis: Long = TimeUnit.MINUTES.toMillis(15) @@ -81,6 +80,8 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, // for active containers, this will be set to true, while false for standby containers. val stopConsumerAfterFirstRead: Boolean = new TaskConfig(config).getCheckpointManagerConsumerStopAfterFirstRead + val checkpointReadVersion: Short = new TaskConfig(config).getCheckpointReadVersion + /** * Create checkpoint stream prior to start. * @@ -159,19 +160,7 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, * @inheritdoc */ override def writeCheckpoint(taskName: TaskName, checkpoint: Checkpoint) { - val key = new KafkaCheckpointLogKey(KafkaCheckpointLogKey.CHECKPOINT_KEY_TYPE, taskName, expectedGrouperFactory) - val keyBytes = try { - checkpointKeySerde.toBytes(key) - } catch { - case e: Exception => throw new SamzaException(s"Exception when writing checkpoint-key for $taskName: $checkpoint", e) - } - val msgBytes = try { - checkpointMsgSerde.toBytes(checkpoint) - } catch { - case e: Exception => throw new SamzaException(s"Exception when writing checkpoint for $taskName: $checkpoint", e) - } - - val envelope = new OutgoingMessageEnvelope(checkpointSsp, keyBytes, msgBytes) + val envelope = buildOutgoingMessageEnvelope(taskName, checkpoint) // Used for exponential backoff retries on failure in sending messages through producer. val startTimeInMillis: Long = System.currentTimeMillis() @@ -188,7 +177,7 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, } catch { case exception: Exception => { producerException = exception - warn(s"Retrying failed checkpoint write to key: $key, checkpoint: $checkpoint for task: $taskName", exception) + warn(s"Retrying failed write for checkpoint: $checkpoint for task: $taskName", exception) // TODO: Remove this producer recreation logic after SAMZA-1393. val newProducer: SystemProducer = getSystemProducer() producerCreationLock.synchronized { @@ -276,10 +265,10 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, checkpointKeySerde.fromBytes(keyBytes) } catch { case e: Exception => if (validateCheckpoint) { - throw new SamzaException(s"Exception while serializing checkpoint-key. " + + throw new SamzaException(s"Exception while deserializing checkpoint-key. " + s"Topic: $checkpointTopic Offset: $offset", e) } else { - warn(s"Ignoring exception while serializing checkpoint-key. Topic: $checkpointTopic Offset: $offset", e) + warn(s"Ignoring exception while deserializing checkpoint-key. Topic: $checkpointTopic Offset: $offset", e) null } } @@ -295,17 +284,27 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, } } - // If the type of the key is not KafkaCheckpointLogKey.CHECKPOINT_KEY_TYPE, it can safely be ignored. - if (KafkaCheckpointLogKey.CHECKPOINT_KEY_TYPE.equals(checkpointKey.getType)) { - val checkpointBytes = checkpointEnvelope.getMessage.asInstanceOf[Array[Byte]] - val checkpoint = try { - checkpointMsgSerde.fromBytes(checkpointBytes) - } catch { - case e: Exception => throw new SamzaException(s"Exception while serializing checkpoint-message. " + - s"Topic: $checkpointTopic Offset: $offset", e) - } - - checkpoints.put(checkpointKey.getTaskName, checkpoint) + val msgBytes = checkpointEnvelope.getMessage.asInstanceOf[Array[Byte]] + try { + // if checkpoint key version does not match configured checkpoint version to read, skip the message. + if (checkpointReadVersion == CheckpointV1.CHECKPOINT_VERSION && + KafkaCheckpointLogKey.CHECKPOINT_V1_KEY_TYPE.equals(checkpointKey.getType)) { + val msgBytes = checkpointEnvelope.getMessage.asInstanceOf[Array[Byte]] + val checkpoint = checkpointV1MsgSerde.fromBytes(msgBytes) + checkpoints.put(checkpointKey.getTaskName, checkpoint) + } else if (checkpointReadVersion == CheckpointV2.CHECKPOINT_VERSION && + KafkaCheckpointLogKey.CHECKPOINT_V2_KEY_TYPE.equals(checkpointKey.getType)) { + val checkpoint = checkpointV2MsgSerde.fromBytes(msgBytes) + checkpoints.put(checkpointKey.getTaskName, checkpoint) + } // else ignore and skip the message + } catch { + case e: Exception => + if (validateCheckpoint) { + throw new SamzaException(s"Exception while deserializing checkpoint-message. " + + s"Topic: $checkpointTopic Offset: $offset", e) + } else { + warn(s"Ignoring exception while deserializing checkpoint-msg. Topic: $checkpointTopic Offset: $offset", e) + } } } } @@ -333,4 +332,44 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, partitionMetaData.getOldestOffset } + + private def buildOutgoingMessageEnvelope[T <: Checkpoint](taskName: TaskName, checkpoint: T): OutgoingMessageEnvelope = { + checkpoint match { + case checkpointV1: CheckpointV1 => { + val key = new KafkaCheckpointLogKey( + KafkaCheckpointLogKey.CHECKPOINT_V1_KEY_TYPE, taskName, expectedGrouperFactory) + val keyBytes = try { + checkpointKeySerde.toBytes(key) + } catch { + case e: Exception => + throw new SamzaException(s"Exception when writing checkpoint-key for $taskName: $checkpoint", e) + } + val msgBytes = try { + checkpointV1MsgSerde.toBytes(checkpointV1) + } catch { + case e: Exception => + throw new SamzaException(s"Exception when writing checkpoint for $taskName: $checkpoint", e) + } + new OutgoingMessageEnvelope(checkpointSsp, keyBytes, msgBytes) + } + case checkpointV2: CheckpointV2 => { + val key = new KafkaCheckpointLogKey( + KafkaCheckpointLogKey.CHECKPOINT_V2_KEY_TYPE, taskName, expectedGrouperFactory) + val keyBytes = try { + checkpointKeySerde.toBytes(key) + } catch { + case e: Exception => + throw new SamzaException(s"Exception when writing checkpoint-key for $taskName: $checkpoint", e) + } + val msgBytes = try { + checkpointV2MsgSerde.toBytes(checkpointV2) + } catch { + case e: Exception => + throw new SamzaException(s"Exception when writing checkpoint for $taskName: $checkpoint", e) + } + new OutgoingMessageEnvelope(checkpointSsp, keyBytes, msgBytes) + } + case _ => throw new SamzaException("Unknown checkpoint version: " + checkpoint.getVersion) + } + } } diff --git a/samza-kafka/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointLogKeySerde.java b/samza-kafka/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointLogKeySerde.java index b648b1cca7..7245e70389 100644 --- a/samza-kafka/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointLogKeySerde.java +++ b/samza-kafka/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointLogKeySerde.java @@ -29,7 +29,7 @@ public class TestKafkaCheckpointLogKeySerde { @Test public void testBinaryCompatibility() { - KafkaCheckpointLogKey logKey1 = new KafkaCheckpointLogKey(KafkaCheckpointLogKey.CHECKPOINT_KEY_TYPE, + KafkaCheckpointLogKey logKey1 = new KafkaCheckpointLogKey(KafkaCheckpointLogKey.CHECKPOINT_V1_KEY_TYPE, new TaskName("Partition 0"), GroupByPartitionFactory.class.getCanonicalName()); KafkaCheckpointLogKeySerde checkpointSerde = new KafkaCheckpointLogKeySerde(); @@ -43,11 +43,21 @@ public void testBinaryCompatibility() { @Test public void testSerde() { - KafkaCheckpointLogKey key = new KafkaCheckpointLogKey(KafkaCheckpointLogKey.CHECKPOINT_KEY_TYPE, + KafkaCheckpointLogKey key = new KafkaCheckpointLogKey(KafkaCheckpointLogKey.CHECKPOINT_V1_KEY_TYPE, new TaskName("Partition 0"), GroupByPartitionFactory.class.getCanonicalName()); KafkaCheckpointLogKeySerde checkpointSerde = new KafkaCheckpointLogKeySerde(); // test that deserialize(serialize(k)) == k Assert.assertEquals(key, checkpointSerde.fromBytes(checkpointSerde.toBytes(key))); } + + @Test + public void testCheckpointTypeV2() { + KafkaCheckpointLogKey keyV2 = new KafkaCheckpointLogKey(KafkaCheckpointLogKey.CHECKPOINT_V2_KEY_TYPE, + new TaskName("Partition 0"), GroupByPartitionFactory.class.getCanonicalName()); + KafkaCheckpointLogKeySerde checkpointKeySerde = new KafkaCheckpointLogKeySerde(); + + // test that deserialize(serialize(k)) == k + Assert.assertEquals(keyV2, checkpointKeySerde.fromBytes(checkpointKeySerde.toBytes(keyV2))); + } } diff --git a/samza-kafka/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManagerJava.java b/samza-kafka/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManagerJava.java index f32040a6bf..d0e927f55c 100644 --- a/samza-kafka/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManagerJava.java +++ b/samza-kafka/src/test/java/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManagerJava.java @@ -24,13 +24,14 @@ import kafka.common.TopicAlreadyMarkedForDeletionException; import org.apache.samza.Partition; import org.apache.samza.SamzaException; -import org.apache.samza.checkpoint.Checkpoint; +import org.apache.samza.checkpoint.CheckpointV1; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.container.TaskName; import org.apache.samza.container.grouper.stream.GroupByPartitionFactory; import org.apache.samza.metrics.MetricsRegistry; -import org.apache.samza.serializers.CheckpointSerde; +import org.apache.samza.serializers.CheckpointV1Serde; +import org.apache.samza.serializers.CheckpointV2Serde; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.StreamValidationException; import org.apache.samza.system.SystemAdmin; @@ -72,7 +73,7 @@ public void testStartFailsOnTopicCreationErrors() { SystemFactory factory = newFactory(mock(SystemProducer.class), mock(SystemConsumer.class), mockAdmin); KafkaCheckpointManager checkpointManager = new KafkaCheckpointManager(checkpointSpec, factory, - true, mock(Config.class), mock(MetricsRegistry.class), null, new KafkaCheckpointLogKeySerde()); + true, mock(Config.class), mock(MetricsRegistry.class), null, null, new KafkaCheckpointLogKeySerde()); // expect an exception during startup checkpointManager.createResources(); @@ -91,7 +92,7 @@ public void testStartFailsOnTopicValidationErrors() { SystemFactory factory = newFactory(mock(SystemProducer.class), mock(SystemConsumer.class), mockAdmin); KafkaCheckpointManager checkpointManager = new KafkaCheckpointManager(checkpointSpec, factory, - true, mock(Config.class), mock(MetricsRegistry.class), null, new KafkaCheckpointLogKeySerde()); + true, mock(Config.class), mock(MetricsRegistry.class), null, null, new KafkaCheckpointLogKeySerde()); // expect an exception during startup checkpointManager.createResources(); @@ -116,7 +117,7 @@ public void testReadFailsOnSerdeExceptions() throws Exception { // wire up an exception throwing serde with the checkpointmanager KafkaCheckpointManager checkpointManager = new KafkaCheckpointManager(checkpointSpec, factory, - true, mockConfig, mock(MetricsRegistry.class), new ExceptionThrowingCheckpointSerde(), new KafkaCheckpointLogKeySerde()); + true, mockConfig, mock(MetricsRegistry.class), new ExceptionThrowingCheckpointV1Serde(), null, new KafkaCheckpointLogKeySerde()); checkpointManager.register(TASK1); checkpointManager.start(); @@ -142,7 +143,7 @@ public void testReadSucceedsOnKeySerdeExceptionsWhenValidationIsDisabled() throw // wire up an exception throwing serde with the checkpointmanager KafkaCheckpointManager checkpointManager = new KafkaCheckpointManager(checkpointSpec, factory, - false, mockConfig, mock(MetricsRegistry.class), new ExceptionThrowingCheckpointSerde(), + false, mockConfig, mock(MetricsRegistry.class), new ExceptionThrowingCheckpointV1Serde(), null, new ExceptionThrowingCheckpointKeySerde()); checkpointManager.register(TASK1); checkpointManager.start(); @@ -167,7 +168,8 @@ public void testCheckpointsAreReadFromOldestOffset() throws Exception { SystemAdmin mockAdmin = newAdmin(oldestOffset, "1"); SystemFactory factory = newFactory(mock(SystemProducer.class), mockConsumer, mockAdmin); KafkaCheckpointManager checkpointManager = new KafkaCheckpointManager(checkpointSpec, factory, - true, mockConfig, mock(MetricsRegistry.class), new CheckpointSerde(), new KafkaCheckpointLogKeySerde()); + true, mockConfig, mock(MetricsRegistry.class), new CheckpointV1Serde(), new CheckpointV2Serde(), + new KafkaCheckpointLogKeySerde()); checkpointManager.register(TASK1); // 1. verify that consumer.register is called only during checkpointManager.start. @@ -206,12 +208,13 @@ public void testAllMessagesInTheLogAreRead() throws Exception { SystemFactory factory = newFactory(mock(SystemProducer.class), mockConsumer, mockAdmin); KafkaCheckpointManager checkpointManager = new KafkaCheckpointManager(checkpointSpec, factory, - true, mockConfig, mock(MetricsRegistry.class), new CheckpointSerde(), new KafkaCheckpointLogKeySerde()); + true, mockConfig, mock(MetricsRegistry.class), new CheckpointV1Serde(), new CheckpointV2Serde(), + new KafkaCheckpointLogKeySerde()); checkpointManager.register(TASK1); checkpointManager.start(); // check that all ten messages are read, and the checkpoint is the newest message - Checkpoint checkpoint = checkpointManager.readLastCheckpoint(TASK1); + CheckpointV1 checkpoint = (CheckpointV1) checkpointManager.readLastCheckpoint(TASK1); Assert.assertEquals(checkpoint.getOffsets(), ImmutableMap.of(ssp, Integer.toString(newestOffset))); } @@ -261,15 +264,15 @@ private IncomingMessageEnvelope newCheckpointEnvelope(TaskName taskName, SystemS new KafkaCheckpointLogKey("checkpoint", taskName, GROUPER_FACTORY_CLASS); KafkaCheckpointLogKeySerde checkpointKeySerde = new KafkaCheckpointLogKeySerde(); - Checkpoint checkpointMsg = new Checkpoint(ImmutableMap.of(ssp, offset)); - CheckpointSerde checkpointMsgSerde = new CheckpointSerde(); + CheckpointV1 checkpointMsg = new CheckpointV1(ImmutableMap.of(ssp, offset)); + CheckpointV1Serde checkpointMsgSerde = new CheckpointV1Serde(); return new IncomingMessageEnvelope(CHECKPOINT_SSP, offset, checkpointKeySerde.toBytes(checkpointKey), checkpointMsgSerde.toBytes(checkpointMsg)); } - private static class ExceptionThrowingCheckpointSerde extends CheckpointSerde { - public Checkpoint fromBytes(byte[] bytes) { + private static class ExceptionThrowingCheckpointV1Serde extends CheckpointV1Serde { + public CheckpointV1 fromBytes(byte[] bytes) { throw new KafkaException("exception"); } } diff --git a/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala index 7d6db646bc..f9402921d8 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala @@ -24,12 +24,12 @@ import java.util.Properties import kafka.integration.KafkaServerTestHarness import kafka.utils.{CoreUtils, TestUtils} import com.google.common.collect.ImmutableMap -import org.apache.samza.checkpoint.Checkpoint +import org.apache.samza.checkpoint.{Checkpoint, CheckpointId, CheckpointV1, CheckpointV2} import org.apache.samza.config._ import org.apache.samza.container.TaskName import org.apache.samza.container.grouper.stream.GroupByPartitionFactory import org.apache.samza.metrics.MetricsRegistry -import org.apache.samza.serializers.CheckpointSerde +import org.apache.samza.serializers.CheckpointV1Serde import org.apache.samza.system._ import org.apache.samza.system.kafka.{KafkaStreamSpec, KafkaSystemFactory} import org.apache.samza.util.ScalaJavaUtil.JavaOptionals @@ -48,8 +48,8 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { val sspGrouperFactoryName = classOf[GroupByPartitionFactory].getCanonicalName val ssp = new SystemStreamPartition("kafka", "topic", new Partition(0)) - val checkpoint1 = new Checkpoint(ImmutableMap.of(ssp, "offset-1")) - val checkpoint2 = new Checkpoint(ImmutableMap.of(ssp, "offset-2")) + val checkpoint1 = new CheckpointV1(ImmutableMap.of(ssp, "offset-1")) + val checkpoint2 = new CheckpointV1(ImmutableMap.of(ssp, "offset-2")) val taskName = new TaskName("Partition 0") var config: Config = null @@ -88,7 +88,7 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { checkPointManager.register(taskName) checkPointManager.start - checkPointManager.writeCheckpoint(taskName, new Checkpoint(ImmutableMap.of())) + checkPointManager.writeCheckpoint(taskName, new CheckpointV1(ImmutableMap.of())) checkPointManager.stop() // Verifications after the test @@ -126,6 +126,102 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { assertEquals(checkpoint2, readCheckpoint(checkpointTopic, taskName)) } + @Test + def testCheckpointV1AndV2WriteAndReadV1(): Unit = { + val checkpointTopic = "checkpoint-topic-1" + val kcm1 = createKafkaCheckpointManager(checkpointTopic) + kcm1.register(taskName) + kcm1.createResources + kcm1.start + kcm1.stop + + // check that start actually creates the topic with log compaction enabled + val topicConfig = adminZkClient.getAllTopicConfigs().getOrElse(checkpointTopic, new Properties()) + + assertEquals(topicConfig, new KafkaConfig(config).getCheckpointTopicProperties()) + assertEquals("compact", topicConfig.get("cleanup.policy")) + assertEquals("26214400", topicConfig.get("segment.bytes")) + + // read before topic exists should result in a null checkpoint + val readCp = readCheckpoint(checkpointTopic, taskName) + assertNull(readCp) + + val checkpointV1 = new CheckpointV1(ImmutableMap.of(ssp, "offset-1")) + val checkpointV2 = new CheckpointV2(CheckpointId.create(), ImmutableMap.of(ssp, "offset-2"), + ImmutableMap.of("factory1", ImmutableMap.of("store1", "changelogOffset"))) + + // skips v2 checkpoints from checkpoint topic + writeCheckpoint(checkpointTopic, taskName, checkpointV2) + assertNull(readCheckpoint(checkpointTopic, taskName)) + + // reads latest v1 checkpoints + writeCheckpoint(checkpointTopic, taskName, checkpointV1) + assertEquals(checkpointV1, readCheckpoint(checkpointTopic, taskName)) + + // writing checkpoint v2 still returns the previous v1 checkpoint + writeCheckpoint(checkpointTopic, taskName, checkpointV2) + assertEquals(checkpointV1, readCheckpoint(checkpointTopic, taskName)) + } + + @Test + def testCheckpointV1AndV2WriteAndReadV2(): Unit = { + val checkpointTopic = "checkpoint-topic-1" + val kcm1 = createKafkaCheckpointManager(checkpointTopic) + kcm1.register(taskName) + kcm1.createResources + kcm1.start + kcm1.stop + + // check that start actually creates the topic with log compaction enabled + val topicConfig = adminZkClient.getAllTopicConfigs().getOrElse(checkpointTopic, new Properties()) + + assertEquals(topicConfig, new KafkaConfig(config).getCheckpointTopicProperties()) + assertEquals("compact", topicConfig.get("cleanup.policy")) + assertEquals("26214400", topicConfig.get("segment.bytes")) + + // read before topic exists should result in a null checkpoint + val readCp = readCheckpoint(checkpointTopic, taskName) + assertNull(readCp) + + val checkpointV1 = new CheckpointV1(ImmutableMap.of(ssp, "offset-1")) + val checkpointV2 = new CheckpointV2(CheckpointId.create(), ImmutableMap.of(ssp, "offset-2"), + ImmutableMap.of("factory1", ImmutableMap.of("store1", "changelogOffset"))) + + val overrideConfig = new MapConfig(new ImmutableMap.Builder[String, String]() + .put(JobConfig.JOB_NAME, "some-job-name") + .put(JobConfig.JOB_ID, "i001") + .put(s"systems.$checkpointSystemName.samza.factory", classOf[KafkaSystemFactory].getCanonicalName) + .put(s"systems.$checkpointSystemName.producer.bootstrap.servers", brokerList) + .put(s"systems.$checkpointSystemName.consumer.zookeeper.connect", zkConnect) + .put("task.checkpoint.system", checkpointSystemName) + .put(TaskConfig.CHECKPOINT_READ_VERSION, "2") + .build()) + + // Skips reading any v1 checkpoints + writeCheckpoint(checkpointTopic, taskName, checkpointV1) + assertNull(readCheckpoint(checkpointTopic, taskName, overrideConfig)) + + // writing a v2 checkpoint would allow reading it back + writeCheckpoint(checkpointTopic, taskName, checkpointV2) + assertEquals(checkpointV2, readCheckpoint(checkpointTopic, taskName, overrideConfig)) + + // writing v1 checkpoint is still skipped + writeCheckpoint(checkpointTopic, taskName, checkpointV1) + assertEquals(checkpointV2, readCheckpoint(checkpointTopic, taskName, overrideConfig)) + } + + @Test + def testCheckpointValidationSkipped(): Unit = { + val checkpointTopic = "checkpoint-topic-1" + val kcm1 = createKafkaCheckpointManager(checkpointTopic, serde = new MockCheckpointSerde(), + failOnTopicValidation = false) + kcm1.register(taskName) + kcm1.start + kcm1.writeCheckpoint(taskName, new CheckpointV1(ImmutableMap.of(ssp, "offset-1"))) + kcm1.readLastCheckpoint(taskName) + kcm1.stop + } + @Test def testWriteCheckpointShouldRetryFiniteTimesOnFailure(): Unit = { val checkpointTopic = "checkpoint-topic-2" @@ -142,7 +238,7 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { try { checkPointManager.register(taskName) checkPointManager.start - checkPointManager.writeCheckpoint(taskName, new Checkpoint(ImmutableMap.of())) + checkPointManager.writeCheckpoint(taskName, new CheckpointV1(ImmutableMap.of())) } catch { case _: SamzaException => info("Got SamzaException as expected.") case unexpectedException: Throwable => fail("Expected SamzaException but got %s" format unexpectedException) @@ -175,7 +271,7 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { // create topic with the wrong number of partitions createTopic(checkpointTopic, 8, new KafkaConfig(config).getCheckpointTopicProperties()) val failOnTopicValidation = false - val kcm = createKafkaCheckpointManager(checkpointTopic, new CheckpointSerde, failOnTopicValidation) + val kcm = createKafkaCheckpointManager(checkpointTopic, new CheckpointV1Serde, failOnTopicValidation) kcm.register(taskName) kcm.createResources() kcm.start() @@ -251,24 +347,25 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { .build()) } - private def createKafkaCheckpointManager(cpTopic: String, serde: CheckpointSerde = new CheckpointSerde, failOnTopicValidation: Boolean = true) = { - val kafkaConfig = new org.apache.samza.config.KafkaConfig(config) + private def createKafkaCheckpointManager(cpTopic: String, serde: CheckpointV1Serde = new CheckpointV1Serde, + failOnTopicValidation: Boolean = true, overrideConfig: Config = config) = { + val kafkaConfig = new org.apache.samza.config.KafkaConfig(overrideConfig) val props = kafkaConfig.getCheckpointTopicProperties() val systemName = kafkaConfig.getCheckpointSystem.getOrElse( throw new SamzaException("No system defined for Kafka's checkpoint manager.")) - val systemConfig = new SystemConfig(config) + val systemConfig = new SystemConfig(overrideConfig) val systemFactoryClassName = JavaOptionals.toRichOptional(systemConfig.getSystemFactory(systemName)).toOption .getOrElse(throw new SamzaException("Missing configuration: " + SystemConfig.SYSTEM_FACTORY_FORMAT format systemName)) val systemFactory = ReflectionUtil.getObj(systemFactoryClassName, classOf[SystemFactory]) val spec = new KafkaStreamSpec("id", cpTopic, checkpointSystemName, 1, 1, props) - new KafkaCheckpointManager(spec, systemFactory, failOnTopicValidation, config, new NoOpMetricsRegistry, serde) + new KafkaCheckpointManager(spec, systemFactory, failOnTopicValidation, overrideConfig, new NoOpMetricsRegistry, serde) } - private def readCheckpoint(checkpointTopic: String, taskName: TaskName) : Checkpoint = { - val kcm = createKafkaCheckpointManager(checkpointTopic) + private def readCheckpoint(checkpointTopic: String, taskName: TaskName, config: Config = config) : Checkpoint = { + val kcm = createKafkaCheckpointManager(checkpointTopic, overrideConfig = config) kcm.register(taskName) kcm.start val checkpoint = kcm.readLastCheckpoint(taskName) @@ -300,4 +397,10 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { } } + class MockCheckpointSerde() extends CheckpointV1Serde { + override def fromBytes(bytes: Array[Byte]): CheckpointV1 = { + throw new SamzaException("Failed to deserialize") + } + } + } diff --git a/samza-test/src/test/java/org/apache/samza/checkpoint/CheckpointVersionIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/checkpoint/CheckpointVersionIntegrationTest.java new file mode 100644 index 0000000000..d0f1c2f0b6 --- /dev/null +++ b/samza-test/src/test/java/org/apache/samza/checkpoint/CheckpointVersionIntegrationTest.java @@ -0,0 +1,149 @@ +/* + * 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.samza.checkpoint; + +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.JobCoordinatorConfig; +import org.apache.samza.config.KafkaConfig; +import org.apache.samza.config.TaskConfig; +import org.apache.samza.storage.MyStatefulApplication; +import org.apache.samza.test.framework.StreamApplicationIntegrationTestHarness; +import org.apache.samza.util.FileUtil; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class CheckpointVersionIntegrationTest extends StreamApplicationIntegrationTestHarness { + + private final static Logger LOG = LoggerFactory.getLogger(CheckpointVersionIntegrationTest.class); + + private static final String INPUT_TOPIC = "inputTopic"; + private static final String INPUT_SYSTEM = "kafka"; + private static final String STORE_NAME = "store"; + private static final String CHANGELOG_TOPIC = "changelog"; + private static final String LOGGED_STORE_BASE_DIR = new File(System.getProperty("java.io.tmpdir"), "logged-store").getAbsolutePath(); + private static final Map CONFIGS = new HashMap() { { + put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, "org.apache.samza.standalone.PassthroughJobCoordinatorFactory"); + put(JobConfig.PROCESSOR_ID, "0"); + put(TaskConfig.GROUPER_FACTORY, "org.apache.samza.container.grouper.task.GroupByContainerIdsFactory"); + put(TaskConfig.CHECKPOINT_MANAGER_FACTORY, "org.apache.samza.checkpoint.kafka.KafkaCheckpointManagerFactory"); + put(TaskConfig.COMMIT_MS, "-1"); // manual commit only + put(TaskConfig.TRANSACTIONAL_STATE_RESTORE_ENABLED, "true"); + put(TaskConfig.TRANSACTIONAL_STATE_RETAIN_EXISTING_STATE, "true"); + put(KafkaConfig.CHECKPOINT_REPLICATION_FACTOR(), "1"); + put(JobConfig.JOB_LOGGED_STORE_BASE_DIR, LOGGED_STORE_BASE_DIR); + put(TaskConfig.COMMIT_MAX_DELAY_MS, "0"); // Ensure no commits are skipped due to in progress commits + } }; + + @Before + @Override + public void setUp() { + super.setUp(); + // reset static state shared with task between each parameterized iteration + MyStatefulApplication.resetTestState(); + new FileUtil().rm(new File(LOGGED_STORE_BASE_DIR)); // always clear local store on startup + } + + @Test + public void testStopCheckpointV1V2AndRestartCheckpointV2() { + List inputMessagesOnInitialRun = Arrays.asList("1", "2", "3", "2", "97", "-97", ":98", ":99", ":crash_once"); + // double check collectors.flush + List expectedChangelogMessagesOnInitialRun = Arrays.asList("1", "2", "3", "2", "97", null, "98", "99"); + initialRun(inputMessagesOnInitialRun, expectedChangelogMessagesOnInitialRun); + + // first two are reverts for uncommitted messages from last run for keys 98 and 99 + List expectedChangelogMessagesAfterSecondRun = + Arrays.asList(null, null, "98", "99", "4", "5", "5"); + List expectedInitialStoreContentsOnSecondRun = Arrays.asList("1", "2", "3"); + Map configOverrides = new HashMap<>(CONFIGS); + configOverrides.put(TaskConfig.CHECKPOINT_READ_VERSION, "2"); + secondRun(CHANGELOG_TOPIC, + expectedChangelogMessagesAfterSecondRun, expectedInitialStoreContentsOnSecondRun, configOverrides); + } + + private void initialRun(List inputMessages, List expectedChangelogMessages) { + // create input topic and produce the first batch of input messages + createTopic(INPUT_TOPIC, 1); + inputMessages.forEach(m -> produceMessage(INPUT_TOPIC, 0, m, m)); + + // verify that the input messages were produced successfully + if (inputMessages.size() > 0) { + List> inputRecords = + consumeMessages(Collections.singletonList(INPUT_TOPIC), inputMessages.size()); + List readInputMessages = inputRecords.stream().map(ConsumerRecord::value).collect(Collectors.toList()); + Assert.assertEquals(inputMessages, readInputMessages); + } + + // run the application + RunApplicationContext context = runApplication( + new MyStatefulApplication(INPUT_SYSTEM, INPUT_TOPIC, Collections.singletonMap(STORE_NAME, CHANGELOG_TOPIC)), "myApp", CONFIGS); + + // wait for the application to finish + context.getRunner().waitForFinish(); + + // consume and verify the changelog messages + if (expectedChangelogMessages.size() > 0) { + List> changelogRecords = + consumeMessages(Collections.singletonList(CHANGELOG_TOPIC), expectedChangelogMessages.size()); + List changelogMessages = changelogRecords.stream().map(ConsumerRecord::value).collect(Collectors.toList()); + Assert.assertEquals(expectedChangelogMessages, changelogMessages); + } + + LOG.info("Finished initial run"); + } + + private void secondRun(String changelogTopic, List expectedChangelogMessages, + List expectedInitialStoreContents, Map overriddenConfigs) { + // remove previous files so restore is from the checkpointV2 + new FileUtil().rm(new File(LOGGED_STORE_BASE_DIR)); + + // produce the second batch of input messages + + List inputMessages = Arrays.asList("4", "5", "5", ":shutdown"); + inputMessages.forEach(m -> produceMessage(INPUT_TOPIC, 0, m, m)); + + // run the application + RunApplicationContext context = runApplication( + new MyStatefulApplication(INPUT_SYSTEM, INPUT_TOPIC, Collections.singletonMap(STORE_NAME, changelogTopic)), "myApp", overriddenConfigs); + + // wait for the application to finish + context.getRunner().waitForFinish(); + + // verify the store contents during startup (this is after changelog verification to ensure init has completed) + Assert.assertEquals(expectedInitialStoreContents, MyStatefulApplication.getInitialStoreContents().get(STORE_NAME)); + + // consume and verify any additional changelog messages + List> changelogRecords = + consumeMessages(Collections.singletonList(changelogTopic), expectedChangelogMessages.size()); + List changelogMessages = changelogRecords.stream().map(ConsumerRecord::value).collect(Collectors.toList()); + Assert.assertEquals(expectedChangelogMessages, changelogMessages); + } +} diff --git a/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateIntegrationTest.java index 05e6737768..e9b12a1ac1 100644 --- a/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateIntegrationTest.java +++ b/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateIntegrationTest.java @@ -92,6 +92,7 @@ public static Collection data() { put(TaskConfig.TRANSACTIONAL_STATE_RETAIN_EXISTING_STATE, "true"); put(KafkaConfig.CHECKPOINT_REPLICATION_FACTOR(), "1"); put(JobConfig.JOB_LOGGED_STORE_BASE_DIR, LOGGED_STORE_BASE_DIR); + put(TaskConfig.COMMIT_MAX_DELAY_MS, "0"); // Ensure no commits are skipped due to in progress commits } }; private static List actualInitialStoreContents = new ArrayList<>(); diff --git a/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateMultiStoreIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateMultiStoreIntegrationTest.java index 41eb1ab665..d50d6bf72f 100644 --- a/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateMultiStoreIntegrationTest.java +++ b/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateMultiStoreIntegrationTest.java @@ -91,6 +91,7 @@ public static Collection data() { put(TaskConfig.TRANSACTIONAL_STATE_RETAIN_EXISTING_STATE, "true"); put(KafkaConfig.CHECKPOINT_REPLICATION_FACTOR(), "1"); put(JobConfig.JOB_LOGGED_STORE_BASE_DIR, LOGGED_STORE_BASE_DIR); + put(TaskConfig.COMMIT_MAX_DELAY_MS, "0"); // Ensure no commits are skipped due to in progress commits } }; private static List actualInitialStoreContents = new ArrayList<>(); From c117a685dfa020316c6d9e4ab5b19e8129c4afdb Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Fri, 7 May 2021 14:03:16 -0700 Subject: [PATCH 02/20] SAMZA-2591: Async Commit [2/3]: Task Commit api changes and async commit (#1490) Introduce new state backend APIs for blobstore and kafka changelog Change the task commit lifecycle to separate snapshot, upload and cleanup phases Make the TaskInstance commit upload and cleanup phases nonblocking --- gradle/dependency-versions.gradle | 2 +- .../samza/storage/StateBackendFactory.java | 59 ++ .../samza/storage/TaskBackupManager.java | 92 ++ .../samza/storage/TaskRestoreManager.java | 53 ++ .../samza/storage/TaskStorageAdmin.java | 23 +- .../apache/samza/system/SystemFactory.java | 2 +- .../KafkaChangelogStateBackendFactory.java | 215 +++++ .../samza/storage/StorageManagerUtil.java | 57 +- .../storage/TaskStorageCommitManager.java | 365 ++++++++ .../system/inmemory/InMemoryManager.java | 2 +- .../table/retry/AsyncRetriableTable.java | 11 +- .../samza/table/retry/FailsafeAdapter.java | 43 +- .../org/apache/samza/util/FutureUtil.java | 191 ++++ .../samza/container/SamzaContainer.scala | 119 ++- .../apache/samza/container/TaskInstance.scala | 322 +++++-- .../samza/container/TaskInstanceMetrics.scala | 11 +- ...nTransactionalStateTaskBackupManager.scala | 91 ++ ...aTransactionalStateTaskBackupManager.scala | 97 ++ ...TransactionalStateTaskStorageManager.scala | 144 --- ...TransactionalStateTaskStorageManager.scala | 201 ---- .../samza/config/TestStorageConfig.java | 64 ++ ...TestKafkaChangelogStateBackendFactory.java | 80 ++ .../storage/TestTaskStorageCommitManager.java | 878 ++++++++++++++++++ .../org/apache/samza/util/TestFutureUtil.java | 222 +++++ .../samza/container/TestSamzaContainer.scala | 34 - .../samza/container/TestTaskInstance.scala | 710 ++++++++++++-- ...tTransactionalStateTaskStorageManager.java | 534 ----------- .../storage/TestTaskStorageManager.scala | 210 +---- ...stTransactionalStateTaskBackupManager.java | 276 ++++++ .../samza/storage/MyStatefulApplication.java | 157 ++++ .../kv/TransactionalStateIntegrationTest.java | 134 +-- ...ctionalStateMultiStoreIntegrationTest.java | 119 +-- 32 files changed, 4003 insertions(+), 1515 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java create mode 100644 samza-api/src/main/java/org/apache/samza/storage/TaskBackupManager.java create mode 100644 samza-api/src/main/java/org/apache/samza/storage/TaskRestoreManager.java rename samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala => samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java (62%) create mode 100644 samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java create mode 100644 samza-core/src/main/java/org/apache/samza/util/FutureUtil.java create mode 100644 samza-core/src/main/scala/org/apache/samza/storage/KafkaNonTransactionalStateTaskBackupManager.scala create mode 100644 samza-core/src/main/scala/org/apache/samza/storage/KafkaTransactionalStateTaskBackupManager.scala delete mode 100644 samza-core/src/main/scala/org/apache/samza/storage/NonTransactionalStateTaskStorageManager.scala delete mode 100644 samza-core/src/main/scala/org/apache/samza/storage/TransactionalStateTaskStorageManager.scala create mode 100644 samza-core/src/test/java/org/apache/samza/storage/TestKafkaChangelogStateBackendFactory.java create mode 100644 samza-core/src/test/java/org/apache/samza/storage/TestTaskStorageCommitManager.java create mode 100644 samza-core/src/test/java/org/apache/samza/util/TestFutureUtil.java delete mode 100644 samza-core/src/test/scala/org/apache/samza/storage/TestTransactionalStateTaskStorageManager.java rename {samza-core/src/test/scala => samza-kafka/src/test/java}/org/apache/samza/storage/TestTaskStorageManager.scala (83%) create mode 100644 samza-kafka/src/test/java/org/apache/samza/storage/TestTransactionalStateTaskBackupManager.java create mode 100644 samza-test/src/test/java/org/apache/samza/storage/MyStatefulApplication.java diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index e289af04b5..ebd2d38351 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -49,7 +49,7 @@ yarnVersion = "2.7.1" zkClientVersion = "0.11" zookeeperVersion = "3.4.13" - failsafeVersion = "1.1.0" + failsafeVersion = "2.4.0" jlineVersion = "3.8.2" jnaVersion = "4.5.1" couchbaseClientVersion = "2.7.2" diff --git a/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java b/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java new file mode 100644 index 0000000000..9946d2a6ac --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java @@ -0,0 +1,59 @@ +/* + * 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.samza.storage; + +import java.io.File; +import java.util.concurrent.ExecutorService; +import org.apache.samza.config.Config; +import org.apache.samza.context.ContainerContext; +import org.apache.samza.context.JobContext; +import org.apache.samza.job.model.TaskModel; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.util.Clock; + + +/** + * Factory to build the Samza {@link TaskBackupManager}, {@link TaskRestoreManager} and {@link TaskStorageAdmin} + * for a particular state storage backend, which are used to durably backup the Samza task state. + */ + public interface StateBackendFactory { + TaskBackupManager getBackupManager(JobContext jobContext, + ContainerContext containerContext, + TaskModel taskModel, + ExecutorService backupExecutor, + MetricsRegistry taskInstanceMetricsRegistry, + Config config, + Clock clock, + File loggedStoreBaseDir, + File nonLoggedStoreBaseDir); + + TaskRestoreManager getRestoreManager(JobContext jobContext, + ContainerContext containerContext, + TaskModel taskModel, + ExecutorService restoreExecutor, + MetricsRegistry metricsRegistry, + Config config, + Clock clock, + File loggedStoreBaseDir, + File nonLoggedStoreBaseDir, + KafkaChangelogRestoreParams kafkaChangelogRestoreParams); + + TaskStorageAdmin getAdmin(); +} diff --git a/samza-api/src/main/java/org/apache/samza/storage/TaskBackupManager.java b/samza-api/src/main/java/org/apache/samza/storage/TaskBackupManager.java new file mode 100644 index 0000000000..a00d7159dc --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/storage/TaskBackupManager.java @@ -0,0 +1,92 @@ +/* + * 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.samza.storage; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +import javax.annotation.Nullable; +import org.apache.samza.checkpoint.Checkpoint; +import org.apache.samza.checkpoint.CheckpointId; + + +/** + *

+ * TaskBackupManager is the interface that must be implemented for any remote system that Samza persists its state to + * during the task commit operation. + * {@link #snapshot(CheckpointId)} will be evoked synchronous to task processing and get a snapshot of the stores + * state to be persisted for the commit. {@link #upload(CheckpointId, Map)} will then use the snapshotted state + * to persist to the underlying backup system and will be asynchronous to task processing. + *

+ * The interface will be evoked in the following way: + *
    + *
  • Snapshot will be called before Upload.
  • + *
  • persistToFilesystem will be called after Upload is completed
  • + *
  • Cleanup is only called after Upload and persistToFilesystem has successfully completed
  • + *
+ */ +public interface TaskBackupManager { + + /** + * Initializes the TaskBackupManager instance. + * + * @param checkpoint last recorded checkpoint from the CheckpointManager or null if no last checkpoint was found + */ + void init(@Nullable Checkpoint checkpoint); + + /** + * Snapshot is used to capture the current state of the stores in order to persist it to the backup manager in the + * {@link #upload(CheckpointId, Map)} (CheckpointId, Map)} phase. Performs the commit operation that is + * synchronous to processing. Returns the per store name state checkpoint markers to be used in upload. + * + * @param checkpointId {@link CheckpointId} of the current commit + * @return a map of store name to state checkpoint markers for stores managed by this state backend + */ + Map snapshot(CheckpointId checkpointId); + + /** + * Upload is used to persist the state provided by the {@link #snapshot(CheckpointId)} to the + * underlying backup system. Commit operation that is asynchronous to message processing and returns a + * {@link CompletableFuture} containing the successfully uploaded state checkpoint markers . + * + * @param checkpointId {@link CheckpointId} of the current commit + * @param stateCheckpointMarkers the map of storename to state checkpoint markers returned by + * {@link #snapshot(CheckpointId)} + * @return a {@link CompletableFuture} containing a map of store name to state checkpoint markers + * after the upload is complete + */ + CompletableFuture> upload(CheckpointId checkpointId, Map stateCheckpointMarkers); + + /** + * Cleanup any local or remote state for checkpoint information that is older than the provided checkpointId + * This operation is required to be idempotent. + * + * @param checkpointId the {@link CheckpointId} of the last successfully committed checkpoint + * @param stateCheckpointMarkers a map of store name to state checkpoint markers returned by + * {@link #upload(CheckpointId, Map)} (CheckpointId, Map)} upload} + */ + CompletableFuture cleanUp(CheckpointId checkpointId, Map stateCheckpointMarkers); + + /** + * Shutdown hook the backup manager to cleanup any allocated resources + */ + void close(); + +} \ No newline at end of file diff --git a/samza-api/src/main/java/org/apache/samza/storage/TaskRestoreManager.java b/samza-api/src/main/java/org/apache/samza/storage/TaskRestoreManager.java new file mode 100644 index 0000000000..999325e309 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/storage/TaskRestoreManager.java @@ -0,0 +1,53 @@ +/* + * 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.samza.storage; + +import org.apache.samza.checkpoint.Checkpoint; + + +/** + * The helper interface restores task state. + */ +public interface TaskRestoreManager { + + /** + * Initialize state resources such as store directories. + */ + void init(Checkpoint checkpoint); + + /** + * Restore state from checkpoints, state snapshots and changelogs. + * Currently, store restoration happens on a separate thread pool within {@code ContainerStorageManager}. In case of + * interrupt/shutdown signals from {@code SamzaContainer}, {@code ContainerStorageManager} may interrupt the restore + * thread. + * + * Note: Typically, interrupt signals don't bubble up as {@link InterruptedException} unless the restore thread is + * waiting on IO/network. In case of busy looping, implementors are expected to check the interrupt status of the + * thread periodically and shutdown gracefully before throwing {@link InterruptedException} upstream. + * {@code SamzaContainer} will not wait for clean up and the interrupt signal is the best effort by the container + * to notify that its shutting down. + */ + void restore() throws InterruptedException; + + /** + * Closes all initiated ressources include storage engines + */ + void close(); +} diff --git a/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala b/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java similarity index 62% rename from samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala rename to samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java index 50d64185f9..205077b309 100644 --- a/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala +++ b/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java @@ -17,21 +17,14 @@ * under the License. */ -package org.apache.samza.storage +package org.apache.samza.storage; -import org.apache.samza.checkpoint.CheckpointId -import org.apache.samza.system.SystemStreamPartition - -trait TaskStorageManager { - - def getStore(storeName: String): Option[StorageEngine] - - def flush(): Map[SystemStreamPartition, Option[String]] - - def checkpoint(checkpointId: CheckpointId, newestChangelogOffsets: Map[SystemStreamPartition, Option[String]]): Unit - - def removeOldCheckpoints(checkpointId: CheckpointId): Unit +/** + * Creates and validate resources for the StateBackendFactory + */ +public interface TaskStorageAdmin { - def stop(): Unit + void createResources(); -} \ No newline at end of file + void validateResources(); +} diff --git a/samza-api/src/main/java/org/apache/samza/system/SystemFactory.java b/samza-api/src/main/java/org/apache/samza/system/SystemFactory.java index 08c1b49534..2841bb1446 100644 --- a/samza-api/src/main/java/org/apache/samza/system/SystemFactory.java +++ b/samza-api/src/main/java/org/apache/samza/system/SystemFactory.java @@ -73,7 +73,7 @@ default SystemProducer getProducer(String systemName, Config config, MetricsRegi * * @param systemName The name of the system to create admin for. * @param config The config to create admin with. - * @param adminLabel a string to provide info the admin instance. + * @param adminLabel a string to provide info for the admin instance. * @return A SystemAdmin */ default SystemAdmin getAdmin(String systemName, Config config, String adminLabel) { diff --git a/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java b/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java new file mode 100644 index 0000000000..e3230f6306 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java @@ -0,0 +1,215 @@ +/* + * 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.samza.storage; + +import com.google.common.annotations.VisibleForTesting; +import java.io.File; +import java.time.Duration; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.stream.Collectors; +import org.apache.commons.collections4.MapUtils; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.config.StorageConfig; +import org.apache.samza.config.TaskConfig; +import org.apache.samza.context.ContainerContext; +import org.apache.samza.context.JobContext; +import org.apache.samza.job.model.ContainerModel; +import org.apache.samza.job.model.TaskMode; +import org.apache.samza.job.model.TaskModel; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.system.SSPMetadataCache; +import org.apache.samza.system.StreamMetadataCache; +import org.apache.samza.system.SystemAdmins; +import org.apache.samza.system.SystemStream; +import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.util.Clock; + + +/** + * Class used the provide the {@link TaskRestoreManager} and the {@link TaskBackupManager} for the Kafka changelog + * state backend. + */ +public class KafkaChangelogStateBackendFactory implements StateBackendFactory { + private StreamMetadataCache streamCache; + /* + * This keeps track of the changelog SSPs that are associated with the whole container. This is used so that we can + * prefetch the metadata about the all of the changelog SSPs associated with the container whenever we need the + * metadata about some of the changelog SSPs. + * An example use case is when Samza writes offset files for stores ({@link TaskStorageManager}). Each task is + * responsible for its own offset file, but if we can do prefetching, then most tasks will already have cached + * metadata by the time they need the offset metadata. + * Note: By using all changelog streams to build the sspsToPrefetch, any fetches done for persisted stores will + * include the ssps for non-persisted stores, so this is slightly suboptimal. However, this does not increase the + * actual number of calls to the {@link SystemAdmin}, and we can decouple this logic from the per-task objects (e.g. + * {@link TaskStorageManager}). + */ + private SSPMetadataCache sspCache; + + @Override + public TaskBackupManager getBackupManager(JobContext jobContext, + ContainerContext containerContext, + TaskModel taskModel, + ExecutorService backupExecutor, + MetricsRegistry metricsRegistry, + Config config, + Clock clock, + File loggedStoreBaseDir, + File nonLoggedStoreBaseDir) { + SystemAdmins systemAdmins = new SystemAdmins(config); + StorageConfig storageConfig = new StorageConfig(config); + Map storeChangelogs = storageConfig.getStoreChangelogs(); + + if (new TaskConfig(config).getTransactionalStateCheckpointEnabled()) { + return new KafkaTransactionalStateTaskBackupManager(taskModel.getTaskName(), storeChangelogs, + systemAdmins, taskModel.getChangelogPartition()); + } else { + return new KafkaNonTransactionalStateTaskBackupManager(taskModel.getTaskName(), storeChangelogs, + systemAdmins, taskModel.getChangelogPartition()); + } + } + + @Override + public TaskRestoreManager getRestoreManager(JobContext jobContext, + ContainerContext containerContext, + TaskModel taskModel, + ExecutorService restoreExecutor, + MetricsRegistry metricsRegistry, + Config config, + Clock clock, + File loggedStoreBaseDir, + File nonLoggedStoreBaseDir, + KafkaChangelogRestoreParams kafkaChangelogRestoreParams) { + Map storeChangelogs = new StorageConfig(config).getStoreChangelogs(); + Set changelogSSPs = getChangelogSSPForContainer(storeChangelogs, containerContext); + // filter out standby store-ssp pairs + Map filteredStoreChangelogs = + filterStandbySystemStreams(storeChangelogs, containerContext.getContainerModel()); + SystemAdmins systemAdmins = new SystemAdmins(kafkaChangelogRestoreParams.getSystemAdmins()); + + if (new TaskConfig(config).getTransactionalStateRestoreEnabled()) { + return new TransactionalStateTaskRestoreManager( + kafkaChangelogRestoreParams.getStoreNames(), + jobContext, + containerContext, + taskModel, + filteredStoreChangelogs, + kafkaChangelogRestoreParams.getInMemoryStores(), + kafkaChangelogRestoreParams.getStorageEngineFactories(), + kafkaChangelogRestoreParams.getSerdes(), + systemAdmins, + kafkaChangelogRestoreParams.getStoreConsumers(), + metricsRegistry, + kafkaChangelogRestoreParams.getCollector(), + getSspCache(systemAdmins, clock, changelogSSPs), + loggedStoreBaseDir, + nonLoggedStoreBaseDir, + config, + clock + ); + } else { + return new NonTransactionalStateTaskRestoreManager( + kafkaChangelogRestoreParams.getStoreNames(), + jobContext, + containerContext, + taskModel, + filteredStoreChangelogs, + kafkaChangelogRestoreParams.getInMemoryStores(), + kafkaChangelogRestoreParams.getStorageEngineFactories(), + kafkaChangelogRestoreParams.getSerdes(), + systemAdmins, + getStreamCache(systemAdmins, clock), + kafkaChangelogRestoreParams.getStoreConsumers(), + metricsRegistry, + kafkaChangelogRestoreParams.getCollector(), + jobContext.getJobModel().getMaxChangeLogStreamPartitions(), + loggedStoreBaseDir, + nonLoggedStoreBaseDir, + config, + clock + ); + } + } + + @Override + public TaskStorageAdmin getAdmin() { + throw new SamzaException("getAdmin() method not supported for KafkaStateBackendFactory"); + } + + @VisibleForTesting + Set getChangelogSSPForContainer(Map storeChangelogs, + ContainerContext containerContext) { + return storeChangelogs.values().stream() + .flatMap(ss -> containerContext.getContainerModel().getTasks().values().stream() + .map(tm -> new SystemStreamPartition(ss, tm.getChangelogPartition()))) + .collect(Collectors.toSet()); + } + + /** + * Shared cache across all KafkaRestoreManagers for the Kafka topic + * + * @param admins system admins used the fetch the stream metadata + * @param clock for cache invalidation + * @return StreamMetadataCache containing the stream metadata + */ + @VisibleForTesting + StreamMetadataCache getStreamCache(SystemAdmins admins, Clock clock) { + if (streamCache == null) { + streamCache = new StreamMetadataCache(admins, 5000, clock); + } + return streamCache; + } + + /** + * Shared cache across KafkaRestoreManagers for the Kafka partition + * + * @param admins system admins used the fetch the stream metadata + * @param clock for cache invalidation + * @param ssps SSPs to prefetch + * @return SSPMetadataCache containing the partition metadata + */ + @VisibleForTesting + SSPMetadataCache getSspCache(SystemAdmins admins, Clock clock, Set ssps) { + if (sspCache == null) { + sspCache = new SSPMetadataCache(admins, Duration.ofSeconds(5), clock, ssps); + } + return sspCache; + } + + @VisibleForTesting + Map filterStandbySystemStreams(Map changelogSystemStreams, + ContainerModel containerModel) { + Map changelogSSPToStore = new HashMap<>(); + changelogSystemStreams.forEach((storeName, systemStream) -> + containerModel.getTasks().forEach((taskName, taskModel) -> { + if (TaskMode.Standby.equals(taskModel.getTaskMode())) { + changelogSSPToStore.put(new SystemStreamPartition(systemStream, taskModel.getChangelogPartition()), + storeName); + } + }) + ); + // changelogSystemStreams correspond only to active tasks (since those of standby-tasks moved to sideInputs above) + return MapUtils.invertMap(changelogSSPToStore).entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, x -> x.getValue().getSystemStream())); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java b/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java index cfd24d94fd..91c0c7cf91 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java +++ b/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java @@ -38,12 +38,15 @@ import java.util.Set; import java.util.stream.Collectors; import org.apache.samza.SamzaException; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.checkpoint.CheckpointV2; import org.apache.samza.clustermanager.StandbyTaskUtil; import org.apache.samza.config.Config; import org.apache.samza.config.StorageConfig; import org.apache.samza.container.TaskName; import org.apache.samza.job.model.TaskMode; import org.apache.samza.job.model.TaskModel; +import org.apache.samza.serializers.CheckpointV2Serde; import org.apache.samza.serializers.model.SamzaObjectMapper; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemStream; @@ -56,14 +59,27 @@ public class StorageManagerUtil { private static final Logger LOG = LoggerFactory.getLogger(StorageManagerUtil.class); + public static final String CHECKPOINT_FILE_NAME = "CHECKPOINT-V2"; public static final String OFFSET_FILE_NAME_NEW = "OFFSET-v2"; public static final String OFFSET_FILE_NAME_LEGACY = "OFFSET"; public static final String SIDE_INPUT_OFFSET_FILE_NAME_LEGACY = "SIDE-INPUT-OFFSETS"; private static final ObjectMapper OBJECT_MAPPER = SamzaObjectMapper.getObjectMapper(); private static final TypeReference> OFFSETS_TYPE_REFERENCE = new TypeReference>() { }; - private static final ObjectWriter OBJECT_WRITER = OBJECT_MAPPER.writerWithType(OFFSETS_TYPE_REFERENCE); + private static final ObjectWriter SSP_OFFSET_OBJECT_WRITER = OBJECT_MAPPER.writerFor(OFFSETS_TYPE_REFERENCE); private static final String SST_FILE_SUFFIX = ".sst"; + private static final CheckpointV2Serde CHECKPOINT_V2_SERDE = new CheckpointV2Serde(); + + /** + * Returns the path for a storage engine to create its checkpoint based on the current checkpoint id. + * + * @param taskStoreDir directory of the store as returned by {@link #getTaskStoreDir} + * @param checkpointId current checkpoint id + * @return String denoting the file path of the store with the given checkpoint id + */ + public static String getCheckpointDirPath(File taskStoreDir, CheckpointId checkpointId) { + return taskStoreDir.getPath() + "-" + checkpointId.serialize(); + } /** * Fetch the starting offset for the input {@link SystemStreamPartition} @@ -109,6 +125,7 @@ public String getStartingOffset( * @param isSideInput true if store is a side-input store, false if it is a regular store * @return true if the store is stale, false otherwise */ + // TODO BLOCKER dchen do these methods need to be updated to also read the new checkpoint file? public boolean isStaleStore(File storeDir, long storeDeleteRetentionInMs, long currentTimeMs, boolean isSideInput) { long offsetFileLastModifiedTime; boolean isStaleStore = false; @@ -118,7 +135,7 @@ public boolean isStaleStore(File storeDir, long storeDeleteRetentionInMs, long c // We check if the new offset-file exists, if so we use its last-modified time, if it doesn't we use the legacy file // depending on if it is a side-input or not, - // if neither exists, we use 0L (the defauilt return value of lastModified() when file does not exist + // if neither exists, we use 0L (the default return value of lastModified() when file does not exist File offsetFileRefNew = new File(storeDir, OFFSET_FILE_NAME_NEW); File offsetFileRefLegacy = new File(storeDir, OFFSET_FILE_NAME_LEGACY); File sideInputOffsetFileRefLegacy = new File(storeDir, SIDE_INPUT_OFFSET_FILE_NAME_LEGACY); @@ -179,6 +196,7 @@ && isOffsetFileValid(loggedStoreDir, Collections.singleton(changelogSSP), false) * @param isSideInput true if store is a side-input store, false if it is a regular store * @return true if the offset file is valid. false otherwise. */ + // TODO BLOCKER dchen do these methods need to be updated to also read the new checkpoint file? public boolean isOffsetFileValid(File storeDir, Set storeSSPs, boolean isSideInput) { boolean hasValidOffsetFile = false; if (storeDir.exists()) { @@ -210,14 +228,14 @@ public void writeOffsetFile(File storeDir, Map of // First, we write the new-format offset file File offsetFile = new File(storeDir, OFFSET_FILE_NAME_NEW); - String fileContents = OBJECT_WRITER.writeValueAsString(offsets); + String fileContents = SSP_OFFSET_OBJECT_WRITER.writeValueAsString(offsets); FileUtil fileUtil = new FileUtil(); fileUtil.writeWithChecksum(offsetFile, fileContents); // Now we write the old format offset file, which are different for store-offset and side-inputs if (isSideInput) { offsetFile = new File(storeDir, SIDE_INPUT_OFFSET_FILE_NAME_LEGACY); - fileContents = OBJECT_WRITER.writeValueAsString(offsets); + fileContents = SSP_OFFSET_OBJECT_WRITER.writeValueAsString(offsets); fileUtil.writeWithChecksum(offsetFile, fileContents); } else { offsetFile = new File(storeDir, OFFSET_FILE_NAME_LEGACY); @@ -225,6 +243,19 @@ public void writeOffsetFile(File storeDir, Map of } } + /** + * Writes the checkpoint to the store checkpoint directory based on the checkpointId. + * + * @param storeDir store or store checkpoint directory to write the checkpoint to + * @param checkpoint checkpoint v2 containing the checkpoint Id + */ + public void writeCheckpointV2File(File storeDir, CheckpointV2 checkpoint) { + File offsetFile = new File(storeDir, CHECKPOINT_FILE_NAME); + byte[] fileContents = CHECKPOINT_V2_SERDE.toBytes(checkpoint); + FileUtil fileUtil = new FileUtil(); + fileUtil.writeWithChecksum(offsetFile, new String(fileContents)); + } + /** * Delete the offset file for this store, if one exists. * @param storeDir the directory of the store @@ -283,6 +314,24 @@ public Map readOffsetFile(File storagePartitionDi } } + /** + * Read and return the {@link CheckpointV2} from the directory's {@link #CHECKPOINT_FILE_NAME} file. + * If the file does not exist, returns null. + * // TODO HIGH dchen add tests at all call sites for handling null value. + * + * @param storagePartitionDir store directory to read the checkpoint file from + * @return the {@link CheckpointV2} object retrieved from the checkpoint file if found, otherwise return null + */ + public CheckpointV2 readCheckpointV2File(File storagePartitionDir) { + File checkpointFile = new File(storagePartitionDir, CHECKPOINT_FILE_NAME); + if (checkpointFile.exists()) { + String serializedCheckpointV2 = new FileUtil().readWithChecksum(checkpointFile); + return new CheckpointV2Serde().fromBytes(serializedCheckpointV2.getBytes()); + } else { + return null; + } + } + /** * Read and return the contents of the offset file. * diff --git a/samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java b/samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java new file mode 100644 index 0000000000..cc80a48f42 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java @@ -0,0 +1,365 @@ +/* + * 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.samza.storage; + +import com.google.common.annotations.VisibleForTesting; +import java.io.File; +import java.io.FileFilter; +import java.io.IOException; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.filefilter.WildcardFileFilter; +import org.apache.samza.Partition; +import org.apache.samza.SamzaException; +import org.apache.samza.checkpoint.Checkpoint; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.checkpoint.CheckpointManager; +import org.apache.samza.checkpoint.CheckpointV1; +import org.apache.samza.checkpoint.CheckpointV2; +import org.apache.samza.checkpoint.kafka.KafkaChangelogSSPOffset; +import org.apache.samza.config.Config; +import org.apache.samza.container.TaskInstanceMetrics; +import org.apache.samza.container.TaskName; +import org.apache.samza.job.model.TaskMode; +import org.apache.samza.system.SystemStream; +import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.util.FutureUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Handles the commit of the state stores of the task. + */ +public class TaskStorageCommitManager { + + private static final Logger LOG = LoggerFactory.getLogger(TaskStorageCommitManager.class); + + private final TaskName taskName; + private final CheckpointManager checkpointManager; + private final ContainerStorageManager containerStorageManager; + private final Map stateBackendToBackupManager; + private final Partition taskChangelogPartition; + private final StorageManagerUtil storageManagerUtil; + private final ExecutorService backupExecutor; + private final File durableStoreBaseDir; + private final Map storeChangelogs; + private final TaskInstanceMetrics metrics; + + // Available after init(), since stores are created by ContainerStorageManager#start() + private Map storageEngines; + + public TaskStorageCommitManager(TaskName taskName, Map stateBackendToBackupManager, + ContainerStorageManager containerStorageManager, Map storeChangelogs, Partition changelogPartition, + CheckpointManager checkpointManager, Config config, ExecutorService backupExecutor, + StorageManagerUtil storageManagerUtil, File durableStoreBaseDir, TaskInstanceMetrics metrics) { + this.taskName = taskName; + this.containerStorageManager = containerStorageManager; + this.stateBackendToBackupManager = stateBackendToBackupManager; + this.taskChangelogPartition = changelogPartition; + this.checkpointManager = checkpointManager; + this.backupExecutor = backupExecutor; + this.durableStoreBaseDir = durableStoreBaseDir; + this.storeChangelogs = storeChangelogs; + this.storageManagerUtil = storageManagerUtil; + this.metrics = metrics; + } + + public void init() { + // Assuming that container storage manager has already started and created to stores + storageEngines = containerStorageManager.getAllStores(taskName); + if (checkpointManager != null) { + Checkpoint checkpoint = checkpointManager.readLastCheckpoint(taskName); + LOG.debug("Last checkpoint on start for task: {} is: {}", taskName, checkpoint); + stateBackendToBackupManager.values() + .forEach(storageBackupManager -> storageBackupManager.init(checkpoint)); + } else { + stateBackendToBackupManager.values() + .forEach(storageBackupManager -> storageBackupManager.init(null)); + } + } + + /** + * Synchronously captures the current state of the stores in order to persist it to the backup manager + * in the async {@link #upload(CheckpointId, Map)} phase. Returns a map of state backend factory name to + * a map of store name to state checkpoint markers for all configured state backends and stores. + * + * @param checkpointId {@link CheckpointId} of the current commit + * @return a map of state backend factory name to a map of store name to state checkpoint markers + */ + public Map> snapshot(CheckpointId checkpointId) { + // Flush all stores + storageEngines.values().forEach(StorageEngine::flush); + LOG.debug("Flushed all storage engines for taskName: {}, checkpoint id: {}", + taskName, checkpointId); + + long checkpointStartNs = System.nanoTime(); + // Checkpoint all persisted and durable stores + storageEngines.forEach((storeName, storageEngine) -> { + if (storageEngine.getStoreProperties().isPersistedToDisk() && + storageEngine.getStoreProperties().isDurableStore()) { + storageEngine.checkpoint(checkpointId); + } + }); + long checkpointNs = System.nanoTime() - checkpointStartNs; + metrics.storeCheckpointNs().update(checkpointNs); + LOG.debug("Checkpointed all storage engines for taskName: {}, checkpoint id: {} in {} ns", + taskName, checkpointId, checkpointNs); + + // state backend factory -> store Name -> state checkpoint marker + Map> stateBackendToStoreSCMs = new HashMap<>(); + + // for each configured state backend factory, backup the state for all stores in this task. + stateBackendToBackupManager.forEach((stateBackendFactoryName, backupManager) -> { + Map snapshotSCMs = backupManager.snapshot(checkpointId); + LOG.debug("Created snapshot for taskName: {}, checkpoint id: {}, state backend: {}. Snapshot SCMs: {}", + taskName, checkpointId, stateBackendFactoryName, snapshotSCMs); + stateBackendToStoreSCMs.put(stateBackendFactoryName, snapshotSCMs); + }); + + return stateBackendToStoreSCMs; + } + + /** + * Asynchronously backs up the local state to the remote storage and returns a future containing the committed + * map of state backend factory name to the map of store name to state checkpoint marker. + * + * @param checkpointId the {@link CheckpointId} associated with this commit + * @return a future containing the Map of FactoryName to (Map of StoreName to StateCheckpointMarker). + */ + public CompletableFuture>> upload( + CheckpointId checkpointId, Map> snapshotSCMs) { + // state backend factory -> store Name -> state checkpoint marker + Map>> stateBackendToStoreSCMs = new HashMap<>(); + + // for each configured state backend factory, backup the state for all stores in this task. + stateBackendToBackupManager.forEach((stateBackendFactoryName, backupManager) -> { + try { + Map factorySnapshotSCMs = + snapshotSCMs.getOrDefault(stateBackendFactoryName, Collections.emptyMap()); + LOG.debug("Starting upload for taskName: {}, checkpoint id: {}, state backend snapshot SCM: {}", + taskName, checkpointId, factorySnapshotSCMs); + + CompletableFuture> uploadFuture = + backupManager.upload(checkpointId, factorySnapshotSCMs); + uploadFuture.thenAccept(uploadSCMs -> + LOG.debug("Finished upload for taskName: {}, checkpoint id: {}, state backend: {}. Upload SCMs: {}", + taskName, checkpointId, stateBackendFactoryName, uploadSCMs)); + + stateBackendToStoreSCMs.put(stateBackendFactoryName, uploadFuture); + } catch (Exception e) { + throw new SamzaException( + String.format("Error backing up local state for taskName: %s, checkpoint id: %s, state backend: %s", + taskName, checkpointId, stateBackendFactoryName), e); + } + }); + + return FutureUtil.toFutureOfMap(stateBackendToStoreSCMs); + } + + /** + * Writes the {@link Checkpoint} information returned by {@link #upload(CheckpointId, Map)} + * in each store directory and store checkpoint directory. Written content depends on the type of {@code checkpoint}. + * For {@link CheckpointV2}, writes the entire task {@link CheckpointV2}. + * For {@link CheckpointV1}, only writes the changelog ssp offsets in the OFFSET* files. + * + * Note: The assumption is that this method will be invoked once for each {@link Checkpoint} version that the + * task needs to write as determined by {@link org.apache.samza.config.TaskConfig#getCheckpointWriteVersions()}. + * This is required for upgrade and rollback compatibility. + * + * @param checkpoint the latest checkpoint to be persisted to local file system + */ + public void writeCheckpointToStoreDirectories(Checkpoint checkpoint) { + if (checkpoint instanceof CheckpointV1) { + LOG.debug("Writing CheckpointV1 to store and checkpoint directories for taskName: {} with checkpoint: {}", + taskName, checkpoint); + // Write CheckpointV1 changelog offsets to store and checkpoint directories + writeChangelogOffsetFiles(checkpoint.getOffsets()); + } else if (checkpoint instanceof CheckpointV2) { + LOG.debug("Writing CheckpointV2 to store and checkpoint directories for taskName: {} with checkpoint: {}", + taskName, checkpoint); + storageEngines.forEach((storeName, storageEngine) -> { + // Only write the checkpoint file if the store is durable and persisted to disk + if (storageEngine.getStoreProperties().isDurableStore() && + storageEngine.getStoreProperties().isPersistedToDisk()) { + CheckpointV2 checkpointV2 = (CheckpointV2) checkpoint; + + try { + File storeDir = storageManagerUtil.getTaskStoreDir(durableStoreBaseDir, storeName, taskName, TaskMode.Active); + storageManagerUtil.writeCheckpointV2File(storeDir, checkpointV2); + + CheckpointId checkpointId = checkpointV2.getCheckpointId(); + File checkpointDir = Paths.get(StorageManagerUtil.getCheckpointDirPath(storeDir, checkpointId)).toFile(); + storageManagerUtil.writeCheckpointV2File(checkpointDir, checkpointV2); + } catch (Exception e) { + throw new SamzaException( + String.format("Write checkpoint file failed for task: %s, storeName: %s, checkpointId: %s", + taskName, storeName, ((CheckpointV2) checkpoint).getCheckpointId()), e); + } + } + }); + } else { + throw new SamzaException("Unsupported checkpoint version: " + checkpoint.getVersion()); + } + } + + /** + * Performs any post-commit and cleanup actions after the {@link Checkpoint} is successfully written to the + * checkpoint topic. Invokes {@link TaskBackupManager#cleanUp(CheckpointId, Map)} on each of the configured task + * backup managers. Deletes all local store checkpoint directories older than the {@code latestCheckpointId}. + * + * @param latestCheckpointId CheckpointId of the most recent successful commit + * @param stateCheckpointMarkers map of map(stateBackendFactoryName to map(storeName to state checkpoint markers) from + * the latest commit + */ + public CompletableFuture cleanUp(CheckpointId latestCheckpointId, + Map> stateCheckpointMarkers) { + List> cleanUpFutures = new ArrayList<>(); + + // Call cleanup on each backup manager + stateCheckpointMarkers.forEach((factoryName, storeSCMs) -> { + if (stateBackendToBackupManager.containsKey(factoryName)) { + LOG.debug("Cleaning up commit for factory: {} for task: {}", factoryName, taskName); + TaskBackupManager backupManager = stateBackendToBackupManager.get(factoryName); + cleanUpFutures.add(backupManager.cleanUp(latestCheckpointId, storeSCMs)); + } else { + // This may happen during migration from one state backend to another, where the latest commit contains + // a state backend that is no longer supported for the current commit manager + LOG.warn("Ignored cleanup for scm: {} due to unknown factory: {} ", storeSCMs, factoryName); + } + }); + + return FutureUtil.allOf(cleanUpFutures) + .thenAcceptAsync(aVoid -> deleteOldCheckpointDirs(latestCheckpointId), backupExecutor); + } + + private void deleteOldCheckpointDirs(CheckpointId latestCheckpointId) { + // Delete directories for checkpoints older than latestCheckpointId + if (latestCheckpointId != null) { + LOG.debug("Deleting checkpoints older than checkpoint id: {}", latestCheckpointId); + File[] files = durableStoreBaseDir.listFiles(); + if (files != null) { + for (File storeDir : files) { + String storeName = storeDir.getName(); + String taskStoreName = storageManagerUtil + .getTaskStoreDir(durableStoreBaseDir, storeName, taskName, TaskMode.Active).getName(); + FileFilter fileFilter = new WildcardFileFilter(taskStoreName + "-*"); + File[] checkpointDirs = storeDir.listFiles(fileFilter); + if (checkpointDirs != null) { + for (File checkpointDir : checkpointDirs) { + if (!checkpointDir.getName().contains(latestCheckpointId.serialize())) { + try { + FileUtils.deleteDirectory(checkpointDir); + } catch (IOException e) { + throw new SamzaException( + String.format("Unable to delete checkpoint directory: %s", checkpointDir.getName()), e); + } + } + } + } + } + } + } + } + + /** + * Close all the state backup managers + */ + public void close() { + LOG.debug("Stopping backup managers for task {}.", taskName); + stateBackendToBackupManager.values().forEach(storageBackupManager -> { + if (storageBackupManager != null) { + storageBackupManager.close(); + } + }); + } + + /** + * Writes the newest changelog ssp offset for each logged and persistent store to the OFFSET file in the current + * store directory (for allowing rollbacks). If the Kafka transactional backup manager is enabled, also writes to + * the store checkpoint directory. + * + * These files are used during container startup to ensure transactional state, and to determine whether the + * there is any new information in the changelog that is not reflected in the on-disk copy of the store. + * If there is any delta, it is replayed from the changelog. E.g. this can happen if the job was run on this host, + * then another host, and then back to this host. + */ + @VisibleForTesting + void writeChangelogOffsetFiles(Map checkpointOffsets) { + if (storageEngines == null) { + throw new SamzaException(String.format( + "Storage engines are not initialized and writeChangelogOffsetFiles not be written for task %s", taskName)); + } + storeChangelogs.forEach((storeName, systemStream) -> { + SystemStreamPartition changelogSSP = new SystemStreamPartition( + systemStream.getSystem(), systemStream.getStream(), taskChangelogPartition); + + // Only write if the store is durable and persisted to disk + if (checkpointOffsets.containsKey(changelogSSP) && + storageEngines.containsKey(storeName) && + storageEngines.get(storeName).getStoreProperties().isDurableStore() && + storageEngines.get(storeName).getStoreProperties().isPersistedToDisk()) { + LOG.debug("Writing changelog offset for taskName {} store {} changelog {}.", taskName, storeName, systemStream); + File currentStoreDir = storageManagerUtil.getTaskStoreDir(durableStoreBaseDir, storeName, taskName, TaskMode.Active); + try { + KafkaChangelogSSPOffset kafkaChangelogSSPOffset = KafkaChangelogSSPOffset + .fromString(checkpointOffsets.get(changelogSSP)); + // Write offsets to file system if it is non-null + String newestOffset = kafkaChangelogSSPOffset.getChangelogOffset(); + if (newestOffset != null) { + // Write changelog SSP offset to the OFFSET files in the task store directory + writeChangelogOffsetFile(storeName, changelogSSP, newestOffset, currentStoreDir); + + // Write changelog SSP offset to the OFFSET files in the store checkpoint directory + File checkpointDir = Paths.get(StorageManagerUtil.getCheckpointDirPath( + currentStoreDir, kafkaChangelogSSPOffset.getCheckpointId())).toFile(); + writeChangelogOffsetFile(storeName, changelogSSP, newestOffset, checkpointDir); + } else { + // If newestOffset is null, then it means the changelog ssp is (or has become) empty. This could be + // either because the changelog topic was newly added, repartitioned, or manually deleted and recreated. + // No need to persist the offset file. + LOG.debug("Deleting OFFSET file for taskName {} store {} changelog ssp {} since the newestOffset is null.", + taskName, storeName, changelogSSP); + storageManagerUtil.deleteOffsetFile(currentStoreDir); + } + } catch (IOException e) { + throw new SamzaException( + String.format("Error storing offset for taskName %s store %s changelog %s.", taskName, storeName, + systemStream), e); + } + } + }); + LOG.debug("Done writing OFFSET files for logged persistent key value stores for task {}", taskName); + } + + @VisibleForTesting + void writeChangelogOffsetFile(String storeName, SystemStreamPartition ssp, String newestOffset, + File writeDirectory) throws IOException { + LOG.debug("Storing newest offset {} for taskName {} store {} changelog ssp {} in OFFSET file at path: {}.", + newestOffset, taskName, storeName, ssp, writeDirectory); + storageManagerUtil.writeOffsetFile(writeDirectory, Collections.singletonMap(ssp, newestOffset), false); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemoryManager.java b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemoryManager.java index 13ebf6edb9..bc4b227620 100644 --- a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemoryManager.java +++ b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemoryManager.java @@ -54,7 +54,7 @@ public InMemoryManager() { } private List newSynchronizedLinkedList() { - return Collections.synchronizedList(new LinkedList()); + return Collections.synchronizedList(new LinkedList()); } /** diff --git a/samza-core/src/main/java/org/apache/samza/table/retry/AsyncRetriableTable.java b/samza-core/src/main/java/org/apache/samza/table/retry/AsyncRetriableTable.java index 589fb144a0..938a1d3d50 100644 --- a/samza-core/src/main/java/org/apache/samza/table/retry/AsyncRetriableTable.java +++ b/samza-core/src/main/java/org/apache/samza/table/retry/AsyncRetriableTable.java @@ -20,15 +20,13 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import net.jodah.failsafe.RetryPolicy; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; - import java.util.function.Predicate; -import net.jodah.failsafe.RetryPolicy; - import org.apache.samza.context.Context; import org.apache.samza.storage.kv.Entry; import org.apache.samza.table.AsyncReadWriteTable; @@ -36,9 +34,8 @@ import org.apache.samza.table.remote.TableWriteFunction; import org.apache.samza.table.utils.TableMetricsUtil; -import static org.apache.samza.table.retry.FailsafeAdapter.failsafe; - import static org.apache.samza.table.BaseReadWriteTable.Func1; +import static org.apache.samza.table.retry.FailsafeAdapter.failsafe; /** @@ -156,13 +153,13 @@ public void close() { private CompletableFuture doRead(Func1 func) { return readRetryPolicy != null - ? failsafe(readRetryPolicy, readRetryMetrics, retryExecutor).future(() -> func.apply()) + ? failsafe(readRetryPolicy, readRetryMetrics, retryExecutor).getStageAsync(() -> func.apply()) : func.apply(); } private CompletableFuture doWrite(Func1 func) { return writeRetryPolicy != null - ? failsafe(writeRetryPolicy, writeRetryMetrics, retryExecutor).future(() -> func.apply()) + ? failsafe(writeRetryPolicy, writeRetryMetrics, retryExecutor).getStageAsync(() -> func.apply()) : func.apply(); } } diff --git a/samza-core/src/main/java/org/apache/samza/table/retry/FailsafeAdapter.java b/samza-core/src/main/java/org/apache/samza/table/retry/FailsafeAdapter.java index 650d03af01..a9466b0e9a 100644 --- a/samza-core/src/main/java/org/apache/samza/table/retry/FailsafeAdapter.java +++ b/samza-core/src/main/java/org/apache/samza/table/retry/FailsafeAdapter.java @@ -19,15 +19,14 @@ package org.apache.samza.table.retry; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -import org.apache.samza.SamzaException; - -import net.jodah.failsafe.AsyncFailsafe; import net.jodah.failsafe.Failsafe; +import net.jodah.failsafe.FailsafeExecutor; import net.jodah.failsafe.RetryPolicy; +import java.time.temporal.ChronoUnit; +import java.util.concurrent.ScheduledExecutorService; +import org.apache.samza.SamzaException; + /** * Helper class adapting the generic {@link TableRetryPolicy} to a failsafe {@link RetryPolicy} and @@ -39,23 +38,24 @@ class FailsafeAdapter { * @return this policy instance */ static RetryPolicy valueOf(TableRetryPolicy policy) { - RetryPolicy failSafePolicy = new RetryPolicy(); + // max retries default changed to 2 in v2.0. switching back to infinite retries by default for back compat. + RetryPolicy failSafePolicy = new RetryPolicy().withMaxRetries(-1); switch (policy.getBackoffType()) { case NONE: break; case FIXED: - failSafePolicy.withDelay(policy.getSleepTime().toMillis(), TimeUnit.MILLISECONDS); + failSafePolicy.withDelay(policy.getSleepTime()); break; case RANDOM: - failSafePolicy.withDelay(policy.getRandomMin().toMillis(), policy.getRandomMax().toMillis(), TimeUnit.MILLISECONDS); + failSafePolicy.withDelay(policy.getRandomMin().toMillis(), policy.getRandomMax().toMillis(), ChronoUnit.MILLIS); break; case EXPONENTIAL: - failSafePolicy.withBackoff(policy.getSleepTime().toMillis(), policy.getExponentialMaxSleep().toMillis(), TimeUnit.MILLISECONDS, - policy.getExponentialFactor()); + failSafePolicy.withBackoff(policy.getSleepTime().toMillis(), policy.getExponentialMaxSleep().toMillis(), + ChronoUnit.MILLIS, policy.getExponentialFactor()); break; default: @@ -63,17 +63,16 @@ static RetryPolicy valueOf(TableRetryPolicy policy) { } if (policy.getMaxDuration() != null) { - failSafePolicy.withMaxDuration(policy.getMaxDuration().toMillis(), TimeUnit.MILLISECONDS); + failSafePolicy.withMaxDuration(policy.getMaxDuration()); } if (policy.getMaxAttempts() != null) { failSafePolicy.withMaxRetries(policy.getMaxAttempts()); } if (policy.getJitter() != null && policy.getBackoffType() != TableRetryPolicy.BackoffType.RANDOM) { - failSafePolicy.withJitter(policy.getJitter().toMillis(), TimeUnit.MILLISECONDS); + failSafePolicy.withJitter(policy.getJitter()); } - failSafePolicy.retryOn(e -> policy.getRetryPredicate().test(e)); - + failSafePolicy.abortOn(policy.getRetryPredicate().negate()); return failSafePolicy; } @@ -82,22 +81,24 @@ static RetryPolicy valueOf(TableRetryPolicy policy) { * @param retryPolicy retry policy * @param metrics retry metrics * @param retryExec executor service for scheduling async retries - * @return {@link net.jodah.failsafe.AsyncFailsafe} instance + * @return {@link net.jodah.failsafe.FailsafeExecutor} instance */ - static AsyncFailsafe failsafe(RetryPolicy retryPolicy, RetryMetrics metrics, ScheduledExecutorService retryExec) { + static FailsafeExecutor failsafe(RetryPolicy retryPolicy, RetryMetrics metrics, ScheduledExecutorService retryExec) { long startMs = System.currentTimeMillis(); - return Failsafe.with(retryPolicy).with(retryExec) + + RetryPolicy retryPolicyWithMetrics = retryPolicy .onRetry(e -> metrics.retryCount.inc()) .onRetriesExceeded(e -> { metrics.retryTimer.update(System.currentTimeMillis() - startMs); metrics.permFailureCount.inc(); - }) - .onSuccess((e, ctx) -> { - if (ctx.getExecutions() > 1) { + }).onSuccess((e) -> { + if (e.getAttemptCount() > 1) { metrics.retryTimer.update(System.currentTimeMillis() - startMs); } else { metrics.successCount.inc(); } }); + + return Failsafe.with(retryPolicyWithMetrics).with(retryExec); } } diff --git a/samza-core/src/main/java/org/apache/samza/util/FutureUtil.java b/samza-core/src/main/java/org/apache/samza/util/FutureUtil.java new file mode 100644 index 0000000000..dc527e984c --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/util/FutureUtil.java @@ -0,0 +1,191 @@ +/* + * 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.samza.util; + +import net.jodah.failsafe.Failsafe; +import net.jodah.failsafe.RetryPolicy; + +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.ExecutorService; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class FutureUtil { + private static final Logger LOG = LoggerFactory.getLogger(FutureUtil.class); + + /** + * Returns a future that completes when all the futures in the provided collections of futures are complete. + * @param futureCollections collections of futures to complete before the returned future is complete + */ + @SafeVarargs + public static CompletableFuture allOf(Collection>... futureCollections) { + List> fvs = new ArrayList<>(); + for (Collection> futureCollection : futureCollections) { + if (!futureCollection.isEmpty()) { + fvs.add(CompletableFuture.allOf(futureCollection.toArray(new CompletableFuture[0]))); + } + } + + return CompletableFuture.allOf(fvs.toArray(new CompletableFuture[0])); + } + + /** + * Returns a future that completes when all the futures futures are complete. + * Returned future completes exceptionally if any future complete with a non-ignored error. + */ + public static CompletableFuture allOf(Predicate ignoreError, CompletableFuture... futures) { + CompletableFuture allFuture = CompletableFuture.allOf(futures); + return allFuture.handle((aVoid, t) -> { + for (CompletableFuture future : futures) { + try { + future.join(); + } catch (Throwable th) { + if (ignoreError.test(th)) { + // continue + } else { + throw th; + } + } + } + return null; + }); + } + + /** + * Helper method to convert: {@code Pair, CompletableFuture>} + * to: {@code CompletableFuture>} + * + * Returns a future that completes when both futures complete. + * Returned future completes exceptionally if either of the futures complete exceptionally. + */ + public static CompletableFuture> toFutureOfPair( + Pair, CompletableFuture> pairOfFutures) { + return CompletableFuture + .allOf(pairOfFutures.getLeft(), pairOfFutures.getRight()) + .thenApply(v -> Pair.of(pairOfFutures.getLeft().join(), pairOfFutures.getRight().join())); + } + + /** + * Helper method to convert: {@code Map>} + * to: {@code CompletableFuture>} + * + * Returns a future that completes when all value futures complete. + * Returned future completes exceptionally if any of the value futures complete exceptionally. + */ + public static CompletableFuture> toFutureOfMap(Map> keyToValueFutures) { + return CompletableFuture + .allOf(keyToValueFutures.values().toArray(new CompletableFuture[0])) + .thenApply(v -> keyToValueFutures.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().join()))); + } + + /** + * Helper method to convert: {@code Map>} + * to: {@code CompletableFuture>} + * + * Returns a future that completes with successful map entries, skipping any entries with ignored errors, + * when all value futures complete. + * Returned future completes exceptionally if any of the futures complete with a non-ignored error. + */ + public static CompletableFuture> toFutureOfMap( + Predicate ignoreError, Map> keyToValueFutures) { + CompletableFuture allEntriesFuture = + CompletableFuture.allOf(keyToValueFutures.values().toArray(new CompletableFuture[]{})); + + return allEntriesFuture.handle((aVoid, t) -> { + Map successfulResults = new HashMap<>(); + for (Map.Entry> entry : keyToValueFutures.entrySet()) { + K key = entry.getKey(); + try { + V value = entry.getValue().join(); + successfulResults.put(key, value); + } catch (Throwable th) { + if (ignoreError.test(th)) { + // else ignore and continue + LOG.warn("Ignoring value future completion error for key: {}", key, th); + } else { + throw th; + } + } + } + return successfulResults; + }); + } + + public static CompletableFuture executeAsyncWithRetries(String opName, + Supplier> action, + Predicate abortRetries, + ExecutorService executor) { + Duration maxDuration = Duration.ofMinutes(1); + + RetryPolicy retryPolicy = new RetryPolicy<>() + .withBackoff(100, 10000, ChronoUnit.MILLIS) + .withMaxDuration(maxDuration) + .abortOn(abortRetries) // stop retrying if predicate returns true + .onRetry(e -> LOG.warn("Action: {} attempt: {} completed with error {} after start. Retrying up to {}.", + opName, e.getAttemptCount(), e.getElapsedTime(), maxDuration, e.getLastFailure())); + + return Failsafe.with(retryPolicy).with(executor).getStageAsync(action::get); + } + + public static CompletableFuture failedFuture(Throwable t) { + final CompletableFuture cf = new CompletableFuture<>(); + cf.completeExceptionally(t); + return cf; + } + + /** + * Removes wrapper exceptions of the provided type from the provided throwable and returns the first cause + * that does not match the wrapper type. Useful for unwrapping CompletionException / SamzaException + * in stack traces and getting to the underlying cause. + * + * Returns null if provided Throwable is null or if there is no cause of non-wrapper type in the stack. + */ + public static Throwable unwrapExceptions(Class wrapperClassToUnwrap, T t) { + if (t == null) return null; + if (wrapperClassToUnwrap == null) return t; + + Throwable originalException = t; + while (wrapperClassToUnwrap.isAssignableFrom(originalException.getClass()) && + originalException.getCause() != null) { + originalException = originalException.getCause(); + } + + // can still be the wrapper class if no other cause was found. + if (wrapperClassToUnwrap.isAssignableFrom(originalException.getClass())) { + return null; + } else { + return originalException; + } + } +} diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 83f5c9df4b..dceb27b1a3 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -27,7 +27,7 @@ import java.time.Duration import java.util import java.util.{Base64, Optional} import java.util.concurrent.{CountDownLatch, ExecutorService, Executors, ScheduledExecutorService, ThreadPoolExecutor, TimeUnit} - +import java.util.function.Consumer import com.google.common.annotations.VisibleForTesting import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.samza.checkpoint.{CheckpointListener, OffsetManager, OffsetManagerMetrics} @@ -38,7 +38,7 @@ import org.apache.samza.container.host.{StatisticsMonitorImpl, SystemMemoryStati import org.apache.samza.context._ import org.apache.samza.diagnostics.DiagnosticsManager import org.apache.samza.job.model.{ContainerModel, JobModel, TaskMode} -import org.apache.samza.metrics.{JmxServer, JvmMetrics, MetricsRegistryMap, MetricsReporter} +import org.apache.samza.metrics.{JmxServer, JvmMetrics, MetricsRegistry, MetricsRegistryMap, MetricsReporter} import org.apache.samza.serializers._ import org.apache.samza.serializers.model.SamzaObjectMapper import org.apache.samza.startpoint.StartpointManager @@ -52,7 +52,6 @@ import org.apache.samza.util.{Util, _} import org.apache.samza.SamzaException import org.apache.samza.clustermanager.StandbyTaskUtil -import scala.collection.JavaConversions import scala.collection.JavaConverters._ object SamzaContainer extends Logging { @@ -146,7 +145,6 @@ object SamzaContainer extends Logging { val systemConfig = new SystemConfig(config) val containerModel = jobModel.getContainers.get(containerId) val containerName = "samza-container-%s" format containerId - val maxChangeLogStreamPartitions = jobModel.maxChangeLogStreamPartitions val containerPID = ManagementFactory.getRuntimeMXBean().getName() @@ -343,31 +341,10 @@ object SamzaContainer extends Logging { debug("Got system stream message serdes: %s" format systemStreamMessageSerdes) - val storeChangelogs = storageConfig - .getStoreNames.asScala - .filter(storageConfig.getChangelogStream(_).isPresent) - .map(name => (name, storageConfig.getChangelogStream(name).get)).toMap - .mapValues(StreamUtil.getSystemStreamFromNames(_)) + val storeChangelogs = storageConfig.getStoreChangelogs info("Got change log system streams: %s" format storeChangelogs) - /* - * This keeps track of the changelog SSPs that are associated with the whole container. This is used so that we can - * prefetch the metadata about the all of the changelog SSPs associated with the container whenever we need the - * metadata about some of the changelog SSPs. - * An example use case is when Samza writes offset files for stores ({@link TaskStorageManager}). Each task is - * responsible for its own offset file, but if we can do prefetching, then most tasks will already have cached - * metadata by the time they need the offset metadata. - * Note: By using all changelog streams to build the sspsToPrefetch, any fetches done for persisted stores will - * include the ssps for non-persisted stores, so this is slightly suboptimal. However, this does not increase the - * actual number of calls to the {@link SystemAdmin}, and we can decouple this logic from the per-task objects (e.g. - * {@link TaskStorageManager}). - */ - val changelogSSPMetadataCache = new SSPMetadataCache(systemAdmins, - Duration.ofSeconds(5), - SystemClock.instance, - getChangelogSSPsForContainer(containerModel, storeChangelogs).asJava) - val intermediateStreams = streamConfig .getStreamIds() .asScala @@ -398,7 +375,7 @@ object SamzaContainer extends Logging { systemMessageSerdes = systemMessageSerdes, systemStreamKeySerdes = systemStreamKeySerdes, systemStreamMessageSerdes = systemStreamMessageSerdes, - changeLogSystemStreams = storeChangelogs.values.toSet, + changeLogSystemStreams = storeChangelogs.asScala.values.toSet, controlMessageKeySerdes = controlMessageKeySerdes, intermediateMessageSerdes = intermediateStreamMessageSerdes) @@ -484,11 +461,19 @@ object SamzaContainer extends Logging { null } - val finalTaskFactory = TaskFactoryUtil.finalizeTaskFactory( taskFactory, taskThreadPool) + // executor for performing async commit operations for a task. + val commitThreadPoolSize = + Math.min( + Math.max(containerModel.getTasks.size() * 2, jobConfig.getCommitThreadPoolSize), + jobConfig.getCommitThreadPoolMaxSize + ) + val commitThreadPool = Executors.newFixedThreadPool(commitThreadPoolSize, + new ThreadFactoryBuilder().setNameFormat("Samza Task Commit Thread-%d").setDaemon(true).build()) + val taskModels = containerModel.getTasks.values.asScala val containerContext = new ContainerContextImpl(containerModel, samzaContainerMetrics.registry) val applicationContainerContextOption = applicationContainerContextFactoryOption @@ -498,8 +483,6 @@ object SamzaContainer extends Logging { val timerExecutor = Executors.newSingleThreadScheduledExecutor - var taskStorageManagers : Map[TaskName, TaskStorageManager] = Map() - val taskInstanceMetrics: Map[TaskName, TaskInstanceMetrics] = taskModels.map(taskModel => { (taskModel.getTaskName, new TaskInstanceMetrics("TaskName-%s" format taskModel.getTaskName)) }).toMap @@ -517,13 +500,15 @@ object SamzaContainer extends Logging { val loggedStorageBaseDir = getLoggedStorageBaseDir(jobConfig, defaultStoreBaseDir) info("Got base directory for logged data stores: %s" format loggedStorageBaseDir) + val stateStorageBackendRestoreFactory = ReflectionUtil + .getObj(storageConfig.getStateBackendRestoreFactory(), classOf[StateBackendFactory]) + val containerStorageManager = new ContainerStorageManager( checkpointManager, containerModel, streamMetadataCache, - changelogSSPMetadataCache, systemAdmins, - storeChangelogs.asJava, + storeChangelogs, sideInputStoresToSystemStreams.mapValues(systemStreamSet => systemStreamSet.toSet.asJava).asJava, storageEngineFactories.asJava, systemFactories.asJava, @@ -533,15 +518,19 @@ object SamzaContainer extends Logging { samzaContainerMetrics, jobContext, containerContext, + stateStorageBackendRestoreFactory, taskCollectors.asJava, loggedStorageBaseDir, nonLoggedStorageBaseDir, - maxChangeLogStreamPartitions, serdeManager, new SystemClock) storeWatchPaths.addAll(containerStorageManager.getStoreDirectoryPaths) + val stateStorageBackendBackupFactories = storageConfig.getStateBackendBackupFactories.asScala.map( + ReflectionUtil.getObj(_, classOf[StateBackendFactory]) + ) + // Create taskInstances val taskInstances: Map[TaskName, TaskInstance] = taskModels .filter(taskModel => taskModel.getTaskMode.eq(TaskMode.Active)).map(taskModel => { @@ -563,15 +552,23 @@ object SamzaContainer extends Logging { val taskSideInputSSPs = sideInputStoresToSSPs.values.flatMap(_.asScala).toSet info ("Got task side input SSPs: %s" format taskSideInputSSPs) - val storageManager = TaskStorageManagerFactory.create( - taskName, - containerStorageManager, - storeChangelogs, - systemAdmins, - loggedStorageBaseDir, - taskModel.getChangelogPartition, - config, - taskModel.getTaskMode) + val taskBackupManagerMap = new util.HashMap[String, TaskBackupManager]() + stateStorageBackendBackupFactories.asJava.forEach(new Consumer[StateBackendFactory] { + override def accept(factory: StateBackendFactory): Unit = { + val taskMetricsRegistry = + if (taskInstanceMetrics.contains(taskName) && + taskInstanceMetrics.get(taskName).isDefined) taskInstanceMetrics.get(taskName).get.registry + else new MetricsRegistryMap + val taskBackupManager = factory.getBackupManager(jobContext, containerContext, + taskModel, commitThreadPool, taskMetricsRegistry, config, new SystemClock, + loggedStorageBaseDir, nonLoggedStorageBaseDir) + taskBackupManagerMap.put(factory.getClass.getName, taskBackupManager) + } + }) + + val commitManager = new TaskStorageCommitManager(taskName, taskBackupManagerMap, + containerStorageManager, storeChangelogs, taskModel.getChangelogPartition, checkpointManager, config, + commitThreadPool, new StorageManagerUtil, loggedStorageBaseDir, taskInstanceMetrics.get(taskName).get) val tableManager = new TableManager(config) @@ -585,14 +582,16 @@ object SamzaContainer extends Logging { consumerMultiplexer = consumerMultiplexer, collector = taskCollectors.get(taskName).get, offsetManager = offsetManager, - storageManager = storageManager, + commitManager = commitManager, + containerStorageManager = containerStorageManager, tableManager = tableManager, - systemStreamPartitions = JavaConversions.setAsJavaSet(taskSSPs -- taskSideInputSSPs), + systemStreamPartitions = (taskSSPs -- taskSideInputSSPs).asJava, exceptionHandler = TaskInstanceExceptionHandler(taskInstanceMetrics.get(taskName).get, taskConfig), jobModel = jobModel, streamMetadataCache = streamMetadataCache, inputStreamMetadata = inputStreamMetadata, timerExecutor = timerExecutor, + commitThreadPool = commitThreadPool, jobContext = jobContext, containerContext = containerContext, applicationContainerContextOption = applicationContainerContextOption, @@ -601,7 +600,6 @@ object SamzaContainer extends Logging { val taskInstance = createTaskInstance(task) - taskStorageManagers += taskInstance.taskName -> storageManager (taskName, taskInstance) }).toMap @@ -684,6 +682,7 @@ object SamzaContainer extends Logging { diskSpaceMonitor = diskSpaceMonitor, hostStatisticsMonitor = hostStatisticsMonitor, taskThreadPool = taskThreadPool, + commitThreadPool = commitThreadPool, timerExecutor = timerExecutor, containerContext = containerContext, applicationContainerContextOption = applicationContainerContextOption, @@ -691,19 +690,6 @@ object SamzaContainer extends Logging { containerStorageManager = containerStorageManager, diagnosticsManager = diagnosticsManager) } - - /** - * Builds the set of SSPs for all changelogs on this container. - */ - @VisibleForTesting - private[container] def getChangelogSSPsForContainer(containerModel: ContainerModel, - changeLogSystemStreams: Map[String, SystemStream]): Set[SystemStreamPartition] = { - containerModel.getTasks.values().asScala - .map(taskModel => taskModel.getChangelogPartition) - .flatMap(changelogPartition => changeLogSystemStreams.map { case (_, systemStream) => - new SystemStreamPartition(systemStream, changelogPartition) }) - .toSet - } } class SamzaContainer( @@ -723,6 +709,7 @@ class SamzaContainer( reporters: Map[String, MetricsReporter] = Map(), jvm: JvmMetrics = null, taskThreadPool: ExecutorService = null, + commitThreadPool: ExecutorService = null, timerExecutor: ScheduledExecutorService = Executors.newSingleThreadScheduledExecutor, containerContext: ContainerContext, applicationContainerContextOption: Option[ApplicationContainerContext], @@ -1053,7 +1040,7 @@ class SamzaContainer( info("Shutting down task thread pool") try { taskThreadPool.shutdown() - if(taskThreadPool.awaitTermination(shutdownMs, TimeUnit.MILLISECONDS)) { + if (!taskThreadPool.awaitTermination(shutdownMs, TimeUnit.MILLISECONDS)) { taskThreadPool.shutdownNow() } } catch { @@ -1061,11 +1048,23 @@ class SamzaContainer( } } + if (commitThreadPool != null) { + info("Shutting down task commit thread pool") + try { + commitThreadPool.shutdown() + if(!commitThreadPool.awaitTermination(shutdownMs, TimeUnit.MILLISECONDS)) { + commitThreadPool.shutdownNow() + } + } catch { + case e: Exception => error(e.getMessage, e) + } + } + if (timerExecutor != null) { info("Shutting down timer executor") try { timerExecutor.shutdown() - if (timerExecutor.awaitTermination(shutdownMs, TimeUnit.MILLISECONDS)) { + if (!timerExecutor.awaitTermination(shutdownMs, TimeUnit.MILLISECONDS)) { timerExecutor.shutdownNow() } } catch { diff --git a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala index 2ebe46594a..8a872dedda 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala @@ -21,21 +21,26 @@ package org.apache.samza.container import java.util.{Collections, Objects, Optional} -import java.util.concurrent.ScheduledExecutorService - +import java.util.concurrent.{CompletableFuture, ExecutorService, ScheduledExecutorService, Semaphore, TimeUnit} import org.apache.samza.SamzaException -import org.apache.samza.checkpoint.{Checkpoint, CheckpointId, CheckpointedChangelogOffset, OffsetManager} +import org.apache.samza.checkpoint.kafka.{KafkaChangelogSSPOffset, KafkaStateCheckpointMarker} +import org.apache.samza.checkpoint.{CheckpointId, CheckpointV1, CheckpointV2, OffsetManager} import org.apache.samza.config.{Config, StreamConfig, TaskConfig} import org.apache.samza.context._ import org.apache.samza.job.model.{JobModel, TaskModel} import org.apache.samza.scheduler.{CallbackSchedulerImpl, EpochTimeScheduler, ScheduledCallback} import org.apache.samza.storage.kv.KeyValueStore -import org.apache.samza.storage.TaskStorageManager +import org.apache.samza.storage.{ContainerStorageManager, TaskStorageCommitManager} import org.apache.samza.system._ import org.apache.samza.table.TableManager import org.apache.samza.task._ +import org.apache.samza.util.ScalaJavaUtil.JavaOptionals.toRichOptional import org.apache.samza.util.{Logging, ScalaJavaUtil} +import java.util +import java.util.concurrent.atomic.AtomicReference +import java.util.function.BiConsumer +import java.util.function.Function import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ import scala.collection.{JavaConverters, Map} @@ -48,14 +53,16 @@ class TaskInstance( consumerMultiplexer: SystemConsumers, collector: TaskInstanceCollector, override val offsetManager: OffsetManager = new OffsetManager, - storageManager: TaskStorageManager = null, + commitManager: TaskStorageCommitManager = null, + containerStorageManager: ContainerStorageManager = null, tableManager: TableManager = null, val systemStreamPartitions: java.util.Set[SystemStreamPartition] = Collections.emptySet(), val exceptionHandler: TaskInstanceExceptionHandler = new TaskInstanceExceptionHandler, jobModel: JobModel = null, streamMetadataCache: StreamMetadataCache = null, inputStreamMetadata: Map[SystemStream, SystemStreamMetadata] = Map(), - timerExecutor : ScheduledExecutorService = null, + timerExecutor: ScheduledExecutorService = null, + commitThreadPool: ExecutorService = null, jobContext: JobContext, containerContext: ContainerContext, applicationContainerContextOption: Option[ApplicationContainerContext], @@ -73,8 +80,9 @@ class TaskInstance( private val kvStoreSupplier = ScalaJavaUtil.toJavaFunction( (storeName: String) => { - if (storageManager != null && storageManager.getStore(storeName).isDefined) { - storageManager.getStore(storeName).get.asInstanceOf[KeyValueStore[_, _]] + if (containerStorageManager != null) { + val storeOption = containerStorageManager.getStore(taskName, storeName).toOption + if (storeOption.isDefined) storeOption.get.asInstanceOf[KeyValueStore[_, _]] else null } else { null } @@ -97,12 +105,21 @@ class TaskInstance( systemStreamPartitions.foreach(ssp2CaughtupMapping += _ -> false) private val config: Config = jobContext.getConfig + val taskConfig = new TaskConfig(config) val streamConfig: StreamConfig = new StreamConfig(config) override val intermediateStreams: java.util.Set[String] = JavaConverters.setAsJavaSetConverter(streamConfig.getStreamIds.filter(streamConfig.getIsIntermediateStream)).asJava val streamsToDeleteCommittedMessages: Set[String] = streamConfig.getStreamIds.filter(streamConfig.getDeleteCommittedMessages).map(streamConfig.getPhysicalName).toSet + val checkpointWriteVersions = new TaskConfig(config).getCheckpointWriteVersions + + @volatile var lastCommitStartTimeMs = System.currentTimeMillis() + val commitMaxDelayMs = taskConfig.getCommitMaxDelayMs + val commitTimeoutMs = taskConfig.getCommitTimeoutMs + val commitInProgress = new Semaphore(1) + val commitException = new AtomicReference[Exception]() + def registerOffsets { debug("Registering offsets for taskName: %s" format taskName) offsetManager.register(taskName, systemStreamPartitions) @@ -121,10 +138,33 @@ class TaskInstance( def initTask { initCaughtUpMapping() - val taskConfig = new TaskConfig(config) + if (commitManager != null) { + debug("Starting commit manager for taskName: %s" format taskName) + + commitManager.init() + } else { + debug("Skipping commit manager initialization for taskName: %s" format taskName) + } + + if (offsetManager != null) { + val checkpoint = offsetManager.getLastTaskCheckpoint(taskName) + // Only required for checkpointV2 + if (checkpoint != null && checkpoint.getVersion == 2) { + val checkpointV2 = checkpoint.asInstanceOf[CheckpointV2] + // call cleanUp on backup managers in case the container previously failed during commit + // before completing this step + + // WARNING: cleanUp is NOT optional with blob stores since this is where we reset the TTL for + // tracked blobs. if this TTL reset is skipped, some of the blobs retained by future commits may + // be deleted in the background by the blob store, leading to data loss. + debug("Cleaning up stale state from previous run for taskName: %s" format taskName) + commitManager.cleanUp(checkpointV2.getCheckpointId, checkpointV2.getStateCheckpointMarkers) + } + } + if (taskConfig.getTransactionalStateRestoreEnabled() && taskConfig.getCommitMs > 0) { - // Commit immediately so the trimmed changelog messages - // will be sealed in a checkpoint + debug("Committing immediately on startup for taskName: %s so that the trimmed changelog " + + "messages will be sealed in a checkpoint" format taskName) commit } @@ -178,8 +218,9 @@ class TaskInstance( if (ssp2CaughtupMapping(incomingMessageSsp)) { metrics.messagesActuallyProcessed.inc - trace("Processing incoming message envelope for taskName and SSP: %s, %s" - format (taskName, incomingMessageSsp)) + // TODO BLOCKER pmaheshw reenable after demo +// trace("Processing incoming message envelope for taskName: %s SSP: %s offset: %s" +// format (taskName, incomingMessageSsp, envelope.getOffset)) exceptionHandler.maybeHandle { val callback = callbackFactory.createCallback() @@ -219,68 +260,245 @@ class TaskInstance( } def commit { - metrics.commits.inc + // ensure that only one commit (including sync and async phases) is ever in progress for a task. + + val commitStartNs = System.nanoTime() + // first check if there were any unrecoverable errors during the async stage of the pending commit + // and if so, shut down the container. + if (commitException.get() != null) { + throw new SamzaException("Unrecoverable error during pending commit for taskName: %s." format taskName, + commitException.get()) + } - val allCheckpointOffsets = new java.util.HashMap[SystemStreamPartition, String]() - val inputCheckpoint = offsetManager.buildCheckpoint(taskName) - if (inputCheckpoint != null) { - trace("Got input offsets for taskName: %s as: %s" format(taskName, inputCheckpoint.getOffsets)) - allCheckpointOffsets.putAll(inputCheckpoint.getOffsets) + // if no commit is in progress for this task, continue with this commit. + // if a previous commit is in progress but less than {@code task.commit.max.delay.ms} + // have elapsed since it started, skip this commit request. + // if more time has elapsed than that, block this commit until the previous commit + // is complete, then continue with this commit. + if (!commitInProgress.tryAcquire()) { + val timeSinceLastCommit = System.currentTimeMillis() - lastCommitStartTimeMs + if (timeSinceLastCommit < commitMaxDelayMs) { + info("Skipping commit for taskName: %s since another commit is in progress. " + + "%s ms have elapsed since the pending commit started." format (taskName, timeSinceLastCommit)) + metrics.commitsSkipped.set(metrics.commitsSkipped.getValue + 1) + return + } else { + warn("Blocking processing for taskName: %s until in-flight commit is complete. " + + "%s ms have elapsed since the pending commit started, " + + "which is greater than the max allowed commit delay: %s." + format (taskName, timeSinceLastCommit, commitMaxDelayMs)) + + if (!commitInProgress.tryAcquire(commitTimeoutMs, TimeUnit.MILLISECONDS)) { + val timeSinceLastCommit = System.currentTimeMillis() - lastCommitStartTimeMs + throw new SamzaException("Timeout waiting for pending commit for taskName: %s to finish. " + + "%s ms have elapsed since the pending commit started. Max allowed commit delay is %s ms " + + "and commit timeout beyond that is %s ms" format (taskName, timeSinceLastCommit, + commitMaxDelayMs, commitTimeoutMs)) + } + } } + // at this point the permit for semaphore has been acquired, proceed with commit. + // the first part of the commit needs to be exclusive with processing, so do it on the caller thread. + lastCommitStartTimeMs = System.currentTimeMillis() + + metrics.commits.inc + val checkpointId = CheckpointId.create() + + debug("Starting sync stage of commit for taskName: %s checkpointId: %s" format (taskName, checkpointId)) + + val inputOffsets = offsetManager.getLastProcessedOffsets(taskName) + trace("Got last processed input offsets for taskName: %s checkpointId: %s as: %s" + format(taskName, checkpointId, inputOffsets)) - trace("Flushing producers for taskName: %s" format taskName) + trace("Flushing producers for taskName: %s checkpointId: %s" format (taskName, checkpointId)) + // Flushes output, checkpoint and changelog producers collector.flush if (tableManager != null) { - trace("Flushing tables for taskName: %s" format taskName) + trace("Flushing tables for taskName: %s checkpointId: %s" format (taskName, checkpointId)) tableManager.flush() } - var newestChangelogOffsets: Map[SystemStreamPartition, Option[String]] = null - if (storageManager != null) { - trace("Flushing state stores for taskName: %s" format taskName) - newestChangelogOffsets = storageManager.flush() - trace("Got newest changelog offsets for taskName: %s as: %s " format(taskName, newestChangelogOffsets)) - } + // create a synchronous snapshot of stores for commit + debug("Creating synchronous state store snapshots for taskName: %s checkpointId: %s" + format (taskName, checkpointId)) + val snapshotStartTimeNs = System.nanoTime() + val snapshotSCMs = commitManager.snapshot(checkpointId) + metrics.snapshotNs.update(System.nanoTime() - snapshotStartTimeNs) + trace("Got synchronous snapshot SCMs for taskName: %s checkpointId: %s as: %s " + format(taskName, checkpointId, snapshotSCMs)) + + debug("Submitting async stage of commit for taskName: %s checkpointId: %s for execution" + format (taskName, checkpointId)) + val asyncStageStartNs = System.nanoTime() + // rest of the commit can happen asynchronously and concurrently with processing. + // schedule it on the commit executor and return. submitted runnable releases the + // commit semaphore permit when this commit is complete. + commitThreadPool.submit(new Runnable { + override def run(): Unit = { + debug("Starting async stage of commit for taskName: %s checkpointId: %s" format (taskName, checkpointId)) + + try { + val uploadStartTimeNs = System.nanoTime() + val uploadSCMsFuture = commitManager.upload(checkpointId, snapshotSCMs) + uploadSCMsFuture.whenComplete(new BiConsumer[util.Map[String, util.Map[String, String]], Throwable] { + override def accept(t: util.Map[String, util.Map[String, String]], throwable: Throwable): Unit = { + if (throwable == null) { + metrics.asyncUploadNs.update(System.nanoTime() - uploadStartTimeNs) + } else { + warn("Commit upload did not complete successfully for taskName: %s checkpointId: %s with error msg: %s" + format (taskName, checkpointId, throwable.getMessage)) + } + } + }) + + // explicit types required to make scala compiler happy + val checkpointWriteFuture: CompletableFuture[util.Map[String, util.Map[String, String]]] = + uploadSCMsFuture.thenApplyAsync(writeCheckpoint(checkpointId, inputOffsets), commitThreadPool) + + val cleanupStartTimeNs = System.nanoTime() + val cleanUpFuture: CompletableFuture[Void] = + checkpointWriteFuture.thenComposeAsync(cleanUp(checkpointId), commitThreadPool) + cleanUpFuture.whenComplete(new BiConsumer[Void, Throwable] { + override def accept(v: Void, throwable: Throwable): Unit = { + if (throwable == null) { + metrics.asyncCleanupNs.update(System.nanoTime() - cleanupStartTimeNs) + } else { + warn("Commit cleanup did not complete successfully for taskName: %s checkpointId: %s with error msg: %s" + format (taskName, checkpointId, throwable.getMessage)) + } + } + }) - val checkpointId = CheckpointId.create() - if (storageManager != null && newestChangelogOffsets != null) { - trace("Checkpointing stores for taskName: %s with checkpoint id: %s" format (taskName, checkpointId)) - storageManager.checkpoint(checkpointId, newestChangelogOffsets.toMap) - } + val trimFuture = cleanUpFuture.thenRunAsync( + trim(checkpointId, inputOffsets), commitThreadPool) + + trimFuture.whenCompleteAsync(handleCompletion(checkpointId, commitStartNs, asyncStageStartNs), commitThreadPool) + } catch { + case t: Throwable => handleCompletion(checkpointId, commitStartNs, asyncStageStartNs).accept(null, t) + } + } + }) + + metrics.commitSyncNs.update(System.nanoTime() - commitStartNs) + debug("Finishing sync stage of commit for taskName: %s checkpointId: %s" format (taskName, checkpointId)) + } + + private def writeCheckpoint(checkpointId: CheckpointId, inputOffsets: util.Map[SystemStreamPartition, String]) = { + new Function[util.Map[String, util.Map[String, String]], util.Map[String, util.Map[String, String]]]() { + override def apply(uploadSCMs: util.Map[String, util.Map[String, String]]) = { + trace("Got asynchronous upload SCMs for taskName: %s checkpointId: %s as: %s " + format(taskName, checkpointId, uploadSCMs)) + + debug("Creating and writing checkpoints for taskName: %s checkpointId: %s" format (taskName, checkpointId)) + checkpointWriteVersions.foreach(checkpointWriteVersion => { + val checkpoint = if (checkpointWriteVersion == 1) { + // build CheckpointV1 with KafkaChangelogSSPOffset for backwards compatibility + val allCheckpointOffsets = new util.HashMap[SystemStreamPartition, String]() + allCheckpointOffsets.putAll(inputOffsets) + val newestChangelogOffsets = KafkaStateCheckpointMarker.scmsToSSPOffsetMap(uploadSCMs) + newestChangelogOffsets.foreach { case (ssp, newestOffsetOption) => + val offset = new KafkaChangelogSSPOffset(checkpointId, newestOffsetOption.orNull).toString + allCheckpointOffsets.put(ssp, offset) + } + new CheckpointV1(allCheckpointOffsets) + } else if (checkpointWriteVersion == 2) { + new CheckpointV2(checkpointId, inputOffsets, uploadSCMs) + } else { + throw new SamzaException("Unsupported checkpoint write version: " + checkpointWriteVersion) + } + + trace("Writing checkpoint for taskName: %s checkpointId: %s as: %s" + format(taskName, checkpointId, checkpoint)) + + // Write input offsets and state checkpoint markers to task store and checkpoint directories + commitManager.writeCheckpointToStoreDirectories(checkpoint) + + // Write input offsets and state checkpoint markers to the checkpoint topic atomically + offsetManager.writeCheckpoint(taskName, checkpoint) + }) - if (newestChangelogOffsets != null) { - newestChangelogOffsets.foreach {case (ssp, newestOffsetOption) => - val offset = new CheckpointedChangelogOffset(checkpointId, newestOffsetOption.orNull).toString - allCheckpointOffsets.put(ssp, offset) + uploadSCMs } } - val checkpoint = new Checkpoint(allCheckpointOffsets) - trace("Got combined checkpoint offsets for taskName: %s as: %s" format (taskName, allCheckpointOffsets)) + } - offsetManager.writeCheckpoint(taskName, checkpoint) + private def cleanUp(checkpointId: CheckpointId) = { + new Function[util.Map[String, util.Map[String, String]], CompletableFuture[Void]] { + override def apply(uploadSCMs: util.Map[String, util.Map[String, String]]): CompletableFuture[Void] = { + // Perform cleanup on unused checkpoints + debug("Cleaning up old checkpoint state for taskName: %s checkpointId: %s" format(taskName, checkpointId)) + try { + commitManager.cleanUp(checkpointId, uploadSCMs) + } catch { + case e: Exception => + // WARNING: cleanUp is NOT optional with blob stores since this is where we reset the TTL for + // tracked blobs. if this TTL reset is skipped, some of the blobs retained by future commits may + // be deleted in the background by the blob store, leading to data loss. + throw new SamzaException( + "Failed to remove old checkpoint state for taskName: %s checkpointId: %s." + format(taskName, checkpointId), e) + } + } + } + } - if (storageManager != null) { - trace("Remove old checkpoint stores for taskName: %s" format taskName) - try { - storageManager.removeOldCheckpoints(checkpointId) - } catch { - case e: Exception => error("Failed to remove old checkpoints for task: %s. Current checkpointId: %s" format (taskName, checkpointId), e) + private def trim(checkpointId: CheckpointId, inputOffsets: util.Map[SystemStreamPartition, String]) = { + new Runnable { + override def run(): Unit = { + trace("Deleting committed input offsets from intermediate topics for taskName: %s checkpointId: %s" + format (taskName, checkpointId)) + inputOffsets.asScala + .filter { case (ssp, _) => streamsToDeleteCommittedMessages.contains(ssp.getStream) } // Only delete data of intermediate streams + .groupBy { case (ssp, _) => ssp.getSystem } + .foreach { case (systemName: String, offsets: Map[SystemStreamPartition, String]) => + systemAdmins.getSystemAdmin(systemName).deleteMessages(offsets.asJava) + } } } + } - if (inputCheckpoint != null) { - trace("Deleting committed input offsets for taskName: %s" format taskName) - inputCheckpoint.getOffsets.asScala - .filter { case (ssp, _) => streamsToDeleteCommittedMessages.contains(ssp.getStream) } // Only delete data of intermediate streams - .groupBy { case (ssp, _) => ssp.getSystem } - .foreach { case (systemName: String, offsets: Map[SystemStreamPartition, String]) => - systemAdmins.getSystemAdmin(systemName).deleteMessages(offsets.asJava) + private def handleCompletion(checkpointId: CheckpointId, commitStartNs: Long, asyncStageStartNs: Long) = { + new BiConsumer[Void, Throwable] { + override def accept(v: Void, e: Throwable): Unit = { + try { + debug("%s finishing async stage of commit for taskName: %s checkpointId: %s." + format (if (e == null) "Successfully" else "Unsuccessfully", taskName, checkpointId)) + if (e != null) { + val exception = new SamzaException("Unrecoverable error during async stage of commit " + + "for taskName: %s checkpointId: %s" format(taskName, checkpointId), e) + val exceptionSet = commitException.compareAndSet(null, exception) + if (!exceptionSet) { + // should never happen because there should be at most one async stage of commit in progress + // for a task and another one shouldn't be schedule if the previous one failed. throw a new + // exception on the caller thread for logging and debugging if this happens. + error("Should not have encountered a non-null saved exception during async stage of " + + "commit for taskName: %s checkpointId: %s" format(taskName, checkpointId), commitException.get()) + error("New exception during async stage of commit for taskName: %s checkpointId: %s" + format(taskName, checkpointId), exception) + throw new SamzaException("Should not have encountered a non-null saved exception " + + "during async stage of commit for taskName: %s checkpointId: %s. New exception logged above. " + + "Saved exception under Caused By.", commitException.get()) + } + } else { + metrics.commitAsyncNs.update(System.nanoTime() - asyncStageStartNs) + metrics.commitNs.update(System.nanoTime() - commitStartNs) + } + } finally { + // release the permit indicating that previous commit is complete. + commitInProgress.release() } + } } } def shutdownTask { + if (commitManager != null) { + debug("Shutting down commit manager for taskName: %s" format taskName) + commitManager.close() + } else { + debug("Skipping commit manager shutdown for taskName: %s" format taskName) + } applicationTaskContextOption.foreach(applicationTaskContext => { debug("Stopping application-defined task context for taskName: %s" format taskName) applicationTaskContext.stop() diff --git a/samza-core/src/main/scala/org/apache/samza/container/TaskInstanceMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/TaskInstanceMetrics.scala index bdd773c281..f13e37a678 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/TaskInstanceMetrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/TaskInstanceMetrics.scala @@ -37,7 +37,16 @@ class TaskInstanceMetrics( val flushes = newCounter("flush-calls") val pendingMessages = newGauge("pending-messages", 0) val messagesInFlight = newGauge("messages-in-flight", 0) - val asyncCallbackCompleted = newCounter("async-callback-complete-calls"); + val asyncCallbackCompleted = newCounter("async-callback-complete-calls") + + val commitsSkipped = newGauge("commits-skipped", 0) + val commitNs = newTimer("commit-ns") + val commitSyncNs = newTimer("commit-sync-ns") + val commitAsyncNs = newTimer("commit-async-ns") + val snapshotNs = newTimer("snapshot-ns") + val storeCheckpointNs = newTimer("store-checkpoint-ns") + val asyncUploadNs = newTimer("async-upload-ns") + val asyncCleanupNs = newTimer("async-cleanup-ns") def addOffsetGauge(systemStreamPartition: SystemStreamPartition, getValue: () => String) { newGauge("%s-%s-%d-offset" format (systemStreamPartition.getSystem, systemStreamPartition.getStream, systemStreamPartition.getPartition.getPartitionId), getValue) diff --git a/samza-core/src/main/scala/org/apache/samza/storage/KafkaNonTransactionalStateTaskBackupManager.scala b/samza-core/src/main/scala/org/apache/samza/storage/KafkaNonTransactionalStateTaskBackupManager.scala new file mode 100644 index 0000000000..633191bd89 --- /dev/null +++ b/samza-core/src/main/scala/org/apache/samza/storage/KafkaNonTransactionalStateTaskBackupManager.scala @@ -0,0 +1,91 @@ +/* + * 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.samza.storage + +import java.util +import java.util.concurrent.CompletableFuture + +import com.google.common.collect.ImmutableSet +import org.apache.samza.checkpoint.kafka.KafkaStateCheckpointMarker +import org.apache.samza.checkpoint.{Checkpoint, CheckpointId} +import org.apache.samza.container.TaskName +import org.apache.samza.system._ +import org.apache.samza.util.Logging +import org.apache.samza.{Partition, SamzaException} + +import scala.collection.JavaConverters._ + +/** + * Manage all the storage engines for a given task + */ +class KafkaNonTransactionalStateTaskBackupManager( + taskName: TaskName, + storeChangelogs: util.Map[String, SystemStream] = new util.HashMap[String, SystemStream](), + systemAdmins: SystemAdmins, + partition: Partition) extends Logging with TaskBackupManager { + + override def init(checkpoint: Checkpoint): Unit = {} + + override def snapshot(checkpointId: CheckpointId): util.Map[String, String] = { + debug("Getting newest offsets for kafka changelog SSPs.") + getNewestChangelogSSPOffsets() + } + + override def upload(checkpointId: CheckpointId, + stateCheckpointMarkers: util.Map[String, String]): CompletableFuture[util.Map[String, String]] = { + CompletableFuture.completedFuture(stateCheckpointMarkers) + } + + override def cleanUp(checkpointId: CheckpointId, + stateCheckpointMarker: util.Map[String, String]): CompletableFuture[Void] = { + CompletableFuture.completedFuture(null) + } + + override def close() {} + + /** + * Returns the newest offset for each store changelog SSP for this task. + * @return A map of changelog SSPs for this task to their newest offset (or None if ssp is empty) + * @throws SamzaException if there was an error fetching newest offset for any SSP + */ + private def getNewestChangelogSSPOffsets(): util.Map[String, String] = { + storeChangelogs.asScala + .map { case (storeName, systemStream) => { + debug("Fetching newest offset for taskName %s store %s changelog %s" format (taskName, storeName, systemStream)) + val ssp = new SystemStreamPartition(systemStream.getSystem, systemStream.getStream, partition) + val systemAdmin = systemAdmins.getSystemAdmin(systemStream.getSystem) + + try { + val sspMetadataOption = Option(systemAdmin.getSSPMetadata(ImmutableSet.of(ssp)).get(ssp)) + + // newest offset == null implies topic is empty + val newestOffsetOption = sspMetadataOption.flatMap(sspMetadata => Option(sspMetadata.getNewestOffset)) + newestOffsetOption.foreach(newestOffset => + debug("Got newest offset %s for taskName %s store %s changelog %s" format(newestOffset, taskName, storeName, systemStream))) + + (storeName, KafkaStateCheckpointMarker.serialize(new KafkaStateCheckpointMarker(ssp, newestOffsetOption.orNull))) + } catch { + case e: Exception => + throw new SamzaException("Error getting newest changelog offset for taskName %s store %s changelog %s." + format(taskName, storeName, systemStream), e) + } + }}.asJava + } +} diff --git a/samza-core/src/main/scala/org/apache/samza/storage/KafkaTransactionalStateTaskBackupManager.scala b/samza-core/src/main/scala/org/apache/samza/storage/KafkaTransactionalStateTaskBackupManager.scala new file mode 100644 index 0000000000..dc28fe77fa --- /dev/null +++ b/samza-core/src/main/scala/org/apache/samza/storage/KafkaTransactionalStateTaskBackupManager.scala @@ -0,0 +1,97 @@ +package org.apache.samza.storage + +/* + * 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. + */ + +import java.util +import java.util.concurrent.CompletableFuture + +import com.google.common.annotations.VisibleForTesting +import com.google.common.collect.ImmutableSet +import org.apache.samza.checkpoint.kafka.KafkaStateCheckpointMarker +import org.apache.samza.checkpoint.{Checkpoint, CheckpointId} +import org.apache.samza.container.TaskName +import org.apache.samza.system._ +import org.apache.samza.util.Logging +import org.apache.samza.{Partition, SamzaException} + +import scala.collection.JavaConverters._ + +/** + * Manage all the storage engines for a given task + */ +class KafkaTransactionalStateTaskBackupManager( + taskName: TaskName, + storeChangelogs: util.Map[String, SystemStream] = new util.HashMap[String, SystemStream](), + systemAdmins: SystemAdmins, + partition: Partition) extends Logging with TaskBackupManager { + + override def init(checkpoint: Checkpoint): Unit = {} + + override def snapshot(checkpointId: CheckpointId): util.Map[String, String] = { + debug("Getting newest offsets for kafka changelog SSPs.") + getNewestChangelogSSPOffsets(taskName, storeChangelogs, partition, systemAdmins) + } + + override def upload(checkpointId: CheckpointId, snapshotCheckpointsMap: util.Map[String, String]): + CompletableFuture[util.Map[String, String]] = { + CompletableFuture.completedFuture(snapshotCheckpointsMap) + } + + override def cleanUp(checkpointId: CheckpointId, + stateCheckpointMarker: util.Map[String, String]): CompletableFuture[Void] = { + CompletableFuture.completedFuture(null) + } + + override def close() {} + + /** + * Returns the newest offset for each store changelog SSP for this task. Returned map will + * always contain an entry for every changelog SSP. + * @return A map of storenames for this task to their ssp and newest offset (null if empty) wrapped in KafkaStateCheckpointMarker + * @throws SamzaException if there was an error fetching newest offset for any SSP + */ + @VisibleForTesting + def getNewestChangelogSSPOffsets(taskName: TaskName, storeChangelogs: util.Map[String, SystemStream], + partition: Partition, systemAdmins: SystemAdmins): util.Map[String, String] = { + storeChangelogs.asScala + .map { case (storeName, systemStream) => { + try { + debug("Fetching newest offset for taskName %s store %s changelog %s" format (taskName, storeName, systemStream)) + val ssp = new SystemStreamPartition(systemStream.getSystem, systemStream.getStream, partition) + val systemAdmin = systemAdmins.getSystemAdmin(systemStream.getSystem) + + val sspMetadata = Option(systemAdmin.getSSPMetadata(ImmutableSet.of(ssp)).get(ssp)) + .getOrElse(throw new SamzaException("Received null metadata for ssp: %s" format ssp)) + + // newest offset == null implies topic is empty + val newestOffsetOption = Option(sspMetadata.getNewestOffset) + newestOffsetOption.foreach(newestOffset => + debug("Got newest offset %s for taskName %s store %s changelog %s" format(newestOffset, taskName, storeName, systemStream))) + + (storeName, KafkaStateCheckpointMarker.serialize(new KafkaStateCheckpointMarker(ssp, newestOffsetOption.orNull))) + } catch { + case e: Exception => + throw new SamzaException("Error getting newest changelog offset for taskName %s store %s changelog %s." + format(taskName, storeName, systemStream), e) + } + }} + .toMap.asJava + } +} diff --git a/samza-core/src/main/scala/org/apache/samza/storage/NonTransactionalStateTaskStorageManager.scala b/samza-core/src/main/scala/org/apache/samza/storage/NonTransactionalStateTaskStorageManager.scala deleted file mode 100644 index 7b3874907e..0000000000 --- a/samza-core/src/main/scala/org/apache/samza/storage/NonTransactionalStateTaskStorageManager.scala +++ /dev/null @@ -1,144 +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.samza.storage - -import java.io._ - -import com.google.common.annotations.VisibleForTesting -import com.google.common.collect.ImmutableSet -import org.apache.samza.checkpoint.CheckpointId -import org.apache.samza.container.TaskName -import org.apache.samza.job.model.TaskMode -import org.apache.samza.system._ -import org.apache.samza.util.Logging -import org.apache.samza.util.ScalaJavaUtil.JavaOptionals -import org.apache.samza.{Partition, SamzaException} - -import scala.collection.JavaConverters._ - -/** - * Manage all the storage engines for a given task - */ -class NonTransactionalStateTaskStorageManager( - taskName: TaskName, - containerStorageManager: ContainerStorageManager, - storeChangelogs: Map[String, SystemStream] = Map(), - systemAdmins: SystemAdmins, - loggedStoreBaseDir: File = new File(System.getProperty("user.dir"), "state"), - partition: Partition) extends Logging with TaskStorageManager { - - private val storageManagerUtil = new StorageManagerUtil - private val persistedStores = containerStorageManager.getAllStores(taskName).asScala - .filter { case (storeName, storageEngine) => storageEngine.getStoreProperties.isPersistedToDisk } - - def getStore(storeName: String): Option[StorageEngine] = JavaOptionals.toRichOptional(containerStorageManager.getStore(taskName, storeName)).toOption - - def flush(): Map[SystemStreamPartition, Option[String]] = { - debug("Flushing stores.") - containerStorageManager.getAllStores(taskName).asScala.values.foreach(_.flush) - val newestChangelogSSPOffsets = getNewestChangelogSSPOffsets() - writeChangelogOffsetFiles(newestChangelogSSPOffsets) - newestChangelogSSPOffsets - } - - override def checkpoint(checkpointId: CheckpointId, - newestChangelogOffsets: Map[SystemStreamPartition, Option[String]]): Unit = {} - - override def removeOldCheckpoints(checkpointId: CheckpointId): Unit = {} - - @VisibleForTesting - def stop() { - debug("Stopping stores.") - containerStorageManager.stopStores() - } - - /** - * Returns the newest offset for each store changelog SSP for this task. - * @return A map of changelog SSPs for this task to their newest offset (or None if ssp is empty) - * @throws SamzaException if there was an error fetching newest offset for any SSP - */ - private def getNewestChangelogSSPOffsets(): Map[SystemStreamPartition, Option[String]] = { - storeChangelogs - .map { case (storeName, systemStream) => { - debug("Fetching newest offset for taskName %s store %s changelog %s" format (taskName, storeName, systemStream)) - val ssp = new SystemStreamPartition(systemStream.getSystem, systemStream.getStream, partition) - val systemAdmin = systemAdmins.getSystemAdmin(systemStream.getSystem) - - try { - val sspMetadataOption = Option(systemAdmin.getSSPMetadata(ImmutableSet.of(ssp)).get(ssp)) - - // newest offset == null implies topic is empty - val newestOffsetOption = sspMetadataOption.flatMap(sspMetadata => Option(sspMetadata.getNewestOffset)) - newestOffsetOption.foreach(newestOffset => - debug("Got newest offset %s for taskName %s store %s changelog %s" format(newestOffset, taskName, storeName, systemStream))) - - (ssp, newestOffsetOption) - } catch { - case e: Exception => - throw new SamzaException("Error getting newest changelog offset for taskName %s store %s changelog %s." - format(taskName, storeName, systemStream), e) - } - }} - } - - /** - * Writes the newest changelog ssp offset for each persistent store to the OFFSET file on disk. - * These files are used during container startup to determine whether there is any new information in the - * changelog that is not reflected in the on-disk copy of the store. If there is any delta, it is replayed - * from the changelog e.g. This can happen if the job was run on this host, then another - * host and back to this host. - */ - private def writeChangelogOffsetFiles(newestChangelogOffsets: Map[SystemStreamPartition, Option[String]]) { - debug("Writing OFFSET files for logged persistent key value stores for task %s." format(taskName)) - - storeChangelogs - .filterKeys(storeName => persistedStores.contains(storeName)) - .foreach { case (storeName, systemStream) => { - debug("Writing changelog offset for taskName %s store %s changelog %s." format(taskName, storeName, systemStream)) - val currentStoreDir = storageManagerUtil.getTaskStoreDir(loggedStoreBaseDir, storeName, taskName, TaskMode.Active) - try { - val ssp = new SystemStreamPartition(systemStream.getSystem, systemStream.getStream, partition) - newestChangelogOffsets(ssp) match { - case Some(newestOffset) => { - debug("Storing newest offset %s for taskName %s store %s changelog %s in OFFSET file." - format(newestOffset, taskName, storeName, systemStream)) - // TaskStorageManagers are only created for active tasks - storageManagerUtil.writeOffsetFile(currentStoreDir, Map(ssp -> newestOffset).asJava, false) - debug("Successfully stored offset %s for taskName %s store %s changelog %s in OFFSET file." - format(newestOffset, taskName, storeName, systemStream)) - } - case None => { - // if newestOffset is null, then it means the changelog ssp is (or has become) empty. This could be - // either because the changelog topic was newly added, repartitioned, or manually deleted and recreated. - // No need to persist the offset file. - storageManagerUtil.deleteOffsetFile(currentStoreDir) - debug("Deleting OFFSET file for taskName %s store %s changelog ssp %s since the newestOffset is null." - format (taskName, storeName, ssp)) - } - } - } catch { - case e: Exception => - throw new SamzaException("Error storing offset for taskName %s store %s changelog %s." - format(taskName, storeName, systemStream), e) - } - }} - debug("Done writing OFFSET files for logged persistent key value stores for task %s" format(taskName)) - } -} diff --git a/samza-core/src/main/scala/org/apache/samza/storage/TransactionalStateTaskStorageManager.scala b/samza-core/src/main/scala/org/apache/samza/storage/TransactionalStateTaskStorageManager.scala deleted file mode 100644 index 0335710a35..0000000000 --- a/samza-core/src/main/scala/org/apache/samza/storage/TransactionalStateTaskStorageManager.scala +++ /dev/null @@ -1,201 +0,0 @@ -package org.apache.samza.storage - -/* - * 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. - */ - -import java.io._ -import java.nio.file.Path - -import com.google.common.annotations.VisibleForTesting -import com.google.common.collect.ImmutableSet -import org.apache.commons.io.FileUtils -import org.apache.commons.io.filefilter.WildcardFileFilter -import org.apache.samza.checkpoint.CheckpointId -import org.apache.samza.{Partition, SamzaException} -import org.apache.samza.container.TaskName -import org.apache.samza.job.model.TaskMode -import org.apache.samza.system._ -import org.apache.samza.util.ScalaJavaUtil.JavaOptionals -import org.apache.samza.util.Logging - -import scala.collection.JavaConverters._ - -/** - * Manage all the storage engines for a given task - */ -class TransactionalStateTaskStorageManager( - taskName: TaskName, - containerStorageManager: ContainerStorageManager, - storeChangelogs: Map[String, SystemStream] = Map(), - systemAdmins: SystemAdmins, - loggedStoreBaseDir: File = new File(System.getProperty("user.dir"), "state"), - partition: Partition, - taskMode: TaskMode, - storageManagerUtil: StorageManagerUtil) extends Logging with TaskStorageManager { - - def getStore(storeName: String): Option[StorageEngine] = JavaOptionals.toRichOptional(containerStorageManager.getStore(taskName, storeName)).toOption - - def flush(): Map[SystemStreamPartition, Option[String]] = { - debug("Flushing stores.") - containerStorageManager.getAllStores(taskName).asScala.values.foreach(_.flush) - getNewestChangelogSSPOffsets(taskName, storeChangelogs, partition, systemAdmins) - } - - def checkpoint(checkpointId: CheckpointId, newestChangelogOffsets: Map[SystemStreamPartition, Option[String]]): Unit = { - debug("Checkpointing stores.") - - val checkpointPaths = containerStorageManager.getAllStores(taskName).asScala - .filter { case (storeName, storeEngine) => - storeEngine.getStoreProperties.isLoggedStore && storeEngine.getStoreProperties.isPersistedToDisk} - .flatMap { case (storeName, storeEngine) => { - val pathOptional = storeEngine.checkpoint(checkpointId) - if (pathOptional.isPresent) { - Some(storeName, pathOptional.get()) - } else { - None - } - }} - .toMap - - writeChangelogOffsetFiles(checkpointPaths, storeChangelogs, newestChangelogOffsets) - } - - def removeOldCheckpoints(latestCheckpointId: CheckpointId): Unit = { - if (latestCheckpointId != null) { - debug("Removing older checkpoints before " + latestCheckpointId) - - val files = loggedStoreBaseDir.listFiles() - if (files != null) { - files - .foreach(storeDir => { - val storeName = storeDir.getName - val taskStoreName = storageManagerUtil.getTaskStoreDir( - loggedStoreBaseDir, storeName, taskName, taskMode).getName - val fileFilter: FileFilter = new WildcardFileFilter(taskStoreName + "-*") - val checkpointDirs = storeDir.listFiles(fileFilter) - - if (checkpointDirs != null) { - checkpointDirs - .filter(!_.getName.contains(latestCheckpointId.toString)) - .foreach(checkpointDir => { - FileUtils.deleteDirectory(checkpointDir) - }) - } - }) - } - } - } - - @VisibleForTesting - def stop() { - debug("Stopping stores.") - containerStorageManager.stopStores() - } - - /** - * Returns the newest offset for each store changelog SSP for this task. Returned map will - * always contain an entry for every changelog SSP. - * @return A map of changelog SSPs for this task to their newest offset (or None if ssp is empty) - * @throws SamzaException if there was an error fetching newest offset for any SSP - */ - @VisibleForTesting - def getNewestChangelogSSPOffsets(taskName: TaskName, storeChangelogs: Map[String, SystemStream], - partition: Partition, systemAdmins: SystemAdmins): Map[SystemStreamPartition, Option[String]] = { - storeChangelogs - .map { case (storeName, systemStream) => { - try { - debug("Fetching newest offset for taskName %s store %s changelog %s" format (taskName, storeName, systemStream)) - val ssp = new SystemStreamPartition(systemStream.getSystem, systemStream.getStream, partition) - val systemAdmin = systemAdmins.getSystemAdmin(systemStream.getSystem) - - val sspMetadata = Option(systemAdmin.getSSPMetadata(ImmutableSet.of(ssp)).get(ssp)) - .getOrElse(throw new SamzaException("Received null metadata for ssp: %s" format ssp)) - - // newest offset == null implies topic is empty - val newestOffsetOption = Option(sspMetadata.getNewestOffset) - newestOffsetOption.foreach(newestOffset => - debug("Got newest offset %s for taskName %s store %s changelog %s" format(newestOffset, taskName, storeName, systemStream))) - - (ssp, newestOffsetOption) - } catch { - case e: Exception => - throw new SamzaException("Error getting newest changelog offset for taskName %s store %s changelog %s." - format(taskName, storeName, systemStream), e) - } - }} - .toMap - } - - /** - * Writes the newest changelog ssp offset for each persistent store the OFFSET file in both the checkpoint - * and the current store directory (the latter for allowing rollbacks). - * - * These files are used during container startup to ensure transactional state, and to determine whether the - * there is any new information in the changelog that is not reflected in the on-disk copy of the store. - * If there is any delta, it is replayed from the changelog e.g. This can happen if the job was run on this host, - * then another host, and then back to this host. - */ - @VisibleForTesting - def writeChangelogOffsetFiles(checkpointPaths: Map[String, Path], storeChangelogs: Map[String, SystemStream], - newestChangelogOffsets: Map[SystemStreamPartition, Option[String]]): Unit = { - debug("Writing OFFSET files for logged persistent key value stores for task %s." format(checkpointPaths)) - - storeChangelogs - .filterKeys(storeName => checkpointPaths.contains(storeName)) - .foreach { case (storeName, systemStream) => { - try { - val ssp = new SystemStreamPartition(systemStream.getSystem, systemStream.getStream, partition) - val currentStoreDir = storageManagerUtil.getTaskStoreDir(loggedStoreBaseDir, storeName, taskName, TaskMode.Active) - newestChangelogOffsets(ssp) match { - case Some(newestOffset) => { - // write the offset file for the checkpoint directory - val checkpointPath = checkpointPaths(storeName) - writeChangelogOffsetFile(storeName, ssp, newestOffset, checkpointPath.toFile) - // write the OFFSET file for the current store (for backwards compatibility / allowing rollbacks) - writeChangelogOffsetFile(storeName, ssp, newestOffset, currentStoreDir) - } - case None => { - // retain existing behavior for current store directory for backwards compatibility / allowing rollbacks - - // if newestOffset is null, then it means the changelog ssp is (or has become) empty. This could be - // either because the changelog topic was newly added, repartitioned, or manually deleted and recreated. - // No need to persist the offset file. - storageManagerUtil.deleteOffsetFile(currentStoreDir) - debug("Deleting OFFSET file for taskName %s current store %s changelog ssp %s since the newestOffset is null." - format (taskName, storeName, ssp)) - } - } - } catch { - case e: Exception => - throw new SamzaException("Error storing offset for taskName %s store %s changelog %s." - format(taskName, storeName, systemStream), e) - } - }} - debug("Done writing OFFSET files for logged persistent key value stores for task %s" format(taskName)) - } - - private def writeChangelogOffsetFile(storeName: String, ssp: SystemStreamPartition, - newestOffset: String, dir: File): Unit = { - debug("Storing newest offset: %s for taskName: %s store: %s changelog: %s in OFFSET file at path: %s." - format(newestOffset, taskName, storeName, ssp, dir)) - storageManagerUtil.writeOffsetFile(dir, Map(ssp -> newestOffset).asJava, false) - debug("Successfully stored offset: %s for taskName: %s store: %s changelog: %s in OFFSET file at path: %s." - format(newestOffset, taskName, storeName, ssp, dir)) - } -} diff --git a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java index 88fbbe0a6c..e634940999 100644 --- a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java +++ b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java @@ -41,6 +41,8 @@ public class TestStorageConfig { private static final String STORE_NAME0 = "store0"; private static final String STORE_NAME1 = "store1"; + private static final String STORE_NAME2 = "store2"; + private static final String STORE_NAME3 = "store3"; @Test public void testGetStoreNames() { @@ -137,6 +139,68 @@ public void testGetChangelogStreamMissingSystem() { storageConfig.getChangelogStream(STORE_NAME0); } + @Test + public void testGetBackupManagerFactories() { + String factory1 = "factory1"; + String factory2 = "factory2"; + String factory3 = "factory3"; + StorageConfig storageConfig = new StorageConfig(new MapConfig( + ImmutableMap.of( + String.format(STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME0), factory1 + "," + factory2, + String.format(STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME1), factory1, + String.format(STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME2), factory3, + // store_name3 should use DEFAULT_STATE_BACKEND_FACTORY due to changelog presence + String.format(CHANGELOG_STREAM, STORE_NAME3), "nondefault-changelog-system.streamName"), + ImmutableMap.of( + String.format(FACTORY, STORE_NAME0), "store0.factory.class", + String.format(FACTORY, STORE_NAME1), "store1.factory.class", + String.format(FACTORY, STORE_NAME2), "store2.factory.class", + String.format(FACTORY, STORE_NAME3), "store3.factory.class", + // this store should have no backend factory configured + String.format(FACTORY, "noFactoryStore"), "noFactory.factory.class" + ) + )); + Set factories = storageConfig.getStateBackendBackupFactories(); + assertTrue(factories.contains(factory1)); + assertTrue(factories.contains(factory2)); + assertTrue(factories.contains(factory3)); + assertTrue(factories.contains(DEFAULT_STATE_BACKEND_FACTORY)); + assertEquals(4, factories.size()); + assertEquals(ImmutableList.of(factory1, factory2), storageConfig.getStoreBackupManagerClassName(STORE_NAME0)); + assertEquals(ImmutableList.of(factory1), storageConfig.getStoreBackupManagerClassName(STORE_NAME1)); + assertEquals(ImmutableList.of(factory3), storageConfig.getStoreBackupManagerClassName(STORE_NAME2)); + assertEquals(DEFAULT_STATE_BACKEND_BACKUP_FACTORIES, storageConfig.getStoreBackupManagerClassName(STORE_NAME3)); + assertTrue(storageConfig.getStoreBackupManagerClassName("emptyStore").isEmpty()); + assertTrue(storageConfig.getStoreBackupManagerClassName("noFactoryStore").isEmpty()); + } + + @Test + public void testGetStoreToBackup() { + String targetFactory = "target.class"; + StorageConfig config = new StorageConfig(new MapConfig( + ImmutableMap.of( + String.format(StorageConfig.STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME0), targetFactory, + String.format(StorageConfig.STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME1), targetFactory + "," + + DEFAULT_STATE_BACKEND_FACTORY, + String.format(StorageConfig.STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME2), DEFAULT_STATE_BACKEND_FACTORY), + ImmutableMap.of( + String.format(FACTORY, STORE_NAME0), "store0.factory.class", + String.format(FACTORY, STORE_NAME1), "store1.factory.class", + String.format(FACTORY, STORE_NAME2), "store2.factory.class", + String.format(FACTORY, STORE_NAME3), "store3.factory.class", + String.format(CHANGELOG_STREAM, STORE_NAME3), "nondefault-changelog-system.streamName" + ) + )); + + List targetStoreNames = config.getBackupStoreNamesForStateBackupFactory(targetFactory); + List defaultStoreNames = config.getBackupStoreNamesForStateBackupFactory( + DEFAULT_STATE_BACKEND_FACTORY); + assertTrue(targetStoreNames.containsAll(ImmutableList.of(STORE_NAME0, STORE_NAME1))); + assertEquals(2, targetStoreNames.size()); + assertTrue(defaultStoreNames.containsAll(ImmutableList.of(STORE_NAME2, STORE_NAME1, STORE_NAME3))); + assertEquals(3, defaultStoreNames.size()); + } + @Test public void testGetAccessLogEnabled() { // empty config, access log disabled diff --git a/samza-core/src/test/java/org/apache/samza/storage/TestKafkaChangelogStateBackendFactory.java b/samza-core/src/test/java/org/apache/samza/storage/TestKafkaChangelogStateBackendFactory.java new file mode 100644 index 0000000000..5782a75331 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/storage/TestKafkaChangelogStateBackendFactory.java @@ -0,0 +1,80 @@ +/* + * 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.samza.storage; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import java.util.Collections; +import java.util.Map; +import java.util.Set; +import org.apache.samza.Partition; +import org.apache.samza.container.TaskName; +import org.apache.samza.context.ContainerContextImpl; +import org.apache.samza.job.model.ContainerModel; +import org.apache.samza.job.model.TaskModel; +import org.apache.samza.system.SystemStream; +import org.apache.samza.system.SystemStreamPartition; +import org.junit.Assert; +import org.junit.Test; + + +public class TestKafkaChangelogStateBackendFactory { + + @Test + public void testGetChangelogSSP() { + KafkaChangelogStateBackendFactory factory = new KafkaChangelogStateBackendFactory(); + TaskName taskName0 = new TaskName("task0"); + TaskName taskName1 = new TaskName("task1"); + TaskModel taskModel0 = new TaskModel(taskName0, + ImmutableSet.of(new SystemStreamPartition("input", "stream", new Partition(0))), + new Partition(10)); + TaskModel taskModel1 = new TaskModel(taskName1, + ImmutableSet.of(new SystemStreamPartition("input", "stream", new Partition(1))), new Partition(11)); + ContainerModel containerModel = new ContainerModel("processorId", + ImmutableMap.of(taskName0, taskModel0, taskName1, taskModel1)); + Map changeLogSystemStreams = ImmutableMap.of( + "store0", new SystemStream("changelogSystem0", "store0-changelog"), + "store1", new SystemStream("changelogSystem1", "store1-changelog")); + Set expected = ImmutableSet.of( + new SystemStreamPartition("changelogSystem0", "store0-changelog", new Partition(10)), + new SystemStreamPartition("changelogSystem1", "store1-changelog", new Partition(10)), + new SystemStreamPartition("changelogSystem0", "store0-changelog", new Partition(11)), + new SystemStreamPartition("changelogSystem1", "store1-changelog", new Partition(11))); + Assert.assertEquals(expected, factory.getChangelogSSPForContainer(changeLogSystemStreams, + new ContainerContextImpl(containerModel, null))); + } + + @Test + public void testGetChangelogSSPsForContainerNoChangelogs() { + KafkaChangelogStateBackendFactory factory = new KafkaChangelogStateBackendFactory(); + TaskName taskName0 = new TaskName("task0"); + TaskName taskName1 = new TaskName("task1"); + TaskModel taskModel0 = new TaskModel(taskName0, + ImmutableSet.of(new SystemStreamPartition("input", "stream", new Partition(0))), + new Partition(10)); + TaskModel taskModel1 = new TaskModel(taskName1, + ImmutableSet.of(new SystemStreamPartition("input", "stream", new Partition(1))), + new Partition(11)); + ContainerModel containerModel = new ContainerModel("processorId", + ImmutableMap.of(taskName0, taskModel0, taskName1, taskModel1)); + Assert.assertEquals(Collections.emptySet(), factory.getChangelogSSPForContainer(Collections.emptyMap(), + new ContainerContextImpl(containerModel, null))); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/storage/TestTaskStorageCommitManager.java b/samza-core/src/test/java/org/apache/samza/storage/TestTaskStorageCommitManager.java new file mode 100644 index 0000000000..e682aef332 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/storage/TestTaskStorageCommitManager.java @@ -0,0 +1,878 @@ +/* + * 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.samza.storage; + +import com.google.common.collect.ImmutableMap; +import java.io.File; +import java.io.FileFilter; +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ForkJoinPool; +import org.apache.samza.Partition; +import org.apache.samza.SamzaException; +import org.apache.samza.checkpoint.Checkpoint; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.checkpoint.CheckpointManager; +import org.apache.samza.checkpoint.CheckpointV1; +import org.apache.samza.checkpoint.CheckpointV2; +import org.apache.samza.checkpoint.kafka.KafkaChangelogSSPOffset; +import org.apache.samza.config.MapConfig; +import org.apache.samza.container.TaskInstanceMetrics; +import org.apache.samza.container.TaskName; +import org.apache.samza.job.model.TaskMode; +import org.apache.samza.metrics.Timer; +import org.apache.samza.system.SystemStream; +import org.apache.samza.system.SystemStreamPartition; +import org.junit.Test; + +import static org.junit.Assert.*; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.*; + + +public class TestTaskStorageCommitManager { + @Test + public void testCommitManagerStart() { + CheckpointManager checkpointManager = mock(CheckpointManager.class); + TaskBackupManager taskBackupManager1 = mock(TaskBackupManager.class); + TaskBackupManager taskBackupManager2 = mock(TaskBackupManager.class); + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + Checkpoint checkpoint = mock(Checkpoint.class); + + TaskName taskName = new TaskName("task1"); + Map backupManagers = ImmutableMap.of( + "factory1", taskBackupManager1, + "factory2", taskBackupManager2 + ); + TaskStorageCommitManager cm = new TaskStorageCommitManager(taskName, backupManagers, containerStorageManager, + Collections.emptyMap(), new Partition(1), checkpointManager, new MapConfig(), + ForkJoinPool.commonPool(), new StorageManagerUtil(), null, null); + + when(checkpointManager.readLastCheckpoint(taskName)).thenReturn(checkpoint); + cm.init(); + verify(taskBackupManager1).init(eq(checkpoint)); + verify(taskBackupManager2).init(eq(checkpoint)); + } + + @Test + public void testCommitManagerStartNullCheckpointManager() { + TaskBackupManager taskBackupManager1 = mock(TaskBackupManager.class); + TaskBackupManager taskBackupManager2 = mock(TaskBackupManager.class); + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + + TaskName task = new TaskName("task1"); + Map backupManagers = ImmutableMap.of( + "factory1", taskBackupManager1, + "factory2", taskBackupManager2 + ); + TaskStorageCommitManager cm = new TaskStorageCommitManager(task, backupManagers, containerStorageManager, + Collections.emptyMap(), new Partition(1), null, new MapConfig(), + ForkJoinPool.commonPool(), new StorageManagerUtil(), null, null); + cm.init(); + verify(taskBackupManager1).init(eq(null)); + verify(taskBackupManager2).init(eq(null)); + } + + @Test + public void testSnapshotAndCommitAllFactories() { + CheckpointManager checkpointManager = mock(CheckpointManager.class); + TaskBackupManager taskBackupManager1 = mock(TaskBackupManager.class); + TaskBackupManager taskBackupManager2 = mock(TaskBackupManager.class); + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + Checkpoint checkpoint = mock(Checkpoint.class); + TaskInstanceMetrics metrics = mock(TaskInstanceMetrics.class); + Timer checkpointTimer = mock(Timer.class); + when(metrics.storeCheckpointNs()).thenReturn(checkpointTimer); + + TaskName taskName = new TaskName("task1"); + Map backupManagers = ImmutableMap.of( + "factory1", taskBackupManager1, + "factory2", taskBackupManager2 + ); + TaskStorageCommitManager cm = new TaskStorageCommitManager(taskName, backupManagers, containerStorageManager, + Collections.emptyMap(), new Partition(1), checkpointManager, new MapConfig(), + ForkJoinPool.commonPool(), new StorageManagerUtil(), null, metrics); + when(checkpointManager.readLastCheckpoint(taskName)).thenReturn(checkpoint); + cm.init(); + verify(taskBackupManager1).init(eq(checkpoint)); + verify(taskBackupManager2).init(eq(checkpoint)); + + CheckpointId newCheckpointId = CheckpointId.create(); + Map factory1Checkpoints = ImmutableMap.of( + "store1", "system;stream;1", + "store2", "system;stream;2" + ); + Map factory2Checkpoints = ImmutableMap.of( + "store1", "blobId1", + "store2", "blobId2" + ); + + when(containerStorageManager.getAllStores(taskName)).thenReturn(Collections.emptyMap()); + when(taskBackupManager1.snapshot(newCheckpointId)).thenReturn(factory1Checkpoints); + when(taskBackupManager2.snapshot(newCheckpointId)).thenReturn(factory2Checkpoints); + + when(taskBackupManager1.upload(newCheckpointId, factory1Checkpoints)) + .thenReturn(CompletableFuture.completedFuture(factory1Checkpoints)); + when(taskBackupManager2.upload(newCheckpointId, factory2Checkpoints)) + .thenReturn(CompletableFuture.completedFuture(factory2Checkpoints)); + + Map> snapshotSCMs = cm.snapshot(newCheckpointId); + cm.upload(newCheckpointId, snapshotSCMs); + + // Test flow for snapshot + verify(taskBackupManager1).snapshot(newCheckpointId); + verify(taskBackupManager2).snapshot(newCheckpointId); + + // Test flow for upload + verify(taskBackupManager1).upload(newCheckpointId, factory1Checkpoints); + verify(taskBackupManager2).upload(newCheckpointId, factory2Checkpoints); + verify(checkpointTimer).update(anyLong()); + } + + @Test + public void testFlushAndCheckpointOnSnapshot() { + CheckpointManager checkpointManager = mock(CheckpointManager.class); + TaskBackupManager taskBackupManager1 = mock(TaskBackupManager.class); + TaskBackupManager taskBackupManager2 = mock(TaskBackupManager.class); + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + Checkpoint checkpoint = mock(Checkpoint.class); + + StorageEngine mockLPStore = mock(StorageEngine.class); + StoreProperties lpStoreProps = mock(StoreProperties.class); + when(mockLPStore.getStoreProperties()).thenReturn(lpStoreProps); + when(lpStoreProps.isPersistedToDisk()).thenReturn(true); + when(lpStoreProps.isDurableStore()).thenReturn(true); + Path mockPath = mock(Path.class); + when(mockLPStore.checkpoint(any())).thenReturn(Optional.of(mockPath)); + + StorageEngine mockPStore = mock(StorageEngine.class); + StoreProperties pStoreProps = mock(StoreProperties.class); + when(mockPStore.getStoreProperties()).thenReturn(pStoreProps); + when(pStoreProps.isPersistedToDisk()).thenReturn(true); + when(pStoreProps.isDurableStore()).thenReturn(false); + + StorageEngine mockLIStore = mock(StorageEngine.class); + StoreProperties liStoreProps = mock(StoreProperties.class); + when(mockLIStore.getStoreProperties()).thenReturn(liStoreProps); + when(liStoreProps.isPersistedToDisk()).thenReturn(false); + when(liStoreProps.isDurableStore()).thenReturn(true); + + StorageEngine mockIStore = mock(StorageEngine.class); + StoreProperties iStoreProps = mock(StoreProperties.class); + when(mockIStore.getStoreProperties()).thenReturn(iStoreProps); + when(iStoreProps.isPersistedToDisk()).thenReturn(false); + when(iStoreProps.isDurableStore()).thenReturn(false); + + TaskName taskName = new TaskName("task1"); + Map backupManagers = ImmutableMap.of( + "factory1", taskBackupManager1, + "factory2", taskBackupManager2 + ); + Map storageEngines = ImmutableMap.of( + "storeLP", mockLPStore, + "storeP", mockPStore, + "storeLI", mockLIStore, + "storeI", mockIStore + ); + + TaskInstanceMetrics metrics = mock(TaskInstanceMetrics.class); + Timer checkpointTimer = mock(Timer.class); + when(metrics.storeCheckpointNs()).thenReturn(checkpointTimer); + + TaskStorageCommitManager cm = new TaskStorageCommitManager(taskName, backupManagers, containerStorageManager, + Collections.emptyMap(), new Partition(1), checkpointManager, new MapConfig(), + ForkJoinPool.commonPool(), new StorageManagerUtil(), null, metrics); + when(checkpointManager.readLastCheckpoint(taskName)).thenReturn(checkpoint); + cm.init(); + verify(taskBackupManager1).init(eq(checkpoint)); + verify(taskBackupManager2).init(eq(checkpoint)); + + CheckpointId newCheckpointId = CheckpointId.create(); + Map factory1Checkpoints = ImmutableMap.of( + "store1", "system;stream;1", + "store2", "system;stream;2" + ); + Map factory2Checkpoints = ImmutableMap.of( + "store1", "blobId1", + "store2", "blobId2" + ); + + when(containerStorageManager.getAllStores(taskName)).thenReturn(storageEngines); + when(taskBackupManager1.snapshot(newCheckpointId)).thenReturn(factory1Checkpoints); + when(taskBackupManager1.upload(newCheckpointId, factory1Checkpoints)) + .thenReturn(CompletableFuture.completedFuture(factory1Checkpoints)); + when(taskBackupManager2.snapshot(newCheckpointId)).thenReturn(factory2Checkpoints); + when(taskBackupManager2.upload(newCheckpointId, factory2Checkpoints)) + .thenReturn(CompletableFuture.completedFuture(factory2Checkpoints)); + when(mockLIStore.checkpoint(newCheckpointId)).thenReturn(Optional.empty()); + + cm.init(); + cm.snapshot(newCheckpointId); + + // Assert stores where flushed + verify(mockIStore).flush(); + verify(mockPStore).flush(); + verify(mockLIStore).flush(); + verify(mockLPStore).flush(); + // only logged and persisted stores are checkpointed + verify(mockLPStore).checkpoint(newCheckpointId); + // ensure that checkpoint is never called for non-logged persistent stores since they're + // always cleared on restart. + verify(mockPStore, never()).checkpoint(any()); + // ensure that checkpoint is never called for non-persistent stores + verify(mockIStore, never()).checkpoint(any()); + verify(mockLIStore, never()).checkpoint(any()); + verify(checkpointTimer).update(anyLong()); + } + + @Test(expected = IllegalStateException.class) + public void testSnapshotFailsIfErrorCreatingCheckpoint() { + CheckpointManager checkpointManager = mock(CheckpointManager.class); + TaskBackupManager taskBackupManager1 = mock(TaskBackupManager.class); + TaskBackupManager taskBackupManager2 = mock(TaskBackupManager.class); + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + + StorageEngine mockLPStore = mock(StorageEngine.class); + StoreProperties lpStoreProps = mock(StoreProperties.class); + when(mockLPStore.getStoreProperties()).thenReturn(lpStoreProps); + when(lpStoreProps.isPersistedToDisk()).thenReturn(true); + when(lpStoreProps.isDurableStore()).thenReturn(true); + when(mockLPStore.checkpoint(any())).thenThrow(new IllegalStateException()); + + TaskInstanceMetrics metrics = mock(TaskInstanceMetrics.class); + Timer checkpointTimer = mock(Timer.class); + when(metrics.storeCheckpointNs()).thenReturn(checkpointTimer); + + TaskName taskName = new TaskName("task1"); + Map backupManagers = ImmutableMap.of( + "factory1", taskBackupManager1, + "factory2", taskBackupManager2 + ); + Map storageEngines = ImmutableMap.of( + "storeLP", mockLPStore + ); + + TaskStorageCommitManager cm = new TaskStorageCommitManager(taskName, backupManagers, containerStorageManager, + Collections.emptyMap(), new Partition(1), checkpointManager, new MapConfig(), + ForkJoinPool.commonPool(), new StorageManagerUtil(), null, metrics); + + when(containerStorageManager.getAllStores(taskName)).thenReturn(storageEngines); + CheckpointId newCheckpointId = CheckpointId.create(); + cm.init(); + cm.snapshot(newCheckpointId); + + // Assert stores where flushed + verify(mockLPStore).flush(); + // only logged and persisted stores are checkpointed + verify(mockLPStore).checkpoint(newCheckpointId); + verify(taskBackupManager1, never()).snapshot(any()); + verify(taskBackupManager2, never()).snapshot(any()); + verify(taskBackupManager1, never()).upload(any(), any()); + verify(taskBackupManager2, never()).upload(any(), any()); + fail("Should have thrown an exception when the storageEngine#checkpoint did not succeed"); + } + + @Test + public void testCleanupAllBackupManagers() { + CheckpointManager checkpointManager = mock(CheckpointManager.class); + TaskBackupManager taskBackupManager1 = mock(TaskBackupManager.class); + TaskBackupManager taskBackupManager2 = mock(TaskBackupManager.class); + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + Checkpoint checkpoint = mock(Checkpoint.class); + File durableStoreDir = mock(File.class); + when(durableStoreDir.listFiles()).thenReturn(new File[0]); + TaskInstanceMetrics metrics = mock(TaskInstanceMetrics.class); + Timer checkpointTimer = mock(Timer.class); + when(metrics.storeCheckpointNs()).thenReturn(checkpointTimer); + + TaskName taskName = new TaskName("task1"); + Map backupManagers = ImmutableMap.of( + "factory1", taskBackupManager1, + "factory2", taskBackupManager2 + ); + TaskStorageCommitManager cm = new TaskStorageCommitManager(taskName, backupManagers, containerStorageManager, + Collections.emptyMap(), new Partition(1), checkpointManager, new MapConfig(), + ForkJoinPool.commonPool(), new StorageManagerUtil(), durableStoreDir, metrics); + when(checkpointManager.readLastCheckpoint(taskName)).thenReturn(checkpoint); + when(containerStorageManager.getAllStores(taskName)).thenReturn(Collections.emptyMap()); + when(taskBackupManager1.cleanUp(any(), any())).thenReturn(CompletableFuture.completedFuture(null)); + when(taskBackupManager2.cleanUp(any(), any())).thenReturn(CompletableFuture.completedFuture(null)); + Map factory1Checkpoints = ImmutableMap.of( + "store1", "system;stream;1", + "store2", "system;stream;2" + ); + Map factory2Checkpoints = ImmutableMap.of( + "store1", "blobId1", + "store2", "blobId2" + ); + Map> factoryCheckpointsMap = ImmutableMap.of( + "factory1", factory1Checkpoints, + "factory2", factory2Checkpoints + ); + + when(taskBackupManager1.cleanUp(any(), any())).thenReturn(CompletableFuture.completedFuture(null)); + when(taskBackupManager2.cleanUp(any(), any())).thenReturn(CompletableFuture.completedFuture(null)); + + CheckpointId newCheckpointId = CheckpointId.create(); + cm.cleanUp(newCheckpointId, factoryCheckpointsMap).join(); + + verify(taskBackupManager1).cleanUp(newCheckpointId, factory1Checkpoints); + verify(taskBackupManager2).cleanUp(newCheckpointId, factory2Checkpoints); + } + + @Test + public void testCleanupFailsIfBackupManagerNotInitiated() { + CheckpointManager checkpointManager = mock(CheckpointManager.class); + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + Checkpoint checkpoint = mock(Checkpoint.class); + File durableStoreDir = mock(File.class); + when(durableStoreDir.listFiles()).thenReturn(new File[0]); + TaskInstanceMetrics metrics = mock(TaskInstanceMetrics.class); + Timer checkpointTimer = mock(Timer.class); + when(metrics.storeCheckpointNs()).thenReturn(checkpointTimer); + + TaskName taskName = new TaskName("task1"); + when(containerStorageManager.getAllStores(taskName)).thenReturn(Collections.emptyMap()); + TaskStorageCommitManager cm = new TaskStorageCommitManager(taskName, Collections.emptyMap(), containerStorageManager, + Collections.emptyMap(), new Partition(1), checkpointManager, new MapConfig(), + ForkJoinPool.commonPool(), new StorageManagerUtil(), durableStoreDir, metrics); + when(checkpointManager.readLastCheckpoint(taskName)).thenReturn(checkpoint); + + Map> factoryCheckpointsMap = ImmutableMap.of( + "factory3", Collections.emptyMap() // factory 3 should be ignored + ); + + CheckpointId newCheckpointId = CheckpointId.create(); + cm.cleanUp(newCheckpointId, factoryCheckpointsMap); + // should not fail the commit because the job should ignore any factories checkpoints not initialized + // in case the user is in a migration phase from on state backend to another + } + + @Test + public void testPersistToFileSystemCheckpointV1AndV2Checkpoint() throws IOException { + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + StorageEngine mockLPStore = mock(StorageEngine.class); + StoreProperties lpStoreProps = mock(StoreProperties.class); + when(mockLPStore.getStoreProperties()).thenReturn(lpStoreProps); + when(lpStoreProps.isPersistedToDisk()).thenReturn(true); + when(lpStoreProps.isDurableStore()).thenReturn(true); + Path mockPath = mock(Path.class); + when(mockLPStore.checkpoint(any())).thenReturn(Optional.of(mockPath)); + + StorageEngine mockPStore = mock(StorageEngine.class); + StoreProperties pStoreProps = mock(StoreProperties.class); + when(mockPStore.getStoreProperties()).thenReturn(pStoreProps); + when(pStoreProps.isPersistedToDisk()).thenReturn(true); + when(pStoreProps.isDurableStore()).thenReturn(false); + + StorageEngine mockLIStore = mock(StorageEngine.class); + StoreProperties liStoreProps = mock(StoreProperties.class); + when(mockLIStore.getStoreProperties()).thenReturn(liStoreProps); + when(liStoreProps.isPersistedToDisk()).thenReturn(false); + when(liStoreProps.isDurableStore()).thenReturn(true); + + StorageEngine mockIStore = mock(StorageEngine.class); + StoreProperties iStoreProps = mock(StoreProperties.class); + when(mockIStore.getStoreProperties()).thenReturn(iStoreProps); + when(iStoreProps.isPersistedToDisk()).thenReturn(false); + when(iStoreProps.isDurableStore()).thenReturn(false); + + java.util.Map taskStores = ImmutableMap.of( + "loggedPersistentStore", mockLPStore, + "persistentStore", mockPStore, + "loggedInMemStore", mockLIStore, + "inMemStore", mockIStore + ); + + Partition changelogPartition = new Partition(0); + SystemStream changelogSystemStream = new SystemStream("changelogSystem", "changelogStream"); + SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); + java.util.Map storeChangelogsStreams = ImmutableMap.of( + "loggedPersistentStore", changelogSystemStream, + "loggedInMemStore", new SystemStream("system", "stream") + ); + + StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class); + File durableStoreDir = new File("durableStorePath"); + when(storageManagerUtil.getTaskStoreDir(eq(durableStoreDir), any(), any(), any())).thenReturn(durableStoreDir); + TaskName taskName = new TaskName("task"); + + TaskInstanceMetrics metrics = mock(TaskInstanceMetrics.class); + Timer checkpointTimer = mock(Timer.class); + when(metrics.storeCheckpointNs()).thenReturn(checkpointTimer); + + when(containerStorageManager.getAllStores(taskName)).thenReturn(taskStores); + TaskStorageCommitManager commitManager = spy(new TaskStorageCommitManager(taskName, + Collections.emptyMap(), containerStorageManager, storeChangelogsStreams, changelogPartition, + null, null, ForkJoinPool.commonPool(), storageManagerUtil, durableStoreDir, metrics)); + doNothing().when(commitManager).writeChangelogOffsetFile(any(), any(), any(), any()); + + CheckpointId newCheckpointId = CheckpointId.create(); + + String newestOffset = "1"; + KafkaChangelogSSPOffset kafkaChangelogSSPOffset = new KafkaChangelogSSPOffset(newCheckpointId, newestOffset); + java.util.Map offsetsJava = ImmutableMap.of( + changelogSSP, kafkaChangelogSSPOffset.toString() + ); + + commitManager.init(); + // invoke persist to file system for v2 checkpoint + commitManager.writeCheckpointToStoreDirectories(new CheckpointV1(offsetsJava)); + + verify(commitManager).writeChangelogOffsetFiles(offsetsJava); + // evoked twice, for OFFSET-V1 and OFFSET-V2 + verify(commitManager).writeChangelogOffsetFile( + eq("loggedPersistentStore"), eq(changelogSSP), eq(newestOffset), eq(durableStoreDir)); + File checkpointFile = Paths.get(StorageManagerUtil + .getCheckpointDirPath(durableStoreDir, kafkaChangelogSSPOffset.getCheckpointId())).toFile(); + verify(commitManager).writeChangelogOffsetFile( + eq("loggedPersistentStore"), eq(changelogSSP), eq(newestOffset), eq(checkpointFile)); + + java.util.Map storeSCM = ImmutableMap.of( + "loggedPersistentStore", "system;loggedPersistentStoreStream;1", + "persistentStore", "system;persistentStoreStream;1", + "loggedInMemStore", "system;loggedInMemStoreStream;1", + "inMemStore", "system;inMemStoreStream;1" + ); + CheckpointV2 checkpoint = new CheckpointV2(newCheckpointId, Collections.emptyMap(), Collections.singletonMap("factory", storeSCM)); + + // invoke persist to file system for v2 checkpoint + commitManager.writeCheckpointToStoreDirectories(checkpoint); + // Validate only durable and persisted stores are persisted + // This should be evoked twice, for checkpointV1 and checkpointV2 + verify(storageManagerUtil, times(2)).getTaskStoreDir(eq(durableStoreDir), eq("loggedPersistentStore"), eq(taskName), any()); + File checkpointPath = Paths.get(StorageManagerUtil.getCheckpointDirPath(durableStoreDir, newCheckpointId)).toFile(); + verify(storageManagerUtil).writeCheckpointV2File(eq(checkpointPath), eq(checkpoint)); + } + + @Test + public void testPersistToFileSystemCheckpointV2Only() throws IOException { + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + StorageEngine mockLPStore = mock(StorageEngine.class); + StoreProperties lpStoreProps = mock(StoreProperties.class); + when(mockLPStore.getStoreProperties()).thenReturn(lpStoreProps); + when(lpStoreProps.isPersistedToDisk()).thenReturn(true); + when(lpStoreProps.isDurableStore()).thenReturn(true); + Path mockPath = mock(Path.class); + when(mockLPStore.checkpoint(any())).thenReturn(Optional.of(mockPath)); + + StorageEngine mockPStore = mock(StorageEngine.class); + StoreProperties pStoreProps = mock(StoreProperties.class); + when(mockPStore.getStoreProperties()).thenReturn(pStoreProps); + when(pStoreProps.isPersistedToDisk()).thenReturn(true); + when(pStoreProps.isDurableStore()).thenReturn(false); + + StorageEngine mockLIStore = mock(StorageEngine.class); + StoreProperties liStoreProps = mock(StoreProperties.class); + when(mockLIStore.getStoreProperties()).thenReturn(liStoreProps); + when(liStoreProps.isPersistedToDisk()).thenReturn(false); + when(liStoreProps.isDurableStore()).thenReturn(true); + + StorageEngine mockIStore = mock(StorageEngine.class); + StoreProperties iStoreProps = mock(StoreProperties.class); + when(mockIStore.getStoreProperties()).thenReturn(iStoreProps); + when(iStoreProps.isPersistedToDisk()).thenReturn(false); + when(iStoreProps.isDurableStore()).thenReturn(false); + + java.util.Map taskStores = ImmutableMap.of( + "loggedPersistentStore", mockLPStore, + "persistentStore", mockPStore, + "loggedInMemStore", mockLIStore, + "inMemStore", mockIStore + ); + + Partition changelogPartition = new Partition(0); + SystemStream changelogSystemStream = new SystemStream("changelogSystem", "changelogStream"); + SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); + java.util.Map storeChangelogsStreams = ImmutableMap.of( + "loggedPersistentStore", changelogSystemStream, + "loggedInMemStore", new SystemStream("system", "stream") + ); + + StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class); + File durableStoreDir = new File("durableStorePath"); + when(storageManagerUtil.getTaskStoreDir(eq(durableStoreDir), eq("loggedPersistentStore"), any(), any())) + .thenReturn(durableStoreDir); + TaskName taskName = new TaskName("task"); + + TaskInstanceMetrics metrics = mock(TaskInstanceMetrics.class); + Timer checkpointTimer = mock(Timer.class); + when(metrics.storeCheckpointNs()).thenReturn(checkpointTimer); + + when(containerStorageManager.getAllStores(taskName)).thenReturn(taskStores); + TaskStorageCommitManager commitManager = spy(new TaskStorageCommitManager(taskName, + Collections.emptyMap(), containerStorageManager, storeChangelogsStreams, changelogPartition, + null, null, ForkJoinPool.commonPool(), storageManagerUtil, durableStoreDir, metrics)); + doNothing().when(commitManager).writeChangelogOffsetFile(any(), any(), any(), any()); + + CheckpointId newCheckpointId = CheckpointId.create(); + + java.util.Map storeSCM = ImmutableMap.of( + "loggedPersistentStore", "system;loggedPersistentStoreStream;1", + "persistentStore", "system;persistentStoreStream;1", + "loggedInMemStore", "system;loggedInMemStoreStream;1", + "inMemStore", "system;inMemStoreStream;1" + ); + CheckpointV2 checkpoint = new CheckpointV2(newCheckpointId, Collections.emptyMap(), Collections.singletonMap("factory", storeSCM)); + + commitManager.init(); + // invoke persist to file system + commitManager.writeCheckpointToStoreDirectories(checkpoint); + // Validate only durable and persisted stores are persisted + verify(storageManagerUtil).getTaskStoreDir(eq(durableStoreDir), eq("loggedPersistentStore"), eq(taskName), any()); + File checkpointPath = Paths.get(StorageManagerUtil.getCheckpointDirPath(durableStoreDir, newCheckpointId)).toFile(); + verify(storageManagerUtil).writeCheckpointV2File(eq(checkpointPath), eq(checkpoint)); + } + + @Test + public void testWriteChangelogOffsetFilesV1() throws IOException { + Map> mockFileSystem = new HashMap<>(); + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + StorageEngine mockLPStore = mock(StorageEngine.class); + StoreProperties lpStoreProps = mock(StoreProperties.class); + when(mockLPStore.getStoreProperties()).thenReturn(lpStoreProps); + when(lpStoreProps.isPersistedToDisk()).thenReturn(true); + when(lpStoreProps.isDurableStore()).thenReturn(true); + Path mockPath = mock(Path.class); + when(mockLPStore.checkpoint(any())).thenReturn(Optional.of(mockPath)); + TaskInstanceMetrics metrics = mock(TaskInstanceMetrics.class); + Timer checkpointTimer = mock(Timer.class); + when(metrics.storeCheckpointNs()).thenReturn(checkpointTimer); + + java.util.Map taskStores = ImmutableMap.of("loggedPersistentStore", mockLPStore); + + Partition changelogPartition = new Partition(0); + SystemStream changelogSystemStream = new SystemStream("changelogSystem", "changelogStream"); + SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); + java.util.Map storeChangelogsStreams = ImmutableMap.of("loggedPersistentStore", changelogSystemStream); + + StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class); + File tmpTestPath = new File("store-checkpoint-test"); + when(storageManagerUtil.getTaskStoreDir(eq(tmpTestPath), eq("loggedPersistentStore"), any(), any())).thenReturn(tmpTestPath); + TaskName taskName = new TaskName("task"); + + when(containerStorageManager.getAllStores(taskName)).thenReturn(taskStores); + TaskStorageCommitManager commitManager = spy(new TaskStorageCommitManager(taskName, + Collections.emptyMap(), containerStorageManager, storeChangelogsStreams, changelogPartition, + null, null, ForkJoinPool.commonPool(), storageManagerUtil, tmpTestPath, metrics)); + + doAnswer(invocation -> { + String fileDir = invocation.getArgumentAt(3, File.class).getName(); + SystemStreamPartition ssp = invocation.getArgumentAt(1, SystemStreamPartition.class); + String offset = invocation.getArgumentAt(2, String.class); + if (mockFileSystem.containsKey(fileDir)) { + mockFileSystem.get(fileDir).put(ssp, offset); + } else { + Map sspOffsets = new HashMap<>(); + sspOffsets.put(ssp, offset); + mockFileSystem.put(fileDir, sspOffsets); + } + return null; + }).when(commitManager).writeChangelogOffsetFile(any(), any(), any(), any()); + + CheckpointId newCheckpointId = CheckpointId.create(); + + String newestOffset = "1"; + KafkaChangelogSSPOffset kafkaChangelogSSPOffset = new KafkaChangelogSSPOffset(newCheckpointId, newestOffset); + java.util.Map offsetsJava = ImmutableMap.of( + changelogSSP, kafkaChangelogSSPOffset.toString() + ); + + commitManager.init(); + // invoke persist to file system for v2 checkpoint + commitManager.writeCheckpointToStoreDirectories(new CheckpointV1(offsetsJava)); + + assertEquals(2, mockFileSystem.size()); + // check if v2 offsets are written correctly + String v2FilePath = StorageManagerUtil + .getCheckpointDirPath(tmpTestPath, newCheckpointId); + assertTrue(mockFileSystem.containsKey(v2FilePath)); + assertTrue(mockFileSystem.get(v2FilePath).containsKey(changelogSSP)); + assertEquals(1, mockFileSystem.get(v2FilePath).size()); + assertEquals(newestOffset, mockFileSystem.get(v2FilePath).get(changelogSSP)); + // check if v1 offsets are written correctly + String v1FilePath = tmpTestPath.getPath(); + assertTrue(mockFileSystem.containsKey(v1FilePath)); + assertTrue(mockFileSystem.get(v1FilePath).containsKey(changelogSSP)); + assertEquals(1, mockFileSystem.get(v1FilePath).size()); + assertEquals(newestOffset, mockFileSystem.get(v1FilePath).get(changelogSSP)); + } + + @Test + public void testWriteChangelogOffsetFilesV2andV1() throws IOException { + Map> mockFileSystem = new HashMap<>(); + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + Map mockCheckpointFileSystem = new HashMap<>(); + StorageEngine mockLPStore = mock(StorageEngine.class); + StoreProperties lpStoreProps = mock(StoreProperties.class); + when(mockLPStore.getStoreProperties()).thenReturn(lpStoreProps); + when(lpStoreProps.isPersistedToDisk()).thenReturn(true); + when(lpStoreProps.isDurableStore()).thenReturn(true); + Path mockPath = mock(Path.class); + when(mockLPStore.checkpoint(any())).thenReturn(Optional.of(mockPath)); + TaskInstanceMetrics metrics = mock(TaskInstanceMetrics.class); + Timer checkpointTimer = mock(Timer.class); + when(metrics.storeCheckpointNs()).thenReturn(checkpointTimer); + + java.util.Map taskStores = ImmutableMap.of("loggedPersistentStore", mockLPStore); + + Partition changelogPartition = new Partition(0); + SystemStream changelogSystemStream = new SystemStream("changelogSystem", "changelogStream"); + SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); + java.util.Map storeChangelogsStreams = ImmutableMap.of("loggedPersistentStore", changelogSystemStream); + + StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class); + File tmpTestPath = new File("store-checkpoint-test"); + when(storageManagerUtil.getTaskStoreDir(eq(tmpTestPath), eq("loggedPersistentStore"), any(), any())).thenReturn(tmpTestPath); + TaskName taskName = new TaskName("task"); + + when(containerStorageManager.getAllStores(taskName)).thenReturn(taskStores); + TaskStorageCommitManager commitManager = spy(new TaskStorageCommitManager(taskName, + Collections.emptyMap(), containerStorageManager, storeChangelogsStreams, changelogPartition, + null, null, ForkJoinPool.commonPool(), storageManagerUtil, tmpTestPath, metrics)); + + doAnswer(invocation -> { + String fileDir = invocation.getArgumentAt(3, File.class).getName(); + SystemStreamPartition ssp = invocation.getArgumentAt(1, SystemStreamPartition.class); + String offset = invocation.getArgumentAt(2, String.class); + if (mockFileSystem.containsKey(fileDir)) { + mockFileSystem.get(fileDir).put(ssp, offset); + } else { + Map sspOffsets = new HashMap<>(); + sspOffsets.put(ssp, offset); + mockFileSystem.put(fileDir, sspOffsets); + } + return null; + }).when(commitManager).writeChangelogOffsetFile(any(), any(), any(), any()); + + doAnswer(invocation -> { + String storeDir = invocation.getArgumentAt(0, File.class).getName(); + CheckpointV2 checkpointV2 = invocation.getArgumentAt(1, CheckpointV2.class); + mockCheckpointFileSystem.put(storeDir, checkpointV2); + return null; + }).when(storageManagerUtil).writeCheckpointV2File(any(), any()); + + CheckpointId newCheckpointId = CheckpointId.create(); + + String newestOffset = "1"; + KafkaChangelogSSPOffset kafkaChangelogSSPOffset = new KafkaChangelogSSPOffset(newCheckpointId, newestOffset); + java.util.Map offsetsJava = ImmutableMap.of( + changelogSSP, kafkaChangelogSSPOffset.toString() + ); + + commitManager.init(); + // invoke persist to file system for v1 checkpoint + commitManager.writeCheckpointToStoreDirectories(new CheckpointV1(offsetsJava)); + + assertEquals(2, mockFileSystem.size()); + // check if v2 offsets are written correctly + String v2FilePath = StorageManagerUtil + .getCheckpointDirPath(tmpTestPath, newCheckpointId); + assertTrue(mockFileSystem.containsKey(v2FilePath)); + assertTrue(mockFileSystem.get(v2FilePath).containsKey(changelogSSP)); + assertEquals(1, mockFileSystem.get(v2FilePath).size()); + assertEquals(newestOffset, mockFileSystem.get(v2FilePath).get(changelogSSP)); + // check if v1 offsets are written correctly + String v1FilePath = tmpTestPath.getPath(); + assertTrue(mockFileSystem.containsKey(v1FilePath)); + assertTrue(mockFileSystem.get(v1FilePath).containsKey(changelogSSP)); + assertEquals(1, mockFileSystem.get(v1FilePath).size()); + assertEquals(newestOffset, mockFileSystem.get(v1FilePath).get(changelogSSP)); + + java.util.Map storeSCM = ImmutableMap.of( + "loggedPersistentStore", "system;loggedPersistentStoreStream;1", + "persistentStore", "system;persistentStoreStream;1", + "loggedInMemStore", "system;loggedInMemStoreStream;1", + "inMemStore", "system;inMemStoreStream;1" + ); + CheckpointV2 checkpoint = new CheckpointV2(newCheckpointId, Collections.emptyMap(), Collections.singletonMap("factory", storeSCM)); + + // invoke persist to file system with checkpoint v2 + commitManager.writeCheckpointToStoreDirectories(checkpoint); + + assertTrue(mockCheckpointFileSystem.containsKey(v2FilePath)); + assertEquals(checkpoint, mockCheckpointFileSystem.get(v2FilePath)); + assertTrue(mockCheckpointFileSystem.containsKey(v1FilePath)); + assertEquals(checkpoint, mockCheckpointFileSystem.get(v1FilePath)); + assertEquals(2, mockCheckpointFileSystem.size()); + + CheckpointV2 updatedCheckpoint = new CheckpointV2( + newCheckpointId, ImmutableMap.of( + new SystemStreamPartition("inputSystem", "inputStream", changelogPartition), "5"), + Collections.singletonMap("factory", storeSCM)); + commitManager.writeCheckpointToStoreDirectories(updatedCheckpoint); + + assertEquals(updatedCheckpoint, mockCheckpointFileSystem.get(v2FilePath)); + assertEquals(updatedCheckpoint, mockCheckpointFileSystem.get(v1FilePath)); + assertEquals(2, mockCheckpointFileSystem.size()); + } + + @Test + public void testWriteChangelogOffsetFilesWithEmptyChangelogTopic() throws IOException { + Map> mockFileSystem = new HashMap<>(); + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + StorageEngine mockLPStore = mock(StorageEngine.class); + StoreProperties lpStoreProps = mock(StoreProperties.class); + when(mockLPStore.getStoreProperties()).thenReturn(lpStoreProps); + when(lpStoreProps.isPersistedToDisk()).thenReturn(true); + when(lpStoreProps.isDurableStore()).thenReturn(true); + Path mockPath = mock(Path.class); + when(mockLPStore.checkpoint(any())).thenReturn(Optional.of(mockPath)); + TaskInstanceMetrics metrics = mock(TaskInstanceMetrics.class); + Timer checkpointTimer = mock(Timer.class); + when(metrics.storeCheckpointNs()).thenReturn(checkpointTimer); + + java.util.Map taskStores = ImmutableMap.of("loggedPersistentStore", mockLPStore); + + Partition changelogPartition = new Partition(0); + SystemStream changelogSystemStream = new SystemStream("changelogSystem", "changelogStream"); + SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); + java.util.Map storeChangelogsStreams = ImmutableMap.of("loggedPersistentStore", changelogSystemStream); + + StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class); + File tmpTestPath = new File("store-checkpoint-test"); + when(storageManagerUtil.getTaskStoreDir(eq(tmpTestPath), any(), any(), any())).thenReturn(tmpTestPath); + TaskName taskName = new TaskName("task"); + + when(containerStorageManager.getAllStores(taskName)).thenReturn(taskStores); + TaskStorageCommitManager commitManager = spy(new TaskStorageCommitManager(taskName, + Collections.emptyMap(), containerStorageManager, storeChangelogsStreams, changelogPartition, + null, null, ForkJoinPool.commonPool(), storageManagerUtil, tmpTestPath, metrics)); + + doAnswer(invocation -> { + String storeName = invocation.getArgumentAt(0, String.class); + String fileDir = invocation.getArgumentAt(3, File.class).getName(); + String mockKey = storeName + fileDir; + SystemStreamPartition ssp = invocation.getArgumentAt(1, SystemStreamPartition.class); + String offset = invocation.getArgumentAt(2, String.class); + if (mockFileSystem.containsKey(mockKey)) { + mockFileSystem.get(mockKey).put(ssp, offset); + } else { + Map sspOffsets = new HashMap<>(); + sspOffsets.put(ssp, offset); + mockFileSystem.put(mockKey, sspOffsets); + } + return null; + }).when(commitManager).writeChangelogOffsetFile(any(), any(), any(), any()); + + CheckpointId newCheckpointId = CheckpointId.create(); + + String newestOffset = null; + KafkaChangelogSSPOffset kafkaChangelogSSPOffset = new KafkaChangelogSSPOffset(newCheckpointId, newestOffset); + java.util.Map offsetsJava = ImmutableMap.of( + changelogSSP, kafkaChangelogSSPOffset.toString() + ); + + commitManager.init(); + // invoke persist to file system for v2 checkpoint + commitManager.writeCheckpointToStoreDirectories(new CheckpointV1(offsetsJava)); + assertTrue(mockFileSystem.isEmpty()); + // verify that delete was called on current store dir offset file + verify(storageManagerUtil, times(1)).deleteOffsetFile(eq(tmpTestPath)); + } + + @Test(expected = SamzaException.class) + public void testThrowOnWriteCheckpointDirIfUnsuccessful() { + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + StorageEngine mockLPStore = mock(StorageEngine.class); + StoreProperties lpStoreProps = mock(StoreProperties.class); + when(mockLPStore.getStoreProperties()).thenReturn(lpStoreProps); + when(lpStoreProps.isPersistedToDisk()).thenReturn(true); + when(lpStoreProps.isDurableStore()).thenReturn(true); + Path mockPath = mock(Path.class); + when(mockLPStore.checkpoint(any())).thenReturn(Optional.of(mockPath)); + TaskInstanceMetrics metrics = mock(TaskInstanceMetrics.class); + Timer checkpointTimer = mock(Timer.class); + when(metrics.storeCheckpointNs()).thenReturn(checkpointTimer); + + java.util.Map taskStores = ImmutableMap.of("loggedPersistentStore", mockLPStore); + + java.util.Map storeChangelogsStreams = ImmutableMap.of("loggedPersistentStore", mock(SystemStream.class)); + + StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class); + File tmpTestPath = new File("store-checkpoint-test"); + when(storageManagerUtil.getTaskStoreDir(eq(tmpTestPath), eq("loggedPersistentStore"), any(), any())).thenReturn(tmpTestPath); + + TaskName taskName = new TaskName("task"); + + TaskStorageCommitManager commitManager = spy(new TaskStorageCommitManager(taskName, + Collections.emptyMap(), containerStorageManager, storeChangelogsStreams, mock(Partition.class), + null, null, ForkJoinPool.commonPool(), storageManagerUtil, tmpTestPath, metrics)); + + java.util.Map storeSCM = ImmutableMap.of( + "loggedPersistentStore", "system;loggedPersistentStoreStream;1", + "persistentStore", "system;persistentStoreStream;1", + "loggedInMemStore", "system;loggedInMemStoreStream;1", + "inMemStore", "system;inMemStoreStream;1" + ); + when(containerStorageManager.getAllStores(taskName)).thenReturn(taskStores); + CheckpointV2 checkpoint = new CheckpointV2(CheckpointId.create(), Collections.emptyMap(), Collections.singletonMap("factory", storeSCM)); + doThrow(IOException.class).when(storageManagerUtil).writeCheckpointV2File(eq(tmpTestPath), eq(checkpoint)); + + commitManager.init(); + // Should throw samza exception since writeCheckpointV2 failed + commitManager.writeCheckpointToStoreDirectories(checkpoint); + } + + @Test + public void testRemoveOldCheckpointsWhenBaseDirContainsRegularFiles() { + ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); + CheckpointManager checkpointManager = mock(CheckpointManager.class); + TaskBackupManager taskBackupManager1 = mock(TaskBackupManager.class); + TaskBackupManager taskBackupManager2 = mock(TaskBackupManager.class); + File durableStoreDir = mock(File.class); + TaskInstanceMetrics metrics = mock(TaskInstanceMetrics.class); + Timer checkpointTimer = mock(Timer.class); + when(metrics.storeCheckpointNs()).thenReturn(checkpointTimer); + + StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class); + + TaskName taskName = new TaskName("task1"); + Map backupManagers = ImmutableMap.of( + "factory1", taskBackupManager1, + "factory2", taskBackupManager2 + ); + + when(containerStorageManager.getAllStores(taskName)).thenReturn(Collections.emptyMap()); + TaskStorageCommitManager cm = new TaskStorageCommitManager(taskName, backupManagers, containerStorageManager, + Collections.emptyMap(), new Partition(1), checkpointManager, new MapConfig(), + ForkJoinPool.commonPool(), storageManagerUtil, durableStoreDir, metrics); + + + File mockStoreDir = mock(File.class); + String mockStoreDirName = "notDirectory"; + when(durableStoreDir.listFiles()).thenReturn(new File[] {mockStoreDir}); + when(mockStoreDir.getName()).thenReturn(mockStoreDirName); + when(storageManagerUtil.getTaskStoreDir(eq(durableStoreDir), eq(mockStoreDirName), eq(taskName), eq(TaskMode.Active))).thenReturn(mockStoreDir); + // null here can happen if listFiles is called on a non-directory + when(mockStoreDir.listFiles(any(FileFilter.class))).thenReturn(null); + + cm.cleanUp(CheckpointId.create(), new HashMap<>()).join(); + verify(durableStoreDir).listFiles(); + verify(mockStoreDir).listFiles(any(FileFilter.class)); + verify(storageManagerUtil).getTaskStoreDir(eq(durableStoreDir), eq(mockStoreDirName), eq(taskName), eq(TaskMode.Active)); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/util/TestFutureUtil.java b/samza-core/src/test/java/org/apache/samza/util/TestFutureUtil.java new file mode 100644 index 0000000000..815eb34a71 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/util/TestFutureUtil.java @@ -0,0 +1,222 @@ +/* + * 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.samza.util; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.CompletionStage; +import java.util.function.Predicate; +import org.apache.samza.SamzaException; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class TestFutureUtil { + + /** + * Test all futures in all collections complete before allOf completes. + * Test completes exceptionally if any complete exceptionally. + * Test works with heterogeneous value types. + * Test works with heterogeneous collection types. + * Test works with completion stages as well as completable futures. + */ + @Test + public void testAllOf() { + // verify that there is no short circuiting + CompletableFuture future1 = new CompletableFuture<>(); + CompletableFuture future2 = new CompletableFuture<>(); + CompletableFuture future3 = new CompletableFuture<>(); + CompletableFuture future4 = new CompletableFuture<>(); + ImmutableList> collection1 = + ImmutableList.of(future1, future2); + ImmutableSet> collection2 = + ImmutableSet.of(future3, future4); + + CompletableFuture allFuture = FutureUtil.allOf(collection1, collection2); + future1.complete("1"); + assertFalse(allFuture.isDone()); + RuntimeException ex2 = new RuntimeException("2"); + future2.completeExceptionally(ex2); + assertFalse(allFuture.isDone()); + assertFalse(allFuture.isCompletedExceptionally()); + future3.complete("3"); + assertFalse(allFuture.isDone()); + assertFalse(allFuture.isCompletedExceptionally()); + future4.complete(4); + assertTrue(allFuture.isDone()); + assertTrue(allFuture.isCompletedExceptionally()); + + try { + allFuture.join(); + } catch (Exception e) { + assertEquals(ex2, FutureUtil.unwrapExceptions(CompletionException.class, e)); + } + } + + @Test + public void testAllOfIgnoringErrorsCompletesSuccessfullyIfNoErrors() { + CompletableFuture future1 = new CompletableFuture<>(); + CompletableFuture future2 = new CompletableFuture<>(); + + CompletableFuture allFuture = FutureUtil.allOf(t -> false, future1, future2); + future1.complete("1"); + assertFalse(allFuture.isDone()); + future2.complete("2"); + assertTrue(allFuture.isDone()); + assertFalse(allFuture.isCompletedExceptionally()); + } + + @Test + public void testAllOfIgnoringErrorsCompletesSuccessfullyIfOnlyIgnoredErrors() { + CompletableFuture future1 = new CompletableFuture<>(); + CompletableFuture future2 = new CompletableFuture<>(); + + CompletableFuture allFuture = FutureUtil.allOf(t -> true, future1, future2); + future1.complete("1"); + assertFalse(allFuture.isDone()); + RuntimeException ex2 = new RuntimeException("2"); + future2.completeExceptionally(ex2); + assertTrue(allFuture.isDone()); + assertFalse(allFuture.isCompletedExceptionally()); + } + + @Test + public void testAllOfIgnoringErrorsCompletesExceptionallyIfNonIgnoredErrors() { + // also test that each future is checked individually + CompletableFuture future1 = new CompletableFuture<>(); + CompletableFuture future2 = new CompletableFuture<>(); + + Predicate mockPredicate = mock(Predicate.class); + when(mockPredicate.test(any())) + .thenReturn(true) + .thenReturn(false); + CompletableFuture allFuture = FutureUtil.allOf(mockPredicate, future1, future2); + future1.completeExceptionally(new SamzaException()); + assertFalse(allFuture.isDone()); + RuntimeException ex2 = new RuntimeException("2"); + future2.completeExceptionally(ex2); + assertTrue(allFuture.isDone()); + assertTrue(allFuture.isCompletedExceptionally()); + verify(mockPredicate, times(2)).test(any()); + } + + @Test + public void testFutureOfMapCompletesExceptionallyIfAValueFutureCompletesExceptionally() { + Map> map = new HashMap<>(); + map.put("1", CompletableFuture.completedFuture("1")); + map.put("2", FutureUtil.failedFuture(new SamzaException())); + + assertTrue(FutureUtil.toFutureOfMap(map).isCompletedExceptionally()); + } + + @Test + public void testFutureOfMapCompletesSuccessfullyIfNoErrors() { + Map> map = new HashMap<>(); + map.put("1", CompletableFuture.completedFuture("1")); + map.put("2", CompletableFuture.completedFuture("2")); + + CompletableFuture> result = FutureUtil.toFutureOfMap(t -> true, map); + assertTrue(result.isDone()); + assertFalse(result.isCompletedExceptionally()); + } + + @Test + public void testFutureOfMapCompletesSuccessfullyIfOnlyIgnoredErrors() { + Map> map = new HashMap<>(); + map.put("1", CompletableFuture.completedFuture("1")); + map.put("2", FutureUtil.failedFuture(new SamzaException())); + + CompletableFuture> result = FutureUtil + .toFutureOfMap(t -> FutureUtil.unwrapExceptions(CompletionException.class, t) instanceof SamzaException, map); + assertTrue(result.isDone()); + result.join(); + assertFalse(result.isCompletedExceptionally()); + assertEquals("1", result.join().get("1")); + assertFalse(result.join().containsKey("2")); + } + + @Test + public void testFutureOfMapCompletesExceptionallyIfAnyNonIgnoredErrors() { + Map> map = new HashMap<>(); + map.put("1", FutureUtil.failedFuture(new RuntimeException())); + SamzaException samzaException = new SamzaException(); + map.put("2", FutureUtil.failedFuture(samzaException)); + + Predicate mockPredicate = mock(Predicate.class); + when(mockPredicate.test(any())) + .thenReturn(true) + .thenReturn(false); + + CompletableFuture> result = FutureUtil.toFutureOfMap(mockPredicate, map); + assertTrue(result.isDone()); + assertTrue(result.isCompletedExceptionally()); + verify(mockPredicate, times(2)).test(any()); // verify that each failed value future is tested + + try { + result.join(); + fail("Should have thrown an exception."); + } catch (Exception e) { + assertEquals(samzaException, FutureUtil.unwrapExceptions(CompletionException.class, e)); + } + } + + @Test + public void testUnwrapExceptionUnwrapsMultipleExceptions() { + IllegalArgumentException cause = new IllegalArgumentException(); + Throwable t = new SamzaException(new SamzaException(cause)); + Throwable unwrappedThrowable = FutureUtil.unwrapExceptions(SamzaException.class, t); + assertEquals(cause, unwrappedThrowable); + } + + @Test + public void testUnwrapExceptionReturnsOriginalExceptionIfNoWrapper() { + IllegalArgumentException cause = new IllegalArgumentException(); + Throwable unwrappedThrowable = FutureUtil.unwrapExceptions(SamzaException.class, cause); + assertEquals(cause, unwrappedThrowable); + } + + @Test + public void testUnwrapExceptionReturnsNullIfNoNonWrapperCause() { + Throwable t = new SamzaException(new SamzaException()); + Throwable unwrappedThrowable = FutureUtil.unwrapExceptions(SamzaException.class, t); + assertNull(unwrappedThrowable); + } + + @Test + public void testUnwrapExceptionReturnsNullIfOriginalExceptionIsNull() { + Throwable unwrappedThrowable = FutureUtil.unwrapExceptions(SamzaException.class, null); + assertNull(unwrappedThrowable); + } +} + diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala index 7d180339eb..a86c49f458 100644 --- a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala +++ b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala @@ -308,40 +308,6 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { assertEquals(2, mockJobServlet.exceptionCount) } - @Test - def testGetChangelogSSPsForContainer() { - val taskName0 = new TaskName("task0") - val taskName1 = new TaskName("task1") - val taskModel0 = new TaskModel(taskName0, - Set(new SystemStreamPartition("input", "stream", new Partition(0))), - new Partition(10)) - val taskModel1 = new TaskModel(taskName1, - Set(new SystemStreamPartition("input", "stream", new Partition(1))), - new Partition(11)) - val containerModel = new ContainerModel("processorId", Map(taskName0 -> taskModel0, taskName1 -> taskModel1)) - val changeLogSystemStreams = Map("store0" -> new SystemStream("changelogSystem0", "store0-changelog"), - "store1" -> new SystemStream("changelogSystem1", "store1-changelog")) - val expected = Set(new SystemStreamPartition("changelogSystem0", "store0-changelog", new Partition(10)), - new SystemStreamPartition("changelogSystem1", "store1-changelog", new Partition(10)), - new SystemStreamPartition("changelogSystem0", "store0-changelog", new Partition(11)), - new SystemStreamPartition("changelogSystem1", "store1-changelog", new Partition(11))) - assertEquals(expected, SamzaContainer.getChangelogSSPsForContainer(containerModel, changeLogSystemStreams)) - } - - @Test - def testGetChangelogSSPsForContainerNoChangelogs() { - val taskName0 = new TaskName("task0") - val taskName1 = new TaskName("task1") - val taskModel0 = new TaskModel(taskName0, - Set(new SystemStreamPartition("input", "stream", new Partition(0))), - new Partition(10)) - val taskModel1 = new TaskModel(taskName1, - Set(new SystemStreamPartition("input", "stream", new Partition(1))), - new Partition(11)) - val containerModel = new ContainerModel("processorId", Map(taskName0 -> taskModel0, taskName1 -> taskModel1)) - assertEquals(Set(), SamzaContainer.getChangelogSSPsForContainer(containerModel, Map())) - } - @Test def testStoreContainerLocality():Unit = { this.config = new MapConfig(Map(ClusterManagerConfig.JOB_HOST_AFFINITY_ENABLED -> "true")) diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala index 4cab1859f6..606f86dfac 100644 --- a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala +++ b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala @@ -19,19 +19,20 @@ package org.apache.samza.container -import java.util.Collections - -import com.google.common.collect.ImmutableSet -import org.apache.samza.{Partition, SamzaException} -import org.apache.samza.checkpoint.{Checkpoint, CheckpointedChangelogOffset, OffsetManager} +import com.google.common.collect.{ImmutableMap, ImmutableSet} +import com.google.common.util.concurrent.MoreExecutors +import org.apache.samza.checkpoint._ +import org.apache.samza.checkpoint.kafka.{KafkaChangelogSSPOffset, KafkaStateCheckpointMarker} import org.apache.samza.config.MapConfig import org.apache.samza.context.{TaskContext => _, _} import org.apache.samza.job.model.TaskModel -import org.apache.samza.metrics.Counter -import org.apache.samza.storage.NonTransactionalStateTaskStorageManager +import org.apache.samza.metrics.{Counter, Gauge, Timer} +import org.apache.samza.storage.TaskStorageCommitManager import org.apache.samza.system.{IncomingMessageEnvelope, StreamMetadataCache, SystemAdmin, SystemConsumers, SystemStream, SystemStreamMetadata, _} import org.apache.samza.table.TableManager import org.apache.samza.task._ +import org.apache.samza.util.FutureUtil +import org.apache.samza.{Partition, SamzaException} import org.junit.Assert._ import org.junit.{Before, Test} import org.mockito.Matchers._ @@ -42,6 +43,10 @@ import org.mockito.{ArgumentCaptor, Matchers, Mock, MockitoAnnotations} import org.scalatest.junit.AssertionsForJUnit import org.scalatest.mockito.MockitoSugar +import java.util +import java.util.Collections +import java.util.concurrent.{CompletableFuture, ExecutorService, Executors, ForkJoinPool} +import java.util.function.Consumer import scala.collection.JavaConverters._ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { @@ -68,7 +73,9 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { @Mock private var offsetManager: OffsetManager = null @Mock - private var taskStorageManager: NonTransactionalStateTaskStorageManager = null + private var taskCommitManager: TaskStorageCommitManager = null + @Mock + private var checkpointManager: CheckpointManager = null @Mock private var taskTableManager: TableManager = null // not a mock; using MockTaskInstanceExceptionHandler @@ -88,6 +95,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { private var taskInstance: TaskInstance = null + private val numCheckpointVersions = 2 // checkpoint versions count + @Before def setup(): Unit = { MockitoAnnotations.initMocks(this) @@ -98,7 +107,10 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { Matchers.eq(this.containerContext), any(), Matchers.eq(this.applicationContainerContext))) .thenReturn(this.applicationTaskContext) when(this.systemAdmins.getSystemAdmin(SYSTEM_NAME)).thenReturn(this.systemAdmin) - when(this.jobContext.getConfig).thenReturn(new MapConfig(Collections.singletonMap("task.commit.ms", "-1"))) + val taskConfigsMap = new util.HashMap[String, String]() + taskConfigsMap.put("task.commit.ms", "-1") + taskConfigsMap.put("task.commit.max.delay.ms", "100000") + when(this.jobContext.getConfig).thenReturn(new MapConfig(taskConfigsMap)) setupTaskInstance(Some(this.applicationTaskContextFactory)) } @@ -172,7 +184,6 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { taskInstance.initTask verify(this.offsetManager).setStartingOffset(TASK_NAME, SYSTEM_STREAM_PARTITION, "10") - verifyNoMoreInteractions(this.offsetManager) } @Test @@ -212,18 +223,42 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { def testCommitOrder() { val commitsCounter = mock[Counter] when(this.metrics.commits).thenReturn(commitsCounter) - val inputOffsets = new Checkpoint(Map(SYSTEM_STREAM_PARTITION -> "4").asJava) + val snapshotTimer = mock[Timer] + when(this.metrics.snapshotNs).thenReturn(snapshotTimer) + val commitTimer = mock[Timer] + when(this.metrics.commitNs).thenReturn(commitTimer) + val commitSyncTimer = mock[Timer] + when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) + val commitAsyncTimer = mock[Timer] + when(this.metrics.commitAsyncNs).thenReturn(commitAsyncTimer) + val uploadTimer = mock[Timer] + when(this.metrics.asyncUploadNs).thenReturn(uploadTimer) + val cleanUpTimer = mock[Timer] + when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) + val skippedCounter = mock[Gauge[Int]] + when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + val inputOffsets = new util.HashMap[SystemStreamPartition, String]() + inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") val changelogSSP = new SystemStreamPartition(new SystemStream(SYSTEM_NAME, "test-changelog-stream"), new Partition(0)) - val changelogOffsets = Map(changelogSSP -> Some("5")) - when(this.offsetManager.buildCheckpoint(TASK_NAME)).thenReturn(inputOffsets) - when(this.taskStorageManager.flush()).thenReturn(changelogOffsets) - doNothing().when(this.taskStorageManager).checkpoint(any(), any[Map[SystemStreamPartition, Option[String]]]) + val stateCheckpointMarkers: util.Map[String, String] = new util.HashMap[String, String]() + val stateCheckpointMarker = KafkaStateCheckpointMarker.serialize(new KafkaStateCheckpointMarker(changelogSSP, "5")) + stateCheckpointMarkers.put("storeName", stateCheckpointMarker) + when(this.offsetManager.getLastProcessedOffsets(TASK_NAME)).thenReturn(inputOffsets) + + val snapshotSCMs = ImmutableMap.of(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME, stateCheckpointMarkers) + when(this.taskCommitManager.snapshot(any())).thenReturn(snapshotSCMs) + val snapshotSCMFuture: CompletableFuture[util.Map[String, util.Map[String, String]]] = + CompletableFuture.completedFuture(snapshotSCMs) + when(this.taskCommitManager.upload(any(), Matchers.eq(snapshotSCMs))).thenReturn(snapshotSCMFuture) // kafka is no-op + when(this.taskCommitManager.cleanUp(any(), any())).thenReturn(CompletableFuture.completedFuture[Void](null)) + + taskInstance.commit - val mockOrder = inOrder(this.offsetManager, this.collector, this.taskTableManager, this.taskStorageManager) + val mockOrder = inOrder(this.offsetManager, this.collector, this.taskTableManager, this.taskCommitManager) // We must first get a snapshot of the input offsets so it doesn't change while we flush. SAMZA-1384 - mockOrder.verify(this.offsetManager).buildCheckpoint(TASK_NAME) + mockOrder.verify(this.offsetManager).getLastProcessedOffsets(TASK_NAME) // Producers must be flushed next and ideally the output would be flushed before the changelog // s.t. the changelog and checkpoints (state and inputs) are captured last @@ -233,128 +268,671 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { mockOrder.verify(this.taskTableManager).flush() // Local state should be flushed next next - mockOrder.verify(this.taskStorageManager).flush() + mockOrder.verify(this.taskCommitManager).snapshot(any()) + + // Upload should be called next with the snapshot SCMs. + mockOrder.verify(this.taskCommitManager).upload(any(), Matchers.eq(snapshotSCMs)) // Stores checkpoints should be created next with the newest changelog offsets - mockOrder.verify(this.taskStorageManager).checkpoint(any(), Matchers.eq(changelogOffsets)) + mockOrder.verify(this.taskCommitManager).writeCheckpointToStoreDirectories(any()) // Input checkpoint should be written with the snapshot captured at the beginning of commit and the // newest changelog offset captured during storage manager flush val captor = ArgumentCaptor.forClass(classOf[Checkpoint]) - mockOrder.verify(offsetManager).writeCheckpoint(any(), captor.capture) - val cp = captor.getValue - assertEquals("4", cp.getOffsets.get(SYSTEM_STREAM_PARTITION)) - assertEquals("5", CheckpointedChangelogOffset.fromString(cp.getOffsets.get(changelogSSP)).getOffset) + mockOrder.verify(offsetManager, times(numCheckpointVersions)).writeCheckpoint(any(), captor.capture) + val cp = captor.getAllValues + assertEquals(numCheckpointVersions, cp.size()) + cp.forEach(new Consumer[Checkpoint] { + override def accept(c: Checkpoint): Unit = { + assertEquals("4", c.getOffsets.get(SYSTEM_STREAM_PARTITION)) + if (c.getVersion == 2) { + assertEquals(1, c.getOffsets.size()) + assertTrue(c.isInstanceOf[CheckpointV2]) + val checkpointedStateCheckpointMarkers = c.asInstanceOf[CheckpointV2] + .getStateCheckpointMarkers.get(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME) + assertTrue(checkpointedStateCheckpointMarkers.size() == 1) + val checkpointedStateCheckpointMarker = checkpointedStateCheckpointMarkers.get("storeName") + assertTrue(checkpointedStateCheckpointMarker.equals(stateCheckpointMarker)) + val kafkaMarker = KafkaStateCheckpointMarker.deserialize(checkpointedStateCheckpointMarker) + assertEquals(kafkaMarker.getChangelogOffset, "5") + assertEquals(kafkaMarker.getChangelogSSP, changelogSSP) + } else { // c.getVersion == 1 + assertEquals(2, c.getOffsets.size()) + assertTrue(c.isInstanceOf[CheckpointV1]) + assertEquals("5", KafkaChangelogSSPOffset.fromString(c.getOffsets.get(changelogSSP)).getChangelogOffset) + } + } + }) // Old checkpointed stores should be cleared - mockOrder.verify(this.taskStorageManager).removeOldCheckpoints(any()) + mockOrder.verify(this.taskCommitManager).cleanUp(any(), any()) verify(commitsCounter).inc() + verify(snapshotTimer).update(anyLong()) + verify(uploadTimer).update(anyLong()) + verify(commitTimer).update(anyLong()) } @Test def testEmptyChangelogSSPOffsetInCommit() { // e.g. if changelog topic is empty val commitsCounter = mock[Counter] when(this.metrics.commits).thenReturn(commitsCounter) + val snapshotTimer = mock[Timer] + when(this.metrics.snapshotNs).thenReturn(snapshotTimer) + val commitTimer = mock[Timer] + when(this.metrics.commitNs).thenReturn(commitTimer) + val commitSyncTimer = mock[Timer] + when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) + val commitAsyncTimer = mock[Timer] + when(this.metrics.commitAsyncNs).thenReturn(commitAsyncTimer) + val uploadTimer = mock[Timer] + when(this.metrics.asyncUploadNs).thenReturn(uploadTimer) + val skippedCounter = mock[Gauge[Int]] + when(this.metrics.commitsSkipped).thenReturn(skippedCounter) - val inputOffsets = new Checkpoint(Map(SYSTEM_STREAM_PARTITION -> "4").asJava) + val inputOffsets = Map(SYSTEM_STREAM_PARTITION -> "4").asJava val changelogSSP = new SystemStreamPartition(new SystemStream(SYSTEM_NAME, "test-changelog-stream"), new Partition(0)) - val changelogOffsets = Map(changelogSSP -> None) - when(this.offsetManager.buildCheckpoint(TASK_NAME)).thenReturn(inputOffsets) - when(this.taskStorageManager.flush()).thenReturn(changelogOffsets) + val stateCheckpointMarkers: util.Map[String, String] = new util.HashMap[String, String]() + val nullStateCheckpointMarker = KafkaStateCheckpointMarker.serialize(new KafkaStateCheckpointMarker(changelogSSP, null)) + stateCheckpointMarkers.put("storeName", nullStateCheckpointMarker) + when(this.offsetManager.getLastProcessedOffsets(TASK_NAME)).thenReturn(inputOffsets) + when(this.taskCommitManager.upload(any(), any())) + .thenReturn(CompletableFuture.completedFuture( + Collections.singletonMap(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME, stateCheckpointMarkers))) taskInstance.commit val captor = ArgumentCaptor.forClass(classOf[Checkpoint]) - verify(offsetManager).writeCheckpoint(any(), captor.capture) - val cp = captor.getValue - assertEquals("4", cp.getOffsets.get(SYSTEM_STREAM_PARTITION)) - val message = cp.getOffsets.get(changelogSSP) - val checkpointedOffset = CheckpointedChangelogOffset.fromString(message) - assertNull(checkpointedOffset.getOffset) - assertNotNull(checkpointedOffset.getCheckpointId) + verify(offsetManager, times(numCheckpointVersions)).writeCheckpoint(any(), captor.capture) + val cp = captor.getAllValues + assertEquals(numCheckpointVersions, cp.size()) + cp.forEach(new Consumer[Checkpoint] { + override def accept(checkpoint: Checkpoint): Unit = { + assertEquals("4", checkpoint.getOffsets.get(SYSTEM_STREAM_PARTITION)) + if (checkpoint.getVersion == 2) { + assertEquals(1, checkpoint.getOffsets.size()) + assertTrue(checkpoint.isInstanceOf[CheckpointV2]) + val checkpointedStateCheckpointMarkers = checkpoint.asInstanceOf[CheckpointV2] + .getStateCheckpointMarkers.get(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME) + assertTrue(checkpointedStateCheckpointMarkers.size() == 1) + val checkpointedStateCheckpointMarker = checkpointedStateCheckpointMarkers.get("storeName") + assertTrue(checkpointedStateCheckpointMarker.equals(nullStateCheckpointMarker)) + val kafkaMarker = KafkaStateCheckpointMarker.deserialize(checkpointedStateCheckpointMarker) + assertNull(kafkaMarker.getChangelogOffset) + assertEquals(kafkaMarker.getChangelogSSP, changelogSSP) + } else { // c.getVersion == 1 + assertEquals(2, checkpoint.getOffsets.size()) + assertTrue(checkpoint.isInstanceOf[CheckpointV1]) + val message = checkpoint.getOffsets.get(changelogSSP) + val checkpointedOffset = KafkaChangelogSSPOffset.fromString(message) + assertNull(checkpointedOffset.getChangelogOffset) + assertNotNull(checkpointedOffset.getCheckpointId) + } + } + }) verify(commitsCounter).inc() + verify(snapshotTimer).update(anyLong()) + verify(uploadTimer).update(anyLong()) } @Test def testEmptyChangelogOffsetsInCommit() { // e.g. if stores have no changelogs val commitsCounter = mock[Counter] when(this.metrics.commits).thenReturn(commitsCounter) + val snapshotTimer = mock[Timer] + when(this.metrics.snapshotNs).thenReturn(snapshotTimer) + val commitTimer = mock[Timer] + when(this.metrics.commitNs).thenReturn(commitTimer) + val commitSyncTimer = mock[Timer] + when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) + val commitAsyncTimer = mock[Timer] + when(this.metrics.commitAsyncNs).thenReturn(commitAsyncTimer) + val uploadTimer = mock[Timer] + when(this.metrics.asyncUploadNs).thenReturn(uploadTimer) + val skippedCounter = mock[Gauge[Int]] + when(this.metrics.commitsSkipped).thenReturn(skippedCounter) - val inputOffsets = new Checkpoint(Map(SYSTEM_STREAM_PARTITION -> "4").asJava) - val changelogOffsets = Map[SystemStreamPartition, Option[String]]() - when(this.offsetManager.buildCheckpoint(TASK_NAME)).thenReturn(inputOffsets) - when(this.taskStorageManager.flush()).thenReturn(changelogOffsets) + val inputOffsets = Map(SYSTEM_STREAM_PARTITION -> "4").asJava + val stateCheckpointMarkers: util.Map[String, String] = new util.HashMap[String, String]() + when(this.offsetManager.getLastProcessedOffsets(TASK_NAME)).thenReturn(inputOffsets) + when(this.taskCommitManager.upload(any(), any())) + .thenReturn(CompletableFuture.completedFuture( + Collections.singletonMap(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME, stateCheckpointMarkers))) taskInstance.commit val captor = ArgumentCaptor.forClass(classOf[Checkpoint]) - verify(offsetManager).writeCheckpoint(any(), captor.capture) - val cp = captor.getValue - assertEquals("4", cp.getOffsets.get(SYSTEM_STREAM_PARTITION)) - assertEquals(1, cp.getOffsets.size()) + // verify the write checkpoint is evoked twice, once per checkpoint version + verify(offsetManager, times(numCheckpointVersions)).writeCheckpoint(any(), captor.capture) + val cp = captor.getAllValues + assertEquals(numCheckpointVersions, cp.size()) + cp.forEach(new Consumer[Checkpoint] { + override def accept(c: Checkpoint): Unit = { + assertEquals("4", c.getOffsets.get(SYSTEM_STREAM_PARTITION)) + assertEquals(1, c.getOffsets.size()) + } + }) verify(commitsCounter).inc() + verify(snapshotTimer).update(anyLong()) + verify(uploadTimer).update(anyLong()) } @Test def testCommitFailsIfErrorGettingChangelogOffset() { // required for transactional state val commitsCounter = mock[Counter] when(this.metrics.commits).thenReturn(commitsCounter) + val snapshotTimer = mock[Timer] + when(this.metrics.snapshotNs).thenReturn(snapshotTimer) - val inputOffsets = new Checkpoint(Map(SYSTEM_STREAM_PARTITION -> "4").asJava) - when(this.offsetManager.buildCheckpoint(TASK_NAME)).thenReturn(inputOffsets) - when(this.taskStorageManager.flush()).thenThrow(new SamzaException("Error getting changelog offsets")) + val inputOffsets = new util.HashMap[SystemStreamPartition, String]() + inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") + when(this.offsetManager.getLastProcessedOffsets(TASK_NAME)).thenReturn(inputOffsets) + when(this.taskCommitManager.snapshot(any())).thenThrow(new SamzaException("Error getting changelog offsets")) try { + // sync stage exception should be caught and rethrown immediately taskInstance.commit + + verify(commitsCounter).inc() + verifyZeroInteractions(snapshotTimer) } catch { case e: SamzaException => + val msg = e.getMessage // exception is expected, container should fail if could not get changelog offsets. return } - fail("Should have failed commit if error getting newest changelog offests") + fail("Should have failed commit if error getting newest changelog offsets") } @Test - def testCommitFailsIfErrorCreatingStoreCheckpoints() { // required for transactional state + def testCommitFailsIfPreviousAsyncUploadFails() { val commitsCounter = mock[Counter] when(this.metrics.commits).thenReturn(commitsCounter) + val snapshotTimer = mock[Timer] + when(this.metrics.snapshotNs).thenReturn(snapshotTimer) + val commitTimer = mock[Timer] + when(this.metrics.commitNs).thenReturn(commitTimer) + val commitSyncTimer = mock[Timer] + when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) + val commitAsyncTimer = mock[Timer] + when(this.metrics.commitAsyncNs).thenReturn(commitAsyncTimer) + val uploadTimer = mock[Timer] + when(this.metrics.asyncUploadNs).thenReturn(uploadTimer) + val cleanUpTimer = mock[Timer] + when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) + val skippedCounter = mock[Gauge[Int]] + when(this.metrics.commitsSkipped).thenReturn(skippedCounter) - val inputOffsets = new Checkpoint(Map(SYSTEM_STREAM_PARTITION -> "4").asJava) - when(this.offsetManager.buildCheckpoint(TASK_NAME)).thenReturn(inputOffsets) - when(this.taskStorageManager.flush()).thenReturn(Map[SystemStreamPartition, Option[String]]()) - when(this.taskStorageManager.checkpoint(any(), any())).thenThrow(new SamzaException("Error creating store checkpoint")) + val inputOffsets = new util.HashMap[SystemStreamPartition, String]() + inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") + val stateCheckpointMarkers: util.Map[String, String] = new util.HashMap[String, String]() + when(this.offsetManager.getLastProcessedOffsets(TASK_NAME)).thenReturn(inputOffsets) + when(this.taskCommitManager.upload(any(), any())) + .thenReturn(CompletableFuture.completedFuture( + Collections.singletonMap(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME, stateCheckpointMarkers))) + when(this.taskCommitManager.upload(any(), any())) + .thenReturn(FutureUtil.failedFuture[util.Map[String, util.Map[String, String]]](new RuntimeException)) try { taskInstance.commit + + verify(commitsCounter).inc() + verify(snapshotTimer).update(anyLong()) + verifyZeroInteractions(uploadTimer) + verifyZeroInteractions(commitTimer) + verifyZeroInteractions(skippedCounter) + + // async stage exception in first commit should be caught and rethrown by the subsequent commit + taskInstance.commit + verifyNoMoreInteractions(commitsCounter) } catch { case e: SamzaException => - // exception is expected, container should fail if could not get changelog offsets. + // exception is expected, container should fail if could not upload previous snapshot. return } - fail("Should have failed commit if error getting newest changelog offests") + fail("Should have failed commit if error uploading store contents") } @Test - def testCommitContinuesIfErrorClearingOldCheckpoints() { // required for transactional state + def testCommitFailsIfAsyncStoreDirCheckpointWriteFails() { // required for transactional state val commitsCounter = mock[Counter] when(this.metrics.commits).thenReturn(commitsCounter) + val snapshotTimer = mock[Timer] + when(this.metrics.snapshotNs).thenReturn(snapshotTimer) + val commitTimer = mock[Timer] + when(this.metrics.commitNs).thenReturn(commitTimer) + val commitSyncTimer = mock[Timer] + when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) + val commitAsyncTimer = mock[Timer] + when(this.metrics.commitAsyncNs).thenReturn(commitAsyncTimer) + val uploadTimer = mock[Timer] + when(this.metrics.asyncUploadNs).thenReturn(uploadTimer) + val cleanUpTimer = mock[Timer] + when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) + val skippedCounter = mock[Gauge[Int]] + when(this.metrics.commitsSkipped).thenReturn(skippedCounter) - val inputOffsets = new Checkpoint(Map(SYSTEM_STREAM_PARTITION -> "4").asJava) - when(this.offsetManager.buildCheckpoint(TASK_NAME)).thenReturn(inputOffsets) - when(this.taskStorageManager.flush()).thenReturn(Map[SystemStreamPartition, Option[String]]()) - doNothing().when(this.taskStorageManager).checkpoint(any(), any()) - when(this.taskStorageManager.removeOldCheckpoints(any())) - .thenThrow(new SamzaException("Error clearing old checkpoints")) + val inputOffsets = new util.HashMap[SystemStreamPartition, String]() + inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") + val stateCheckpointMarkers: util.Map[String, String] = new util.HashMap[String, String]() + when(this.offsetManager.getLastProcessedOffsets(TASK_NAME)).thenReturn(inputOffsets) + when(this.taskCommitManager.upload(any(), any())) + .thenReturn(CompletableFuture.completedFuture( + Collections.singletonMap(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME, stateCheckpointMarkers))) + when(this.taskCommitManager.writeCheckpointToStoreDirectories(any())) + .thenThrow(new SamzaException("Error creating store checkpoint")) try { taskInstance.commit + + verify(commitsCounter).inc() + verify(snapshotTimer).update(anyLong()) + verify(uploadTimer).update(anyLong()) + verifyZeroInteractions(commitTimer) + verifyZeroInteractions(skippedCounter) + + // async stage exception in first commit should be caught and rethrown by the subsequent commit + taskInstance.commit + verifyNoMoreInteractions(commitsCounter) } catch { case e: SamzaException => // exception is expected, container should fail if could not get changelog offsets. - fail("Exception from removeOldCheckpoints should have been caught") + return + } + + fail("Should have failed commit if error writing checkpoint to store dirs") + } + + @Test + def testCommitFailsIfPreviousAsyncCheckpointTopicWriteFails() { + val commitsCounter = mock[Counter] + when(this.metrics.commits).thenReturn(commitsCounter) + val snapshotTimer = mock[Timer] + when(this.metrics.snapshotNs).thenReturn(snapshotTimer) + val commitTimer = mock[Timer] + when(this.metrics.commitNs).thenReturn(commitTimer) + val commitSyncTimer = mock[Timer] + when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) + val commitAsyncTimer = mock[Timer] + when(this.metrics.commitAsyncNs).thenReturn(commitAsyncTimer) + val uploadTimer = mock[Timer] + when(this.metrics.asyncUploadNs).thenReturn(uploadTimer) + val cleanUpTimer = mock[Timer] + when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) + val skippedCounter = mock[Gauge[Int]] + when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + + val inputOffsets = new util.HashMap[SystemStreamPartition, String]() + inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") + val stateCheckpointMarkers: util.Map[String, String] = new util.HashMap[String, String]() + when(this.offsetManager.getLastProcessedOffsets(TASK_NAME)).thenReturn(inputOffsets) + when(this.taskCommitManager.upload(any(), any())) + .thenReturn(CompletableFuture.completedFuture( + Collections.singletonMap(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME, stateCheckpointMarkers))) + doNothing().when(this.taskCommitManager).writeCheckpointToStoreDirectories(any()) + when(this.offsetManager.writeCheckpoint(any(), any())) + .thenThrow(new SamzaException("Error writing checkpoint")) + + try { + taskInstance.commit + + verify(commitsCounter).inc() + verify(snapshotTimer).update(anyLong()) + verify(uploadTimer).update(anyLong()) + verifyZeroInteractions(commitTimer) + verifyZeroInteractions(skippedCounter) + + // async stage exception in first commit should be caught and rethrown by the subsequent commit + taskInstance.commit + verifyNoMoreInteractions(commitsCounter) + } catch { + case e: SamzaException => + // exception is expected, container should fail if could not write previous checkpoint. + return + } + + fail("Should have failed commit if error writing checkpoints to checkpoint topic") + } + + @Test + def testCommitFailsIfPreviousAsyncCleanUpFails() { // required for blob store backend + val commitsCounter = mock[Counter] + when(this.metrics.commits).thenReturn(commitsCounter) + val snapshotTimer = mock[Timer] + when(this.metrics.snapshotNs).thenReturn(snapshotTimer) + val commitTimer = mock[Timer] + when(this.metrics.commitNs).thenReturn(commitTimer) + val commitSyncTimer = mock[Timer] + when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) + val commitAsyncTimer = mock[Timer] + when(this.metrics.commitAsyncNs).thenReturn(commitAsyncTimer) + val uploadTimer = mock[Timer] + when(this.metrics.asyncUploadNs).thenReturn(uploadTimer) + val cleanUpTimer = mock[Timer] + when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) + val skippedCounter = mock[Gauge[Int]] + when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + + val inputOffsets = new util.HashMap[SystemStreamPartition, String]() + inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") + val stateCheckpointMarkers: util.Map[String, String] = new util.HashMap[String, String]() + when(this.offsetManager.getLastProcessedOffsets(TASK_NAME)).thenReturn(inputOffsets) + when(this.taskCommitManager.upload(any(), any())) + .thenReturn(CompletableFuture.completedFuture( + Collections.singletonMap(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME, stateCheckpointMarkers))) + doNothing().when(this.taskCommitManager).writeCheckpointToStoreDirectories(any()) + when(this.taskCommitManager.cleanUp(any(), any())) + .thenReturn(FutureUtil.failedFuture[Void](new SamzaException("Error during cleanup"))) + + try { + taskInstance.commit + + verify(commitsCounter).inc() + verify(snapshotTimer).update(anyLong()) + verify(uploadTimer).update(anyLong()) + verifyZeroInteractions(commitTimer) + verifyZeroInteractions(skippedCounter) + + // async stage exception in first commit should be caught and rethrown by the subsequent commit + taskInstance.commit + verifyNoMoreInteractions(commitsCounter) + } catch { + case e: SamzaException => + // exception is expected, container should fail if could not clean up old checkpoint. + return + } + + fail("Should have failed commit if error cleaning up previous commit") + } + + @Test + def testCommitFailsIfPreviousAsyncUploadFailsSynchronously() { + val commitsCounter = mock[Counter] + when(this.metrics.commits).thenReturn(commitsCounter) + val snapshotTimer = mock[Timer] + when(this.metrics.snapshotNs).thenReturn(snapshotTimer) + val commitTimer = mock[Timer] + when(this.metrics.commitNs).thenReturn(commitTimer) + val commitSyncTimer = mock[Timer] + when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) + val commitAsyncTimer = mock[Timer] + when(this.metrics.commitAsyncNs).thenReturn(commitAsyncTimer) + val uploadTimer = mock[Timer] + when(this.metrics.asyncUploadNs).thenReturn(uploadTimer) + val cleanUpTimer = mock[Timer] + when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) + val skippedCounter = mock[Gauge[Int]] + when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + + val inputOffsets = new util.HashMap[SystemStreamPartition, String]() + inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") + val stateCheckpointMarkers: util.Map[String, String] = new util.HashMap[String, String]() + when(this.offsetManager.getLastProcessedOffsets(TASK_NAME)).thenReturn(inputOffsets) + when(this.taskCommitManager.upload(any(), any())) + .thenReturn(CompletableFuture.completedFuture( + Collections.singletonMap(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME, stateCheckpointMarkers))) + + // Fail synchronously instead of returning a failed future. + when(this.taskCommitManager.upload(any(), any())) + .thenThrow(new RuntimeException) + + try { + taskInstance.commit + + verify(commitsCounter).inc() + verify(snapshotTimer).update(anyLong()) + verifyZeroInteractions(uploadTimer) + verifyZeroInteractions(commitTimer) + verifyZeroInteractions(skippedCounter) + + // async stage exception in first commit should be caught and rethrown by the subsequent commit + taskInstance.commit + verifyNoMoreInteractions(commitsCounter) + } catch { + case e: SamzaException => + // exception is expected, container should fail if could not upload previous snapshot. + return + } + + fail("Should have failed commit if synchronous error during upload in async stage of previous commit") + } + + @Test + def testCommitSucceedsIfPreviousAsyncStageSucceeds() { + val commitsCounter = mock[Counter] + when(this.metrics.commits).thenReturn(commitsCounter) + val snapshotTimer = mock[Timer] + when(this.metrics.snapshotNs).thenReturn(snapshotTimer) + val commitTimer = mock[Timer] + when(this.metrics.commitNs).thenReturn(commitTimer) + val commitSyncTimer = mock[Timer] + when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) + val commitAsyncTimer = mock[Timer] + when(this.metrics.commitAsyncNs).thenReturn(commitAsyncTimer) + val uploadTimer = mock[Timer] + when(this.metrics.asyncUploadNs).thenReturn(uploadTimer) + val cleanUpTimer = mock[Timer] + when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) + val skippedCounter = mock[Gauge[Int]] + when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + + val inputOffsets = new util.HashMap[SystemStreamPartition, String]() + inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") + val stateCheckpointMarkers: util.Map[String, String] = new util.HashMap[String, String]() + when(this.offsetManager.getLastProcessedOffsets(TASK_NAME)).thenReturn(inputOffsets) + when(this.taskCommitManager.upload(any(), any())) + .thenReturn(CompletableFuture.completedFuture( + Collections.singletonMap(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME, stateCheckpointMarkers))) + doNothing().when(this.taskCommitManager).writeCheckpointToStoreDirectories(any()) + when(this.taskCommitManager.cleanUp(any(), any())) + .thenReturn(CompletableFuture.completedFuture[Void](null)) + + taskInstance.commit // async stage will be run by caller due to direct executor + + verify(commitsCounter).inc() + verify(snapshotTimer).update(anyLong()) + verify(uploadTimer).update(anyLong()) + verify(commitTimer).update(anyLong()) + + taskInstance.commit + + // verify that all commit operations ran twice + verify(taskCommitManager, times(2)).snapshot(any()) + verify(taskCommitManager, times(2)).upload(any(), any()) + // called 2x per commit, once for each checkpoint version + verify(taskCommitManager, times(4)).writeCheckpointToStoreDirectories(any()) + verify(offsetManager, times(4)).writeCheckpoint(any(), any()) + verify(taskCommitManager, times(2)).cleanUp(any(), any()) + verify(commitsCounter, times(2)).inc() + } + + @Test + def testCommitSkipsIfPreviousAsyncCommitInProgressWithinMaxCommitDelay() { + val commitsCounter = mock[Counter] + when(this.metrics.commits).thenReturn(commitsCounter) + val snapshotTimer = mock[Timer] + when(this.metrics.snapshotNs).thenReturn(snapshotTimer) + val commitTimer = mock[Timer] + when(this.metrics.commitNs).thenReturn(commitTimer) + val commitSyncTimer = mock[Timer] + when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) + val commitAsyncTimer = mock[Timer] + when(this.metrics.commitAsyncNs).thenReturn(commitAsyncTimer) + val uploadTimer = mock[Timer] + when(this.metrics.asyncUploadNs).thenReturn(uploadTimer) + val cleanUpTimer = mock[Timer] + when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) + val skippedCounter = mock[Gauge[Int]] + when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + + val inputOffsets = new util.HashMap[SystemStreamPartition, String]() + inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") + val changelogSSP = new SystemStreamPartition(new SystemStream(SYSTEM_NAME, "test-changelog-stream"), new Partition(0)) + + val stateCheckpointMarkers: util.Map[String, String] = new util.HashMap[String, String]() + val stateCheckpointMarker = KafkaStateCheckpointMarker.serialize(new KafkaStateCheckpointMarker(changelogSSP, "5")) + stateCheckpointMarkers.put("storeName", stateCheckpointMarker) + when(this.offsetManager.getLastProcessedOffsets(TASK_NAME)).thenReturn(inputOffsets) + + val snapshotSCMs = ImmutableMap.of(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME, stateCheckpointMarkers) + when(this.taskCommitManager.snapshot(any())).thenReturn(snapshotSCMs) + val snapshotSCMFuture: CompletableFuture[util.Map[String, util.Map[String, String]]] = + CompletableFuture.completedFuture(snapshotSCMs) + + when(this.taskCommitManager.upload(any(), Matchers.eq(snapshotSCMs))).thenReturn(snapshotSCMFuture) // kafka is no-op + + val cleanUpFuture = new CompletableFuture[Void]() // not completed until subsequent commit starts + when(this.taskCommitManager.cleanUp(any(), any())).thenReturn(cleanUpFuture) + + // use a separate executor to perform async operations on to test caller thread blocking behavior + setupTaskInstance(None, ForkJoinPool.commonPool()) + + taskInstance.commit // async stage will not complete until cleanUpFuture is completed + + taskInstance.commit + + verify(skippedCounter).set(1) + + verify(commitsCounter, times(1)).inc() // should only have been incremented once on the initial commit + verify(snapshotTimer).update(anyLong()) + verify(uploadTimer).update(anyLong()) + verifyZeroInteractions(commitTimer) + + cleanUpFuture.complete(null) // just to unblock shared executor + } + + @Test + def testCommitThrowsIfPreviousAsyncCommitInProgressAfterMaxCommitDelayAndBlockTime() { + val commitsCounter = mock[Counter] + when(this.metrics.commits).thenReturn(commitsCounter) + val snapshotTimer = mock[Timer] + when(this.metrics.snapshotNs).thenReturn(snapshotTimer) + val commitTimer = mock[Timer] + when(this.metrics.commitNs).thenReturn(commitTimer) + val commitSyncTimer = mock[Timer] + when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) + val commitAsyncTimer = mock[Timer] + when(this.metrics.commitAsyncNs).thenReturn(commitAsyncTimer) + val uploadTimer = mock[Timer] + when(this.metrics.asyncUploadNs).thenReturn(uploadTimer) + val cleanUpTimer = mock[Timer] + when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) + val skippedCounter = mock[Gauge[Int]] + when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + + val inputOffsets = new util.HashMap[SystemStreamPartition, String]() + inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") + val changelogSSP = new SystemStreamPartition(new SystemStream(SYSTEM_NAME, "test-changelog-stream"), new Partition(0)) + + val stateCheckpointMarkers: util.Map[String, String] = new util.HashMap[String, String]() + val stateCheckpointMarker = KafkaStateCheckpointMarker.serialize(new KafkaStateCheckpointMarker(changelogSSP, "5")) + stateCheckpointMarkers.put("storeName", stateCheckpointMarker) + when(this.offsetManager.getLastProcessedOffsets(TASK_NAME)).thenReturn(inputOffsets) + + val snapshotSCMs = ImmutableMap.of(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME, stateCheckpointMarkers) + when(this.taskCommitManager.snapshot(any())).thenReturn(snapshotSCMs) + val snapshotSCMFuture: CompletableFuture[util.Map[String, util.Map[String, String]]] = + CompletableFuture.completedFuture(snapshotSCMs) + + when(this.taskCommitManager.upload(any(), Matchers.eq(snapshotSCMs))).thenReturn(snapshotSCMFuture) // kafka is no-op + + val cleanUpFuture = new CompletableFuture[Void]() + when(this.taskCommitManager.cleanUp(any(), any())).thenReturn(cleanUpFuture) + + // use a separate executor to perform async operations on to test caller thread blocking behavior + val taskConfigsMap = new util.HashMap[String, String]() + taskConfigsMap.put("task.commit.ms", "-1") + // "block" immediately if previous commit async stage not complete + taskConfigsMap.put("task.commit.max.delay.ms", "-1") + taskConfigsMap.put("task.commit.timeout.ms", "0") // throw exception immediately if blocked + when(this.jobContext.getConfig).thenReturn(new MapConfig(taskConfigsMap)) // override default behavior + + setupTaskInstance(None, ForkJoinPool.commonPool()) + + taskInstance.commit // async stage will not complete until cleanUpFuture is completed + + try { + taskInstance.commit // should throw exception + fail("Should have thrown an exception if blocked for previous commit async stage.") + } catch { + case e: Exception => + verify(commitsCounter, times(1)).inc() // should only have been incremented once on the initial commit } + + cleanUpFuture.complete(null) // just to unblock shared executor } + @Test + def testCommitBlocksIfPreviousAsyncCommitInProgressAfterMaxCommitDelayButWithinBlockTime() { + val commitsCounter = mock[Counter] + when(this.metrics.commits).thenReturn(commitsCounter) + val snapshotTimer = mock[Timer] + when(this.metrics.snapshotNs).thenReturn(snapshotTimer) + val commitTimer = mock[Timer] + when(this.metrics.commitNs).thenReturn(commitTimer) + val uploadTimer = mock[Timer] + val commitSyncTimer = mock[Timer] + when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) + val commitAsyncTimer = mock[Timer] + when(this.metrics.commitAsyncNs).thenReturn(commitAsyncTimer) + when(this.metrics.asyncUploadNs).thenReturn(uploadTimer) + val cleanUpTimer = mock[Timer] + when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) + val skippedCounter = mock[Gauge[Int]] + when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + + val inputOffsets = new util.HashMap[SystemStreamPartition, String]() + inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") + val changelogSSP = new SystemStreamPartition(new SystemStream(SYSTEM_NAME, "test-changelog-stream"), new Partition(0)) + + val stateCheckpointMarkers: util.Map[String, String] = new util.HashMap[String, String]() + val stateCheckpointMarker = KafkaStateCheckpointMarker.serialize(new KafkaStateCheckpointMarker(changelogSSP, "5")) + stateCheckpointMarkers.put("storeName", stateCheckpointMarker) + when(this.offsetManager.getLastProcessedOffsets(TASK_NAME)).thenReturn(inputOffsets) + + val snapshotSCMs = ImmutableMap.of(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME, stateCheckpointMarkers) + when(this.taskCommitManager.snapshot(any())).thenReturn(snapshotSCMs) + val snapshotSCMFuture: CompletableFuture[util.Map[String, util.Map[String, String]]] = + CompletableFuture.completedFuture(snapshotSCMs) + + when(this.taskCommitManager.upload(any(), Matchers.eq(snapshotSCMs))).thenReturn(snapshotSCMFuture) // kafka is no-op + + val cleanUpFuture = new CompletableFuture[Void]() + when(this.taskCommitManager.cleanUp(any(), any())).thenReturn(cleanUpFuture) + + // use a separate executor to perform async operations on to test caller thread blocking behavior + val taskConfigsMap = new util.HashMap[String, String]() + taskConfigsMap.put("task.commit.ms", "-1") + // "block" immediately if previous commit async stage not complete + taskConfigsMap.put("task.commit.max.delay.ms", "-1") + taskConfigsMap.put("task.commit.timeout.ms", "1000000") // block until previous stage is complete + when(this.jobContext.getConfig).thenReturn(new MapConfig(taskConfigsMap)) // override default behavior + + setupTaskInstance(None, ForkJoinPool.commonPool()) + + taskInstance.commit // async stage will not complete until cleanUpFuture is completed + + val executorService = Executors.newSingleThreadExecutor() + val secondCommitFuture = CompletableFuture.runAsync(new Runnable { + override def run(): Unit = taskInstance.commit // will block on executor + }, executorService) + + var retries = 0 // wait no more than ~100 millis + while (!taskInstance.commitInProgress.hasQueuedThreads && retries < 10) { + retries += 1 + Thread.sleep(10) // wait until commit in other thread blocks on the semaphore. + } + if (!taskInstance.commitInProgress.hasQueuedThreads) { + fail("Other thread should have blocked on semaphore acquisition. " + + "May need to increase retries if transient failure.") + } + + cleanUpFuture.complete(null) // will eventually unblock the 2nd commit in other thread. + secondCommitFuture.join() // will complete when the sync phase of 2nd commit is complete. + verify(commitsCounter, times(2)).inc() // should only have been incremented twice - once for each commit + verify(snapshotTimer, times(2)).update(anyLong()) + } + + /** * Given that no application task context factory is provided, then no lifecycle calls should be made. */ @@ -400,7 +978,7 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { this.consumerMultiplexer, this.collector, offsetManager = offsetManagerMock, - storageManager = this.taskStorageManager, + commitManager = this.taskCommitManager, tableManager = this.taskTableManager, systemStreamPartitions = ImmutableSet.of(ssp), exceptionHandler = this.taskInstanceExceptionHandler, @@ -418,7 +996,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { } private def setupTaskInstance( - applicationTaskContextFactory: Option[ApplicationTaskContextFactory[ApplicationTaskContext]]): Unit = { + applicationTaskContextFactory: Option[ApplicationTaskContextFactory[ApplicationTaskContext]], + commitThreadPool: ExecutorService = MoreExecutors.newDirectExecutorService()): Unit = { this.taskInstance = new TaskInstance(this.task, this.taskModel, this.metrics, @@ -426,10 +1005,11 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { this.consumerMultiplexer, this.collector, offsetManager = this.offsetManager, - storageManager = this.taskStorageManager, + commitManager = this.taskCommitManager, tableManager = this.taskTableManager, systemStreamPartitions = SYSTEM_STREAM_PARTITIONS, exceptionHandler = this.taskInstanceExceptionHandler, + commitThreadPool = commitThreadPool, jobContext = this.jobContext, containerContext = this.containerContext, applicationContainerContextOption = Some(this.applicationContainerContext), diff --git a/samza-core/src/test/scala/org/apache/samza/storage/TestTransactionalStateTaskStorageManager.java b/samza-core/src/test/scala/org/apache/samza/storage/TestTransactionalStateTaskStorageManager.java deleted file mode 100644 index 244a35b824..0000000000 --- a/samza-core/src/test/scala/org/apache/samza/storage/TestTransactionalStateTaskStorageManager.java +++ /dev/null @@ -1,534 +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.samza.storage; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import java.io.FileFilter; -import scala.Option; -import scala.collection.immutable.Map; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.HashMap; -import java.util.Optional; -import org.apache.samza.Partition; -import org.apache.samza.SamzaException; -import org.apache.samza.checkpoint.CheckpointId; -import org.apache.samza.container.TaskName; -import org.apache.samza.job.model.TaskMode; -import org.apache.samza.system.SystemAdmin; -import org.apache.samza.system.SystemAdmins; -import org.apache.samza.system.SystemStream; -import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata; -import org.apache.samza.system.SystemStreamPartition; -import org.apache.samza.util.ScalaJavaUtil; -import org.junit.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.InOrder; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyBoolean; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.inOrder; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -public class TestTransactionalStateTaskStorageManager { - @Test - public void testFlushOrder() { - ContainerStorageManager csm = mock(ContainerStorageManager.class); - StorageEngine mockStore = mock(StorageEngine.class); - java.util.Map taskStores = ImmutableMap.of("mockStore", mockStore); - when(csm.getAllStores(any())).thenReturn(taskStores); - - TransactionalStateTaskStorageManager tsm = spy(buildTSM(csm, mock(Partition.class), new StorageManagerUtil())); - // stub actual method call - doReturn(mock(Map.class)).when(tsm).getNewestChangelogSSPOffsets(any(), any(), any(), any()); - - // invoke flush - tsm.flush(); - - // ensure that stores are flushed before we get newest changelog offsets - InOrder inOrder = inOrder(mockStore, tsm); - inOrder.verify(mockStore).flush(); - inOrder.verify(tsm).getNewestChangelogSSPOffsets(any(), any(), any(), any()); - } - - @Test - public void testGetNewestOffsetsReturnsCorrectOffset() { - ContainerStorageManager csm = mock(ContainerStorageManager.class); - TransactionalStateTaskStorageManager tsm = buildTSM(csm, mock(Partition.class), new StorageManagerUtil()); - - TaskName taskName = mock(TaskName.class); - String changelogSystemName = "systemName"; - String storeName = "storeName"; - String changelogStreamName = "changelogName"; - String newestChangelogSSPOffset = "1"; - SystemStream changelogSystemStream = new SystemStream(changelogSystemName, changelogStreamName); - Partition changelogPartition = new Partition(0); - SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); - - Map storeChangelogs = - ScalaJavaUtil.toScalaMap(ImmutableMap.of(storeName, changelogSystemStream)); - - SystemAdmins systemAdmins = mock(SystemAdmins.class); - SystemAdmin systemAdmin = mock(SystemAdmin.class); - SystemStreamPartitionMetadata metadata = mock(SystemStreamPartitionMetadata.class); - - when(metadata.getNewestOffset()).thenReturn(newestChangelogSSPOffset); - when(systemAdmins.getSystemAdmin(changelogSystemName)).thenReturn(systemAdmin); - when(systemAdmin.getSSPMetadata(eq(ImmutableSet.of(changelogSSP)))).thenReturn(ImmutableMap.of(changelogSSP, metadata)); - - // invoke the method - Map> offsets = - tsm.getNewestChangelogSSPOffsets( - taskName, storeChangelogs, changelogPartition, systemAdmins); - - // verify results - assertEquals(1, offsets.size()); - assertEquals(Option.apply(newestChangelogSSPOffset), offsets.apply(changelogSSP)); - } - - @Test - public void testGetNewestOffsetsReturnsNoneForEmptyTopic() { - // empty topic == null newest offset - ContainerStorageManager csm = mock(ContainerStorageManager.class); - TransactionalStateTaskStorageManager tsm = buildTSM(csm, mock(Partition.class), new StorageManagerUtil()); - - TaskName taskName = mock(TaskName.class); - String changelogSystemName = "systemName"; - String storeName = "storeName"; - String changelogStreamName = "changelogName"; - String newestChangelogSSPOffset = null; - SystemStream changelogSystemStream = new SystemStream(changelogSystemName, changelogStreamName); - Partition changelogPartition = new Partition(0); - SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); - - Map storeChangelogs = - ScalaJavaUtil.toScalaMap(ImmutableMap.of(storeName, changelogSystemStream)); - - SystemAdmins systemAdmins = mock(SystemAdmins.class); - SystemAdmin systemAdmin = mock(SystemAdmin.class); - SystemStreamPartitionMetadata metadata = mock(SystemStreamPartitionMetadata.class); - - when(metadata.getNewestOffset()).thenReturn(newestChangelogSSPOffset); - when(systemAdmins.getSystemAdmin(changelogSystemName)).thenReturn(systemAdmin); - when(systemAdmin.getSSPMetadata(eq(ImmutableSet.of(changelogSSP)))).thenReturn(ImmutableMap.of(changelogSSP, metadata)); - - // invoke the method - Map> offsets = - tsm.getNewestChangelogSSPOffsets( - taskName, storeChangelogs, changelogPartition, systemAdmins); - - // verify results - assertEquals(1, offsets.size()); - assertEquals(Option.empty(), offsets.apply(changelogSSP)); - } - - @Test(expected = SamzaException.class) - public void testGetNewestOffsetsThrowsIfNullMetadata() { - // empty topic == null newest offset - ContainerStorageManager csm = mock(ContainerStorageManager.class); - TransactionalStateTaskStorageManager tsm = buildTSM(csm, mock(Partition.class), new StorageManagerUtil()); - - TaskName taskName = mock(TaskName.class); - String changelogSystemName = "systemName"; - String storeName = "storeName"; - String changelogStreamName = "changelogName"; - String newestChangelogSSPOffset = null; - SystemStream changelogSystemStream = new SystemStream(changelogSystemName, changelogStreamName); - Partition changelogPartition = new Partition(0); - SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); - - Map storeChangelogs = - ScalaJavaUtil.toScalaMap(ImmutableMap.of(storeName, changelogSystemStream)); - - SystemAdmins systemAdmins = mock(SystemAdmins.class); - SystemAdmin systemAdmin = mock(SystemAdmin.class); - SystemStreamPartitionMetadata metadata = mock(SystemStreamPartitionMetadata.class); - - when(metadata.getNewestOffset()).thenReturn(newestChangelogSSPOffset); - when(systemAdmins.getSystemAdmin(changelogSystemName)).thenReturn(systemAdmin); - when(systemAdmin.getSSPMetadata(eq(ImmutableSet.of(changelogSSP)))).thenReturn(null); - - // invoke the method - Map> offsets = - tsm.getNewestChangelogSSPOffsets( - taskName, storeChangelogs, changelogPartition, systemAdmins); - - // verify results - fail("Should have thrown an exception if admin didn't return any metadata"); - } - - @Test(expected = SamzaException.class) - public void testGetNewestOffsetsThrowsIfNullSSPMetadata() { - // empty topic == null newest offset - ContainerStorageManager csm = mock(ContainerStorageManager.class); - TransactionalStateTaskStorageManager tsm = buildTSM(csm, mock(Partition.class), new StorageManagerUtil()); - - TaskName taskName = mock(TaskName.class); - String changelogSystemName = "systemName"; - String storeName = "storeName"; - String changelogStreamName = "changelogName"; - String newestChangelogSSPOffset = null; - SystemStream changelogSystemStream = new SystemStream(changelogSystemName, changelogStreamName); - Partition changelogPartition = new Partition(0); - SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); - - Map storeChangelogs = - ScalaJavaUtil.toScalaMap(ImmutableMap.of(storeName, changelogSystemStream)); - - SystemAdmins systemAdmins = mock(SystemAdmins.class); - SystemAdmin systemAdmin = mock(SystemAdmin.class); - SystemStreamPartitionMetadata metadata = mock(SystemStreamPartitionMetadata.class); - - when(metadata.getNewestOffset()).thenReturn(newestChangelogSSPOffset); - when(systemAdmins.getSystemAdmin(changelogSystemName)).thenReturn(systemAdmin); - java.util.Map metadataMap = new HashMap() { { - put(changelogSSP, null); - } }; - when(systemAdmin.getSSPMetadata(eq(ImmutableSet.of(changelogSSP)))).thenReturn(metadataMap); - - // invoke the method - Map> offsets = - tsm.getNewestChangelogSSPOffsets( - taskName, storeChangelogs, changelogPartition, systemAdmins); - - // verify results - fail("Should have thrown an exception if admin returned null metadata for changelog SSP"); - } - - @Test(expected = SamzaException.class) - public void testGetNewestOffsetsThrowsIfErrorGettingMetadata() { - // empty topic == null newest offset - ContainerStorageManager csm = mock(ContainerStorageManager.class); - TransactionalStateTaskStorageManager tsm = buildTSM(csm, mock(Partition.class), new StorageManagerUtil()); - - TaskName taskName = mock(TaskName.class); - String changelogSystemName = "systemName"; - String storeName = "storeName"; - String changelogStreamName = "changelogName"; - String newestChangelogSSPOffset = null; - SystemStream changelogSystemStream = new SystemStream(changelogSystemName, changelogStreamName); - Partition changelogPartition = new Partition(0); - SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); - - Map storeChangelogs = - ScalaJavaUtil.toScalaMap(ImmutableMap.of(storeName, changelogSystemStream)); - - SystemAdmins systemAdmins = mock(SystemAdmins.class); - SystemAdmin systemAdmin = mock(SystemAdmin.class); - SystemStreamPartitionMetadata metadata = mock(SystemStreamPartitionMetadata.class); - - when(metadata.getNewestOffset()).thenReturn(newestChangelogSSPOffset); - when(systemAdmins.getSystemAdmin(changelogSystemName)).thenThrow(new SamzaException("Error getting metadata")); - when(systemAdmin.getSSPMetadata(eq(ImmutableSet.of(changelogSSP)))).thenReturn(null); - - // invoke the method - Map> offsets = - tsm.getNewestChangelogSSPOffsets( - taskName, storeChangelogs, changelogPartition, systemAdmins); - - // verify results - fail("Should have thrown an exception if admin had an error getting metadata"); - } - - @Test - public void testCheckpoint() { - ContainerStorageManager csm = mock(ContainerStorageManager.class); - - StorageEngine mockLPStore = mock(StorageEngine.class); - StoreProperties lpStoreProps = mock(StoreProperties.class); - when(mockLPStore.getStoreProperties()).thenReturn(lpStoreProps); - when(lpStoreProps.isPersistedToDisk()).thenReturn(true); - when(lpStoreProps.isLoggedStore()).thenReturn(true); - Path mockPath = mock(Path.class); - when(mockLPStore.checkpoint(any())).thenReturn(Optional.of(mockPath)); - - StorageEngine mockPStore = mock(StorageEngine.class); - StoreProperties pStoreProps = mock(StoreProperties.class); - when(mockPStore.getStoreProperties()).thenReturn(pStoreProps); - when(pStoreProps.isPersistedToDisk()).thenReturn(true); - when(pStoreProps.isLoggedStore()).thenReturn(false); - - StorageEngine mockLIStore = mock(StorageEngine.class); - StoreProperties liStoreProps = mock(StoreProperties.class); - when(mockLIStore.getStoreProperties()).thenReturn(liStoreProps); - when(liStoreProps.isPersistedToDisk()).thenReturn(false); - when(liStoreProps.isLoggedStore()).thenReturn(true); - - StorageEngine mockIStore = mock(StorageEngine.class); - StoreProperties iStoreProps = mock(StoreProperties.class); - when(mockIStore.getStoreProperties()).thenReturn(iStoreProps); - when(iStoreProps.isPersistedToDisk()).thenReturn(false); - when(iStoreProps.isLoggedStore()).thenReturn(false); - - java.util.Map taskStores = ImmutableMap.of( - "loggedPersistentStore", mockLPStore, - "persistentStore", mockPStore, - "loggedInMemStore", mockLIStore, - "inMemStore", mockIStore - ); - when(csm.getAllStores(any())).thenReturn(taskStores); - - TransactionalStateTaskStorageManager tsm = spy(buildTSM(csm, mock(Partition.class), new StorageManagerUtil())); - // stub actual method call - ArgumentCaptor checkpointPathsCaptor = ArgumentCaptor.forClass(Map.class); - doNothing().when(tsm).writeChangelogOffsetFiles(any(), any(), any()); - - Map> offsets = ScalaJavaUtil.toScalaMap( - ImmutableMap.of(mock(SystemStreamPartition.class), Option.apply("1"))); - - // invoke checkpoint - tsm.checkpoint(CheckpointId.create(), offsets); - - // ensure that checkpoint is never called for non-logged persistent stores since they're - // always cleared on restart. - verify(mockPStore, never()).checkpoint(any()); - // ensure that checkpoint is never called for in-memory stores since they're not persistent. - verify(mockIStore, never()).checkpoint(any()); - verify(mockLIStore, never()).checkpoint(any()); - verify(tsm).writeChangelogOffsetFiles(checkpointPathsCaptor.capture(), any(), eq(offsets)); - Map checkpointPaths = checkpointPathsCaptor.getValue(); - assertEquals(1, checkpointPaths.size()); - assertEquals(mockPath, checkpointPaths.apply("loggedPersistentStore")); - } - - @Test(expected = IllegalStateException.class) - public void testCheckpointFailsIfErrorCreatingCheckpoint() { - ContainerStorageManager csm = mock(ContainerStorageManager.class); - - StorageEngine mockLPStore = mock(StorageEngine.class); - StoreProperties lpStoreProps = mock(StoreProperties.class); - when(mockLPStore.getStoreProperties()).thenReturn(lpStoreProps); - when(lpStoreProps.isPersistedToDisk()).thenReturn(true); - when(lpStoreProps.isLoggedStore()).thenReturn(true); - when(mockLPStore.checkpoint(any())).thenThrow(new IllegalStateException()); - java.util.Map taskStores = - ImmutableMap.of("loggedPersistentStore", mockLPStore); - when(csm.getAllStores(any())).thenReturn(taskStores); - - TransactionalStateTaskStorageManager tsm = spy(buildTSM(csm, mock(Partition.class), new StorageManagerUtil())); - - Map> offsets = ScalaJavaUtil.toScalaMap( - ImmutableMap.of(mock(SystemStreamPartition.class), Option.apply("1"))); - - // invoke checkpoint - tsm.checkpoint(CheckpointId.create(), offsets); - verify(tsm, never()).writeChangelogOffsetFiles(any(), any(), any()); - fail("Should have thrown an exception if error creating store checkpoint"); - } - - @Test(expected = SamzaException.class) - public void testCheckpointFailsIfErrorWritingOffsetFiles() { - ContainerStorageManager csm = mock(ContainerStorageManager.class); - - StorageEngine mockLPStore = mock(StorageEngine.class); - StoreProperties lpStoreProps = mock(StoreProperties.class); - when(mockLPStore.getStoreProperties()).thenReturn(lpStoreProps); - when(lpStoreProps.isPersistedToDisk()).thenReturn(true); - when(lpStoreProps.isLoggedStore()).thenReturn(true); - Path mockPath = mock(Path.class); - when(mockLPStore.checkpoint(any())).thenReturn(Optional.of(mockPath)); - java.util.Map taskStores = - ImmutableMap.of("loggedPersistentStore", mockLPStore); - when(csm.getAllStores(any())).thenReturn(taskStores); - - TransactionalStateTaskStorageManager tsm = spy(buildTSM(csm, mock(Partition.class), new StorageManagerUtil())); - doThrow(new SamzaException("Error writing offset file")) - .when(tsm).writeChangelogOffsetFiles(any(), any(), any()); - - Map> offsets = ScalaJavaUtil.toScalaMap( - ImmutableMap.of(mock(SystemStreamPartition.class), Option.apply("1"))); - - // invoke checkpoint - tsm.checkpoint(CheckpointId.create(), offsets); - - fail("Should have thrown an exception if error writing offset file."); - } - - @Test - public void testWriteChangelogOffsetFiles() throws IOException { - String storeName = "mockStore"; - ContainerStorageManager csm = mock(ContainerStorageManager.class); - StorageEngine mockStore = mock(StorageEngine.class); - java.util.Map taskStores = ImmutableMap.of(storeName, mockStore); - when(csm.getAllStores(any())).thenReturn(taskStores); - - Partition changelogPartition = new Partition(0); - SystemStream changelogSS = new SystemStream("system", "changelog"); - SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSS, changelogPartition); - StorageManagerUtil smu = spy(new StorageManagerUtil()); - File mockCurrentStoreDir = mock(File.class); - doReturn(mockCurrentStoreDir).when(smu).getTaskStoreDir(any(), eq(storeName), any(), any()); - doNothing().when(smu).writeOffsetFile(eq(mockCurrentStoreDir), any(), anyBoolean()); - TransactionalStateTaskStorageManager tsm = spy(buildTSM(csm, changelogPartition, smu)); - - String changelogNewestOffset = "1"; - Map> offsets = ScalaJavaUtil.toScalaMap( - ImmutableMap.of(changelogSSP, Option.apply(changelogNewestOffset))); - - Path checkpointPath = Files.createTempDirectory("store-checkpoint-test").toAbsolutePath(); - - Map checkpointPaths = ScalaJavaUtil.toScalaMap( - ImmutableMap.of(storeName, checkpointPath)); - Map storeChangelogs = ScalaJavaUtil.toScalaMap( - ImmutableMap.of(storeName, changelogSS)); - - // invoke method - tsm.writeChangelogOffsetFiles(checkpointPaths, storeChangelogs, offsets); - - // verify that offset file was written to the checkpoint dir - java.util.Map fileOffsets = new StorageManagerUtil() - .readOffsetFile(checkpointPath.toFile(), ImmutableSet.of(changelogSSP), false); - assertEquals(1, fileOffsets.size()); - assertEquals(changelogNewestOffset, fileOffsets.get(changelogSSP)); - - // verify that offset file write was called on the current dir - verify(smu, times(1)).writeOffsetFile(eq(mockCurrentStoreDir), any(), anyBoolean()); - } - - @Test - public void testWriteChangelogOffsetFilesWithEmptyChangelogTopic() throws IOException { - String storeName = "mockStore"; - ContainerStorageManager csm = mock(ContainerStorageManager.class); - StorageEngine mockStore = mock(StorageEngine.class); - java.util.Map taskStores = ImmutableMap.of(storeName, mockStore); - when(csm.getAllStores(any())).thenReturn(taskStores); - - Partition changelogPartition = new Partition(0); - SystemStream changelogSS = new SystemStream("system", "changelog"); - SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSS, changelogPartition); - StorageManagerUtil mockSMU = mock(StorageManagerUtil.class); - File mockCurrentStoreDir = mock(File.class); - when(mockSMU.getTaskStoreDir(any(), eq(storeName), any(), any())).thenReturn(mockCurrentStoreDir); - TransactionalStateTaskStorageManager tsm = spy(buildTSM(csm, changelogPartition, mockSMU)); - - String changelogNewestOffset = null; - Map> offsets = ScalaJavaUtil.toScalaMap( - ImmutableMap.of(changelogSSP, Option.apply(changelogNewestOffset))); - - Path checkpointPath = Files.createTempDirectory("store-checkpoint-test").toAbsolutePath(); - - Map checkpointPaths = ScalaJavaUtil.toScalaMap( - ImmutableMap.of(storeName, checkpointPath)); - Map storeChangelogs = ScalaJavaUtil.toScalaMap( - ImmutableMap.of(storeName, changelogSS)); - - // invoke method - tsm.writeChangelogOffsetFiles(checkpointPaths, storeChangelogs, offsets); - - // verify that the offset files were not written to the checkpoint dir - assertFalse(Files.exists(new File(checkpointPath.toFile(), StorageManagerUtil.OFFSET_FILE_NAME_LEGACY).toPath())); - assertFalse(Files.exists(new File(checkpointPath.toFile(), StorageManagerUtil.OFFSET_FILE_NAME_NEW).toPath())); - java.util.Map fileOffsets = new StorageManagerUtil() - .readOffsetFile(checkpointPath.toFile(), ImmutableSet.of(changelogSSP), false); - assertEquals(0, fileOffsets.size()); - - // verify that delete was called on current store dir offset file - verify(mockSMU, times(1)).deleteOffsetFile(eq(mockCurrentStoreDir)); - } - - /** - * This should never happen with CheckpointingTaskStorageManager. #getNewestChangelogSSPOffset must - * return a key for every changelog SSP. If the SSP is empty, the value should be none. If it could - * not fetch metadata, it should throw an exception instead of skipping the SSP. - * If this contract is accidentally broken, ensure that we fail the commit - */ - @Test(expected = SamzaException.class) - public void testWriteChangelogOffsetFilesWithNoChangelogOffset() throws IOException { - String storeName = "mockStore"; - ContainerStorageManager csm = mock(ContainerStorageManager.class); - StorageEngine mockStore = mock(StorageEngine.class); - java.util.Map taskStores = ImmutableMap.of(storeName, mockStore); - when(csm.getAllStores(any())).thenReturn(taskStores); - - Partition changelogPartition = new Partition(0); - SystemStream changelogSS = new SystemStream("system", "changelog"); - SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSS, changelogPartition); - TransactionalStateTaskStorageManager tsm = spy(buildTSM(csm, changelogPartition, new StorageManagerUtil())); - - // no mapping present for changelog newest offset - Map> offsets = ScalaJavaUtil.toScalaMap(ImmutableMap.of()); - - Path checkpointPath = Files.createTempDirectory("store-checkpoint-test").toAbsolutePath(); - Map checkpointPaths = ScalaJavaUtil.toScalaMap( - ImmutableMap.of(storeName, checkpointPath)); - Map storeChangelogs = ScalaJavaUtil.toScalaMap( - ImmutableMap.of(storeName, changelogSS)); - - // invoke method - tsm.writeChangelogOffsetFiles(checkpointPaths, storeChangelogs, offsets); - - fail("Should have thrown an exception if no changelog offset found for checkpointed store"); - } - - @Test - public void testRemoveOldCheckpointsWhenBaseDirContainsRegularFiles() { - TaskName taskName = new TaskName("Partition 0"); - ContainerStorageManager containerStorageManager = mock(ContainerStorageManager.class); - Map changelogSystemStreams = mock(Map.class); - SystemAdmins systemAdmins = mock(SystemAdmins.class); - File loggedStoreBaseDir = mock(File.class); - Partition changelogPartition = new Partition(0); - TaskMode taskMode = TaskMode.Active; - StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class); - - File mockStoreDir = mock(File.class); - String mockStoreDirName = "notDirectory"; - - when(loggedStoreBaseDir.listFiles()).thenReturn(new File[] {mockStoreDir}); - when(mockStoreDir.getName()).thenReturn(mockStoreDirName); - when(storageManagerUtil.getTaskStoreDir(eq(loggedStoreBaseDir), eq(mockStoreDirName), eq(taskName), eq(taskMode))).thenReturn(mockStoreDir); - // null here can happen if listFiles is called on a non-directory - when(mockStoreDir.listFiles(any(FileFilter.class))).thenReturn(null); - - TransactionalStateTaskStorageManager tsm = new TransactionalStateTaskStorageManager(taskName, containerStorageManager, - changelogSystemStreams, systemAdmins, loggedStoreBaseDir, changelogPartition, taskMode, storageManagerUtil); - - tsm.removeOldCheckpoints(CheckpointId.create()); - } - - private TransactionalStateTaskStorageManager buildTSM(ContainerStorageManager csm, Partition changelogPartition, - StorageManagerUtil smu) { - TaskName taskName = new TaskName("Partition 0"); - Map changelogSystemStreams = mock(Map.class); - SystemAdmins systemAdmins = mock(SystemAdmins.class); - File loggedStoreBaseDir = mock(File.class); - TaskMode taskMode = TaskMode.Active; - - return new TransactionalStateTaskStorageManager( - taskName, csm, changelogSystemStreams, systemAdmins, - loggedStoreBaseDir, changelogPartition, taskMode, smu); - } -} \ No newline at end of file diff --git a/samza-core/src/test/scala/org/apache/samza/storage/TestTaskStorageManager.scala b/samza-kafka/src/test/java/org/apache/samza/storage/TestTaskStorageManager.scala similarity index 83% rename from samza-core/src/test/scala/org/apache/samza/storage/TestTaskStorageManager.scala rename to samza-kafka/src/test/java/org/apache/samza/storage/TestTaskStorageManager.scala index 957c00ccc0..167bc78e67 100644 --- a/samza-core/src/test/scala/org/apache/samza/storage/TestTaskStorageManager.scala +++ b/samza-kafka/src/test/java/org/apache/samza/storage/TestTaskStorageManager.scala @@ -22,24 +22,27 @@ package org.apache.samza.storage import java.io.{File, FileOutputStream, ObjectOutputStream} import java.util +import com.google.common.collect.{ImmutableMap, ImmutableSet} import org.apache.samza.Partition +import org.apache.samza.checkpoint.kafka.KafkaStateCheckpointMarker +import org.apache.samza.checkpoint.{CheckpointId, CheckpointManager, CheckpointV1} import org.apache.samza.config._ import org.apache.samza.container.{SamzaContainerMetrics, TaskInstanceMetrics, TaskName} import org.apache.samza.context.{ContainerContext, JobContext} -import org.apache.samza.job.model.{ContainerModel, TaskMode, TaskModel} +import org.apache.samza.job.model.{ContainerModel, JobModel, TaskMode, TaskModel} import org.apache.samza.serializers.{Serde, StringSerdeFactory} import org.apache.samza.storage.StoreProperties.StorePropertiesBuilder import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata import org.apache.samza.system._ import org.apache.samza.task.TaskInstanceCollector -import org.apache.samza.util.{FileUtil, SystemClock} +import org.apache.samza.util.{Clock, FileUtil, SystemClock} import org.junit.Assert._ import org.junit.runner.RunWith import org.junit.runners.Parameterized import org.junit.runners.Parameterized.Parameters import org.junit.{After, Before, Test} import org.mockito.Matchers._ -import org.mockito.{Mockito} +import org.mockito.Mockito import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -48,8 +51,6 @@ import org.scalatest.mockito.MockitoSugar import scala.collection.JavaConverters._ import scala.collection.immutable.HashMap import scala.collection.mutable -import com.google.common.collect.{ImmutableMap, ImmutableSet} -import org.apache.samza.checkpoint.{Checkpoint, CheckpointManager} /** * This test is parameterized on the offsetFileName and is run for both @@ -58,7 +59,7 @@ import org.apache.samza.checkpoint.{Checkpoint, CheckpointManager} * @param offsetFileName the name of the offset file. */ @RunWith(value = classOf[Parameterized]) -class TestNonTransactionalStateTaskStorageManager(offsetFileName: String) extends MockitoSugar { +class TestKafkaNonTransactionalStateTaskBackupManager(offsetFileName: String) extends MockitoSugar { val store = "store1" val loggedStore = "loggedStore1" @@ -131,44 +132,6 @@ class TestNonTransactionalStateTaskStorageManager(offsetFileName: String) extend assertTrue(storeFile.exists()) assertFalse(offsetFile.exists()) verify(mockSystemConsumer).register(ssp, "0") - - // Test 2: flush should update the offset file - taskManager.flush() - assertTrue(offsetFile.exists()) - validateOffsetFileContents(offsetFile, "kafka.testStream-loggedStore1.0", "50") - - // Test 3: Update sspMetadata before shutdown and verify that offset file is not updated - when(mockSystemAdmin.getSSPMetadata(ImmutableSet.of(ssp))) - .thenReturn(ImmutableMap.of(ssp, new SystemStreamPartitionMetadata("0", "100", "101"))) - taskManager.stop() - verify(mockStorageEngine, times(1)).flush() // only called once during Test 2. - assertTrue(storeFile.exists()) - assertTrue(offsetFile.exists()) - validateOffsetFileContents(offsetFile, "kafka.testStream-loggedStore1.0", "50") - - // Test 4: Initialize again with an updated sspMetadata; Verify that it restores from the correct offset - sspMetadata = new SystemStreamPartitionMetadata("0", "150", "151") - metadata = new SystemStreamMetadata(getStreamName(loggedStore), new java.util.HashMap[Partition, SystemStreamPartitionMetadata]() { - { - put(partition, sspMetadata) - } - }) - when(mockStreamMetadataCache.getStreamMetadata(any(), any())).thenReturn(Map(ss -> metadata)) - when(mockSystemAdmin.getSSPMetadata(ImmutableSet.of(ssp))) - .thenReturn(ImmutableMap.of(ssp, sspMetadata)) - when(mockSystemAdmin.getOffsetsAfter(Map(ssp -> "50").asJava)).thenReturn(Map(ssp -> "51").asJava) - Mockito.reset(mockSystemConsumer) - - taskManager = new TaskStorageManagerBuilder() - .addStore(loggedStore, mockStorageEngine, mockSystemConsumer) - .setStreamMetadataCache(mockStreamMetadataCache) - .setSystemAdmin("kafka", mockSystemAdmin) - .initializeContainerStorageManager() - .build - - assertTrue(storeFile.exists()) - assertTrue(offsetFile.exists()) - verify(mockSystemConsumer).register(ssp, "51") } /** @@ -217,7 +180,9 @@ class TestNonTransactionalStateTaskStorageManager(offsetFileName: String) extend verify(mockSystemConsumer).register(ssp, "0") // Test 2: flush should NOT create/update the offset file. Store directory has no files - taskManager.flush() + val checkpointId = CheckpointId.create() + val snapshot = taskManager.snapshot(checkpointId) + val stateCheckpointMarkers = taskManager.upload(checkpointId, snapshot) assertTrue(storeDirectory.list().isEmpty) // Test 3: Update sspMetadata before shutdown and verify that offset file is NOT created @@ -228,7 +193,7 @@ class TestNonTransactionalStateTaskStorageManager(offsetFileName: String) extend }) when(mockStreamMetadataCache.getStreamMetadata(any(), any())).thenReturn(Map(ss -> metadata)) when(mockSystemAdmin.getSSPMetadata(ImmutableSet.of(ssp))).thenReturn(ImmutableMap.of(ssp, sspMetadata)) - taskManager.stop() + taskManager.close() assertTrue(storeDirectory.list().isEmpty) // Test 4: Initialize again with an updated sspMetadata; Verify that it restores from the earliest offset @@ -368,51 +333,12 @@ class TestNonTransactionalStateTaskStorageManager(offsetFileName: String) extend .build //Invoke test method - taskStorageManager.stop() + taskStorageManager.close() //Check conditions assertFalse("Offset file doesn't exist!", offsetFile.exists()) } - /** - * Given that the SSPMetadataCache returns metadata, flush should create the offset files. - */ - @Test - def testFlushCreatesOffsetFileForLoggedStore() { - val partition = new Partition(0) - - val offsetFilePath = new File(storageManagerUtil.getTaskStoreDir(TaskStorageManagerBuilder.defaultLoggedStoreBaseDir, loggedStore, taskName, TaskMode.Active) + File.separator + offsetFileName) - val anotherOffsetPath = new File( - storageManagerUtil.getTaskStoreDir(TaskStorageManagerBuilder.defaultLoggedStoreBaseDir, store, taskName, TaskMode.Active) + File.separator + offsetFileName) - - val ssp1 = new SystemStreamPartition("kafka", getStreamName(loggedStore), partition) - val ssp2 = new SystemStreamPartition("kafka", getStreamName(store), partition) - val sspMetadata = new SystemStreamPartitionMetadata("20", "100", "101") - - val mockSystemAdmin = mock[SystemAdmin] - when(mockSystemAdmin.getSSPMetadata(ImmutableSet.of(ssp1))).thenReturn(ImmutableMap.of(ssp1, sspMetadata)) - when(mockSystemAdmin.getSSPMetadata(ImmutableSet.of(ssp2))).thenReturn(ImmutableMap.of(ssp2, sspMetadata)) - - //Build TaskStorageManager - val taskStorageManager = new TaskStorageManagerBuilder() - .addLoggedStore(loggedStore, true) - .addStore(store, false) - .setSystemAdmin("kafka", mockSystemAdmin) - .setStreamMetadataCache(createMockStreamMetadataCache("20", "100", "101")) - .setPartition(partition) - .initializeContainerStorageManager() - .build - - //Invoke test method - taskStorageManager.flush() - - //Check conditions - assertTrue("Offset file doesn't exist!", offsetFilePath.exists()) - validateOffsetFileContents(offsetFilePath, "kafka.testStream-loggedStore1.0", "100") - - assertTrue("Offset file got created for a store that is not persisted to the disk!!", !anotherOffsetPath.exists()) - } - /** * Flush should delete the existing OFFSET file if the changelog partition (for some reason) becomes empty */ @@ -420,8 +346,6 @@ class TestNonTransactionalStateTaskStorageManager(offsetFileName: String) extend def testFlushDeletesOffsetFileForLoggedStoreForEmptyPartition() { val partition = new Partition(0) - val offsetFilePath = new File(storageManagerUtil.getTaskStoreDir(TaskStorageManagerBuilder.defaultLoggedStoreBaseDir, loggedStore, taskName, TaskMode.Active) + File.separator + offsetFileName) - val ssp = new SystemStreamPartition("kafka", getStreamName(loggedStore), partition) val sspMetadata = new SystemStreamPartitionMetadata("0", "100", "101") val nullSspMetadata = new SystemStreamPartitionMetadata(null, null, null) @@ -430,7 +354,7 @@ class TestNonTransactionalStateTaskStorageManager(offsetFileName: String) extend .thenReturn(ImmutableMap.of(ssp, sspMetadata)) .thenReturn(ImmutableMap.of(ssp, nullSspMetadata)) - var metadata = new SystemStreamMetadata(getStreamName(loggedStore), new java.util.HashMap[Partition, SystemStreamPartitionMetadata]() { + val metadata = new SystemStreamMetadata(getStreamName(loggedStore), new java.util.HashMap[Partition, SystemStreamPartitionMetadata]() { { put(partition, sspMetadata) } @@ -449,67 +373,15 @@ class TestNonTransactionalStateTaskStorageManager(offsetFileName: String) extend .build //Invoke test method - taskStorageManager.flush() - - //Check conditions - assertTrue("Offset file doesn't exist!", offsetFilePath.exists()) - validateOffsetFileContents(offsetFilePath, "kafka.testStream-loggedStore1.0", "100") + val checkpointId = CheckpointId.create() + var snapshot = taskStorageManager.snapshot(checkpointId) + taskStorageManager.upload(checkpointId, snapshot) //Invoke test method again - taskStorageManager.flush() + snapshot = taskStorageManager.snapshot(checkpointId) + val stateCheckpointMarkers2 = taskStorageManager.upload(checkpointId, snapshot) - //Check conditions - assertFalse("Offset file for null offset exists!", offsetFilePath.exists()) - } - - @Test - def testFlushOverwritesOffsetFileForLoggedStore() { - val partition = new Partition(0) - val ssp = new SystemStreamPartition("kafka", getStreamName(loggedStore), partition) - - val offsetFilePath = new File(storageManagerUtil.getTaskStoreDir(TaskStorageManagerBuilder.defaultLoggedStoreBaseDir, loggedStore, taskName, TaskMode.Active) + File.separator + offsetFileName) - fileUtil.writeWithChecksum(offsetFilePath, "100") - - val sspMetadata = new SystemStreamPartitionMetadata("20", "139", "140") - val mockSystemAdmin = mock[SystemAdmin] - when(mockSystemAdmin.getSSPMetadata(ImmutableSet.of(ssp))).thenReturn(ImmutableMap.of(ssp, sspMetadata)) - - - var metadata = new SystemStreamMetadata(getStreamName(loggedStore), new java.util.HashMap[Partition, SystemStreamPartitionMetadata]() { - { - put(partition, sspMetadata) - } - }) - - val mockStreamMetadataCache = mock[StreamMetadataCache] - when(mockStreamMetadataCache.getStreamMetadata(any(), any())).thenReturn(Map(new SystemStream("kafka", getStreamName(loggedStore)) -> metadata)) - - //Build TaskStorageManager - val taskStorageManager = new TaskStorageManagerBuilder() - .addLoggedStore(loggedStore, true) - .setSystemAdmin("kafka", mockSystemAdmin) - .setPartition(partition) - .setStreamMetadataCache(mockStreamMetadataCache) - .initializeContainerStorageManager() - .build - - //Invoke test method - taskStorageManager.flush() - - //Check conditions - assertTrue("Offset file doesn't exist!", offsetFilePath.exists()) - validateOffsetFileContents(offsetFilePath, "kafka.testStream-loggedStore1.0", "139") - - // Flush again - when(mockSystemAdmin.getSSPMetadata(ImmutableSet.of(ssp))) - .thenReturn(ImmutableMap.of(ssp, new SystemStreamPartitionMetadata("20", "193", "194"))) - - //Invoke test method - taskStorageManager.flush() - - //Check conditions - assertTrue("Offset file doesn't exist!", offsetFilePath.exists()) - validateOffsetFileContents(offsetFilePath, "kafka.testStream-loggedStore1.0", "193") + assertNull(KafkaStateCheckpointMarker.deserialize(stateCheckpointMarkers2.get.get(loggedStore)).getChangelogOffset) } /** @@ -517,7 +389,7 @@ class TestNonTransactionalStateTaskStorageManager(offsetFileName: String) extend * The legacy offset file only contains the offset as a string, while the new offset file contains a map of * ssp to offset in json format. * The name of the two offset files are given in {@link StorageManagerUtil.OFFSET_FILE_NAME_NEW} and - * {@link StorageManagerUtil.OFFSET_FILE_LEGACY}. + * {@link StorageManagerUtil.OFFSET_FILE_NAME_LEGACY}. */ private def validateOffsetFileContents(offsetFile: File, ssp: String, offset: String): Unit = { @@ -549,7 +421,7 @@ class TestNonTransactionalStateTaskStorageManager(offsetFileName: String) extend .build //Invoke test method - taskStorageManager.stop() + taskStorageManager.close() //Check conditions assertTrue("Offset file should not exist!", !offsetFilePath.exists()) @@ -777,7 +649,7 @@ class TestNonTransactionalStateTaskStorageManager(offsetFileName: String) extend } } -object TestNonTransactionalStateTaskStorageManager { +object TestKafkaNonTransactionalStateTaskBackupManager { @Parameters def parameters: util.Collection[Array[String]] = { val offsetFileNames = new util.ArrayList[Array[String]]() @@ -864,7 +736,10 @@ class TaskStorageManagerBuilder extends MockitoSugar { var containerModel = new ContainerModel("container", tasks.asJava) val mockSystemAdmins = Mockito.mock(classOf[SystemAdmins]) - Mockito.when(mockSystemAdmins.getSystemAdmin(org.mockito.Matchers.eq("kafka"))).thenReturn(systemAdminsMap.get("kafka").get) + Mockito.when(mockSystemAdmins.getSystemAdmin(org.mockito.Matchers.eq("kafka"))) + .thenReturn(systemAdminsMap.get("kafka").get) + Mockito.when(mockSystemAdmins.getSystemAdmins) + .thenReturn(systemAdminsMap.asJava) var mockStorageEngineFactory : StorageEngineFactory[AnyRef, AnyRef] = Mockito.mock(classOf[StorageEngineFactory[AnyRef, AnyRef]]) @@ -900,15 +775,20 @@ class TaskStorageManagerBuilder extends MockitoSugar { val mockCheckpointManager = Mockito.mock(classOf[CheckpointManager]) when(mockCheckpointManager.readLastCheckpoint(any(classOf[TaskName]))) - .thenReturn(new Checkpoint(new util.HashMap[SystemStreamPartition, String]())) + .thenReturn(new CheckpointV1(new util.HashMap[SystemStreamPartition, String]())) + + val mockContainerContext = Mockito.mock(classOf[ContainerContext]) + when(mockContainerContext.getContainerModel).thenReturn(containerModel); - val mockSSPMetadataCache = Mockito.mock(classOf[SSPMetadataCache]) + val mockJobContext = Mockito.mock(classOf[JobContext]) + val mockJobModel = Mockito.mock(classOf[JobModel]) + when(mockJobContext.getJobModel).thenReturn(mockJobModel) + when(mockJobModel.getMaxChangeLogStreamPartitions).thenReturn(1) containerStorageManager = new ContainerStorageManager( mockCheckpointManager, containerModel, streamMetadataCache, - mockSSPMetadataCache, mockSystemAdmins, changeLogSystemStreams.asJava, Map[String, util.Set[SystemStream]]().asJava, @@ -918,12 +798,12 @@ class TaskStorageManagerBuilder extends MockitoSugar { config, new HashMap[TaskName, TaskInstanceMetrics]().asJava, Mockito.mock(classOf[SamzaContainerMetrics]), - Mockito.mock(classOf[JobContext]), - Mockito.mock(classOf[ContainerContext]), + mockJobContext, + mockContainerContext, + new mockKafkaChangelogBackendManager(changeLogSystemStreams), new HashMap[TaskName, TaskInstanceCollector].asJava, loggedStoreBaseDir, TaskStorageManagerBuilder.defaultStoreBaseDir, - 1, null, new SystemClock) this @@ -931,18 +811,16 @@ class TaskStorageManagerBuilder extends MockitoSugar { - def build: NonTransactionalStateTaskStorageManager = { + def build: KafkaNonTransactionalStateTaskBackupManager = { if (containerStorageManager != null) { containerStorageManager.start() } - new NonTransactionalStateTaskStorageManager( + new KafkaNonTransactionalStateTaskBackupManager( taskName = taskName, - containerStorageManager = containerStorageManager, - storeChangelogs = changeLogSystemStreams, + storeChangelogs = changeLogSystemStreams.asJava, systemAdmins = buildSystemAdmins(systemAdminsMap), - loggedStoreBaseDir = loggedStoreBaseDir, partition = partition ) } @@ -954,4 +832,12 @@ class TaskStorageManagerBuilder extends MockitoSugar { } systemAdmins } + + private class mockKafkaChangelogBackendManager(storeSystemStream: Map[String, SystemStream]) + extends KafkaChangelogStateBackendFactory { + override def filterStandbySystemStreams(changelogSystemStreams: util.Map[String, SystemStream], containerModel: ContainerModel): + util.Map[String, SystemStream] = storeSystemStream.asJava + + override def getStreamCache(admins: SystemAdmins, clock: Clock): StreamMetadataCache = streamMetadataCache + } } diff --git a/samza-kafka/src/test/java/org/apache/samza/storage/TestTransactionalStateTaskBackupManager.java b/samza-kafka/src/test/java/org/apache/samza/storage/TestTransactionalStateTaskBackupManager.java new file mode 100644 index 0000000000..8bb93f769f --- /dev/null +++ b/samza-kafka/src/test/java/org/apache/samza/storage/TestTransactionalStateTaskBackupManager.java @@ -0,0 +1,276 @@ +/* + * 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.samza.storage; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.samza.checkpoint.kafka.KafkaStateCheckpointMarker; + +import java.util.HashMap; +import java.util.concurrent.ForkJoinPool; +import org.apache.samza.Partition; +import org.apache.samza.SamzaException; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.container.TaskInstanceMetrics; +import org.apache.samza.container.TaskName; +import org.apache.samza.metrics.Timer; +import org.apache.samza.system.SystemAdmin; +import org.apache.samza.system.SystemAdmins; +import org.apache.samza.system.SystemStream; +import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata; +import org.apache.samza.system.SystemStreamPartition; +import org.junit.Test; +import org.mockito.InOrder; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + + +public class TestTransactionalStateTaskBackupManager { + @Test + public void testFlushOrder() { + ContainerStorageManager csm = mock(ContainerStorageManager.class); + StorageEngine mockStore = mock(StorageEngine.class); + java.util.Map taskStores = ImmutableMap.of("mockStore", mockStore); + when(csm.getAllStores(any())).thenReturn(taskStores); + when(mockStore.getStoreProperties()).thenReturn(new StoreProperties + .StorePropertiesBuilder().setPersistedToDisk(true).setLoggedStore(true).build()); + TaskInstanceMetrics metrics = mock(TaskInstanceMetrics.class); + Timer checkpointTimer = mock(Timer.class); + when(metrics.storeCheckpointNs()).thenReturn(checkpointTimer); + + KafkaTransactionalStateTaskBackupManager tsm = spy(buildTSM(csm, mock(Partition.class), new StorageManagerUtil())); + TaskStorageCommitManager commitManager = new TaskStorageCommitManager(new TaskName("task"), + ImmutableMap.of("kafka", tsm), csm, null, null, null, null, + ForkJoinPool.commonPool(), new StorageManagerUtil(), null, metrics); + // stub actual method call + doReturn(mock(java.util.Map.class)).when(tsm).getNewestChangelogSSPOffsets(any(), any(), any(), any()); + + // invoke Kafka flush + commitManager.init(); + commitManager.snapshot(CheckpointId.create()); + + // ensure that stores are flushed before we get newest changelog offsets + InOrder inOrder = inOrder(mockStore, tsm); + inOrder.verify(mockStore).flush(); + inOrder.verify(tsm).getNewestChangelogSSPOffsets(any(), any(), any(), any()); + } + + @Test + public void testGetNewestOffsetsReturnsCorrectOffset() { + ContainerStorageManager csm = mock(ContainerStorageManager.class); + KafkaTransactionalStateTaskBackupManager tsm = buildTSM(csm, mock(Partition.class), new StorageManagerUtil()); + + TaskName taskName = mock(TaskName.class); + String changelogSystemName = "systemName"; + String storeName = "storeName"; + String changelogStreamName = "changelogName"; + String newestChangelogSSPOffset = "1"; + SystemStream changelogSystemStream = new SystemStream(changelogSystemName, changelogStreamName); + Partition changelogPartition = new Partition(0); + SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); + + java.util.Map storeChangelogs = new HashMap<>(); + storeChangelogs.put(storeName, changelogSystemStream); + + SystemAdmins systemAdmins = mock(SystemAdmins.class); + SystemAdmin systemAdmin = mock(SystemAdmin.class); + SystemStreamPartitionMetadata metadata = mock(SystemStreamPartitionMetadata.class); + + when(metadata.getNewestOffset()).thenReturn(newestChangelogSSPOffset); + when(systemAdmins.getSystemAdmin(changelogSystemName)).thenReturn(systemAdmin); + when(systemAdmin.getSSPMetadata(eq(ImmutableSet.of(changelogSSP)))).thenReturn(ImmutableMap.of(changelogSSP, metadata)); + + // invoke the method + java.util.Map stateCheckpointMarkerMap = + tsm.getNewestChangelogSSPOffsets( + taskName, storeChangelogs, changelogPartition, systemAdmins); + + // verify results + assertEquals(1, stateCheckpointMarkerMap.size()); + KafkaStateCheckpointMarker kscm = KafkaStateCheckpointMarker.deserialize(stateCheckpointMarkerMap.get(storeName)); + assertEquals(newestChangelogSSPOffset, kscm.getChangelogOffset()); + assertEquals(changelogSSP, kscm.getChangelogSSP()); + } + + @Test + public void testGetNewestOffsetsReturnsNoneForEmptyTopic() { + // empty topic == null newest offset + ContainerStorageManager csm = mock(ContainerStorageManager.class); + KafkaTransactionalStateTaskBackupManager tsm = buildTSM(csm, mock(Partition.class), new StorageManagerUtil()); + + TaskName taskName = mock(TaskName.class); + String changelogSystemName = "systemName"; + String storeName = "storeName"; + String changelogStreamName = "changelogName"; + String newestChangelogSSPOffset = null; + SystemStream changelogSystemStream = new SystemStream(changelogSystemName, changelogStreamName); + Partition changelogPartition = new Partition(0); + SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); + + java.util.Map storeChangelogs = new HashMap(); + storeChangelogs.put(storeName, changelogSystemStream); + + SystemAdmins systemAdmins = mock(SystemAdmins.class); + SystemAdmin systemAdmin = mock(SystemAdmin.class); + SystemStreamPartitionMetadata metadata = mock(SystemStreamPartitionMetadata.class); + + when(metadata.getNewestOffset()).thenReturn(newestChangelogSSPOffset); + when(systemAdmins.getSystemAdmin(changelogSystemName)).thenReturn(systemAdmin); + when(systemAdmin.getSSPMetadata(eq(ImmutableSet.of(changelogSSP)))).thenReturn(ImmutableMap.of(changelogSSP, metadata)); + + // invoke the method + java.util.Map stateCheckpointMarkerMap = + tsm.getNewestChangelogSSPOffsets( + taskName, storeChangelogs, changelogPartition, systemAdmins); + + // verify results + assertEquals(1, stateCheckpointMarkerMap.size()); + KafkaStateCheckpointMarker kscm = KafkaStateCheckpointMarker.deserialize(stateCheckpointMarkerMap.get(storeName)); + assertEquals(changelogSSP, kscm.getChangelogSSP()); + assertNull(kscm.getChangelogOffset()); + } + + @Test(expected = SamzaException.class) + public void testGetNewestOffsetsThrowsIfNullMetadata() { + // empty topic == null newest offset + ContainerStorageManager csm = mock(ContainerStorageManager.class); + KafkaTransactionalStateTaskBackupManager tsm = buildTSM(csm, mock(Partition.class), new StorageManagerUtil()); + + TaskName taskName = mock(TaskName.class); + String changelogSystemName = "systemName"; + String storeName = "storeName"; + String changelogStreamName = "changelogName"; + String newestChangelogSSPOffset = null; + SystemStream changelogSystemStream = new SystemStream(changelogSystemName, changelogStreamName); + Partition changelogPartition = new Partition(0); + SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); + + java.util.Map storeChangelogs = new HashMap<>(); + storeChangelogs.put(storeName, changelogSystemStream); + + SystemAdmins systemAdmins = mock(SystemAdmins.class); + SystemAdmin systemAdmin = mock(SystemAdmin.class); + SystemStreamPartitionMetadata metadata = mock(SystemStreamPartitionMetadata.class); + + when(metadata.getNewestOffset()).thenReturn(newestChangelogSSPOffset); + when(systemAdmins.getSystemAdmin(changelogSystemName)).thenReturn(systemAdmin); + when(systemAdmin.getSSPMetadata(eq(ImmutableSet.of(changelogSSP)))).thenReturn(null); + + // invoke the method + java.util.Map offsets = + tsm.getNewestChangelogSSPOffsets( + taskName, storeChangelogs, changelogPartition, systemAdmins); + + // verify results + fail("Should have thrown an exception if admin didn't return any metadata"); + } + + @Test(expected = SamzaException.class) + public void testGetNewestOffsetsThrowsIfNullSSPMetadata() { + // empty topic == null newest offset + ContainerStorageManager csm = mock(ContainerStorageManager.class); + KafkaTransactionalStateTaskBackupManager tsm = buildTSM(csm, mock(Partition.class), new StorageManagerUtil()); + + TaskName taskName = mock(TaskName.class); + String changelogSystemName = "systemName"; + String storeName = "storeName"; + String changelogStreamName = "changelogName"; + String newestChangelogSSPOffset = null; + SystemStream changelogSystemStream = new SystemStream(changelogSystemName, changelogStreamName); + Partition changelogPartition = new Partition(0); + SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); + + java.util.Map storeChangelogs = new HashMap<>(); + storeChangelogs.put(storeName, changelogSystemStream); + + SystemAdmins systemAdmins = mock(SystemAdmins.class); + SystemAdmin systemAdmin = mock(SystemAdmin.class); + SystemStreamPartitionMetadata metadata = mock(SystemStreamPartitionMetadata.class); + + when(metadata.getNewestOffset()).thenReturn(newestChangelogSSPOffset); + when(systemAdmins.getSystemAdmin(changelogSystemName)).thenReturn(systemAdmin); + java.util.Map metadataMap = new HashMap() { { + put(changelogSSP, null); + } }; + when(systemAdmin.getSSPMetadata(eq(ImmutableSet.of(changelogSSP)))).thenReturn(metadataMap); + + // invoke the method + java.util.Map offsets = + tsm.getNewestChangelogSSPOffsets( + taskName, storeChangelogs, changelogPartition, systemAdmins); + + // verify results + fail("Should have thrown an exception if admin returned null metadata for changelog SSP"); + } + + @Test(expected = SamzaException.class) + public void testGetNewestOffsetsThrowsIfErrorGettingMetadata() { + // empty topic == null newest offset + ContainerStorageManager csm = mock(ContainerStorageManager.class); + KafkaTransactionalStateTaskBackupManager tsm = buildTSM(csm, mock(Partition.class), new StorageManagerUtil()); + + TaskName taskName = mock(TaskName.class); + String changelogSystemName = "systemName"; + String storeName = "storeName"; + String changelogStreamName = "changelogName"; + String newestChangelogSSPOffset = null; + SystemStream changelogSystemStream = new SystemStream(changelogSystemName, changelogStreamName); + Partition changelogPartition = new Partition(0); + SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); + + java.util.Map storeChangelogs = new HashMap<>(); + storeChangelogs.put(storeName, changelogSystemStream); + + SystemAdmins systemAdmins = mock(SystemAdmins.class); + SystemAdmin systemAdmin = mock(SystemAdmin.class); + SystemStreamPartitionMetadata metadata = mock(SystemStreamPartitionMetadata.class); + + when(metadata.getNewestOffset()).thenReturn(newestChangelogSSPOffset); + when(systemAdmins.getSystemAdmin(changelogSystemName)).thenThrow(new SamzaException("Error getting metadata")); + when(systemAdmin.getSSPMetadata(eq(ImmutableSet.of(changelogSSP)))).thenReturn(null); + + // invoke the method + java.util.Map offsets = + tsm.getNewestChangelogSSPOffsets( + taskName, storeChangelogs, changelogPartition, systemAdmins); + + // verify results + fail("Should have thrown an exception if admin had an error getting metadata"); + } + + private KafkaTransactionalStateTaskBackupManager buildTSM(ContainerStorageManager csm, Partition changelogPartition, + StorageManagerUtil smu) { + TaskName taskName = new TaskName("Partition 0"); + java.util.Map changelogSystemStreams = mock(java.util.Map.class); + SystemAdmins systemAdmins = mock(SystemAdmins.class); + + return new KafkaTransactionalStateTaskBackupManager( + taskName, changelogSystemStreams, systemAdmins, changelogPartition); + } +} \ No newline at end of file diff --git a/samza-test/src/test/java/org/apache/samza/storage/MyStatefulApplication.java b/samza-test/src/test/java/org/apache/samza/storage/MyStatefulApplication.java new file mode 100644 index 0000000000..99c54b0f0f --- /dev/null +++ b/samza-test/src/test/java/org/apache/samza/storage/MyStatefulApplication.java @@ -0,0 +1,157 @@ +/* + * 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.samza.storage; + +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 org.apache.samza.application.TaskApplication; +import org.apache.samza.application.descriptors.TaskApplicationDescriptor; +import org.apache.samza.context.Context; +import org.apache.samza.operators.KV; +import org.apache.samza.serializers.KVSerde; +import org.apache.samza.serializers.StringSerde; +import org.apache.samza.storage.kv.KeyValueIterator; +import org.apache.samza.storage.kv.KeyValueStore; +import org.apache.samza.storage.kv.descriptors.RocksDbTableDescriptor; +import org.apache.samza.system.IncomingMessageEnvelope; +import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor; +import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor; +import org.apache.samza.task.InitableTask; +import org.apache.samza.task.MessageCollector; +import org.apache.samza.task.StreamTask; +import org.apache.samza.task.StreamTaskFactory; +import org.apache.samza.task.TaskCoordinator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Stateful TaskApplication used for testing task store backup and restore behaviour. + * {@link #resetTestState()} should be invoked in @Before class of the test using this class. + * + * Input Message format: + * "num" -> put (key = num, value = num) and flush + * "-num" -> delete (key = num) and flush + * ":msg" -> act on msg (including flush) but no commit (may be num, shutdown or crash_once) + * "shutdown" -> always shutdown the job + * "crash_once" -> shut down the job the first time but ignore on subsequent run + */ +public class MyStatefulApplication implements TaskApplication { + + public static final Logger LOG = LoggerFactory.getLogger(MyStatefulApplication.class); + + private static Map> initialStoreContents = new HashMap<>(); + private static boolean crashedOnce = false; + private final String inputSystem; + private final String inputTopic; + private final Map storeToChangelog; + + public MyStatefulApplication(String inputSystem, String inputTopic, Map storeToChangelog) { + this.inputSystem = inputSystem; + this.inputTopic = inputTopic; + this.storeToChangelog = storeToChangelog; + } + + @Override + public void describe(TaskApplicationDescriptor appDescriptor) { + KafkaSystemDescriptor ksd = new KafkaSystemDescriptor(inputSystem); + KVSerde serde = KVSerde.of(new StringSerde(), new StringSerde()); + + KafkaInputDescriptor> isd = ksd.getInputDescriptor(inputTopic, serde); + + TaskApplicationDescriptor desc = appDescriptor + .withInputStream(isd) + .withTaskFactory((StreamTaskFactory) () -> new MyTask(storeToChangelog.keySet())); + + storeToChangelog.forEach((storeName, changelogTopic) -> { + RocksDbTableDescriptor td = new RocksDbTableDescriptor<>(storeName, serde) + .withChangelogStream(changelogTopic) + .withChangelogReplicationFactor(1); + desc.withTable(td); + }); + } + + public static void resetTestState() { + initialStoreContents = new HashMap<>(); + crashedOnce = false; + } + + public static Map> getInitialStoreContents() { + return initialStoreContents; + } + + static class MyTask implements StreamTask, InitableTask { + private final Set> stores = new HashSet<>(); + private final Set storeNames; + + MyTask(Set storeNames) { + this.storeNames = storeNames; + } + + @Override + public void init(Context context) { + storeNames.forEach(storeName -> { + KeyValueStore store = (KeyValueStore) context.getTaskContext().getStore(storeName); + stores.add(store); + KeyValueIterator storeEntries = store.all(); + List storeInitialChangelog = new ArrayList<>(); + while (storeEntries.hasNext()) { + storeInitialChangelog.add(storeEntries.next().getValue()); + } + initialStoreContents.put(storeName, storeInitialChangelog); + storeEntries.close(); + }); + } + + @Override + public void process(IncomingMessageEnvelope envelope, + MessageCollector collector, TaskCoordinator coordinator) { + String key = (String) envelope.getKey(); + LOG.info("Received key: {}", key); + + if (key.endsWith("crash_once")) { // endsWith allows :crash_once and crash_once + if (!crashedOnce) { + crashedOnce = true; + coordinator.shutdown(TaskCoordinator.RequestScope.CURRENT_TASK); + } else { + return; + } + } else if (key.endsWith("shutdown")) { + coordinator.shutdown(TaskCoordinator.RequestScope.CURRENT_TASK); + } else if (key.startsWith("-")) { + stores.forEach(store -> store.delete(key.substring(1))); + } else if (key.startsWith(":")) { + // write the message and flush, but don't invoke commit later + String msg = key.substring(1); + stores.forEach(store -> store.put(msg, msg)); + } else { + stores.forEach(store -> store.put(key, key)); + } + stores.forEach(KeyValueStore::flush); + + if (!key.startsWith(":")) { + coordinator.commit(TaskCoordinator.RequestScope.CURRENT_TASK); + } + } + } +} diff --git a/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateIntegrationTest.java index e9b12a1ac1..6f9427c079 100644 --- a/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateIntegrationTest.java +++ b/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateIntegrationTest.java @@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList; import java.io.File; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -31,25 +30,11 @@ import java.util.Map; import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.descriptors.TaskApplicationDescriptor; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.KafkaConfig; import org.apache.samza.config.TaskConfig; -import org.apache.samza.context.Context; -import org.apache.samza.operators.KV; -import org.apache.samza.serializers.KVSerde; -import org.apache.samza.serializers.StringSerde; -import org.apache.samza.storage.kv.descriptors.RocksDbTableDescriptor; -import org.apache.samza.system.IncomingMessageEnvelope; -import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor; -import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor; -import org.apache.samza.task.InitableTask; -import org.apache.samza.task.MessageCollector; -import org.apache.samza.task.StreamTask; -import org.apache.samza.task.StreamTaskFactory; -import org.apache.samza.task.TaskCoordinator; +import org.apache.samza.storage.MyStatefulApplication; import org.apache.samza.test.framework.StreamApplicationIntegrationTestHarness; import org.apache.samza.util.FileUtil; import org.junit.Assert; @@ -60,14 +45,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * Message format: - * "num" -> put (key = num, value = num) and flush - * "-num" -> delete (key = num) and flush - * ":msg" -> act on msg (including flush) but no commit (may be num, shutdown or crash_once) - * "shutdown" -> always shutdown the job - * "crash_once" -> shut down the job the first time but ignore on subsequent run - */ + @RunWith(value = Parameterized.class) public class TransactionalStateIntegrationTest extends StreamApplicationIntegrationTestHarness { @Parameterized.Parameters(name = "hostAffinity={0}") @@ -95,9 +73,6 @@ public static Collection data() { put(TaskConfig.COMMIT_MAX_DELAY_MS, "0"); // Ensure no commits are skipped due to in progress commits } }; - private static List actualInitialStoreContents = new ArrayList<>(); - private static boolean crashedOnce = false; - private final boolean hostAffinity; public TransactionalStateIntegrationTest(boolean hostAffinity) { @@ -109,43 +84,43 @@ public TransactionalStateIntegrationTest(boolean hostAffinity) { public void setUp() { super.setUp(); // reset static state shared with task between each parameterized iteration - crashedOnce = false; - actualInitialStoreContents = new ArrayList<>(); + MyStatefulApplication.resetTestState(); new FileUtil().rm(new File(LOGGED_STORE_BASE_DIR)); // always clear local store on startup } @Test public void testStopAndRestart() { List inputMessagesOnInitialRun = Arrays.asList("1", "2", "3", "2", "97", "-97", ":98", ":99", ":crash_once"); + // double check collectors.flush List expectedChangelogMessagesOnInitialRun = Arrays.asList("1", "2", "3", "2", "97", null, "98", "99"); initialRun(inputMessagesOnInitialRun, expectedChangelogMessagesOnInitialRun); - // first two are reverts for uncommitted messages from last run - List expectedChangelogMessagesOnSecondRun = + // first two are reverts for uncommitted messages from last run for keys 98 and 99 + List expectedChangelogMessagesAfterSecondRun = Arrays.asList(null, null, "98", "99", "4", "5", "5"); List expectedInitialStoreContentsOnSecondRun = Arrays.asList("1", "2", "3"); secondRun(CHANGELOG_TOPIC, - expectedChangelogMessagesOnSecondRun, expectedInitialStoreContentsOnSecondRun); + expectedChangelogMessagesAfterSecondRun, expectedInitialStoreContentsOnSecondRun, CONFIGS); } @Test public void testWithEmptyChangelogFromInitialRun() { // expected changelog messages will always match since we'll read 0 messages initialRun(ImmutableList.of("crash_once"), Collections.emptyList()); - secondRun(CHANGELOG_TOPIC, ImmutableList.of("4", "5", "5"), Collections.emptyList()); + secondRun(CHANGELOG_TOPIC, ImmutableList.of("4", "5", "5"), Collections.emptyList(), CONFIGS); } @Test public void testWithNewChangelogAfterInitialRun() { List inputMessagesOnInitialRun = Arrays.asList("1", "2", "3", "2", "97", "-97", ":98", ":99", ":crash_once"); - List expectedChangelogMessagesOnInitialRun = Arrays.asList("1", "2", "3", "2", "97", null, "98", "99"); - initialRun(inputMessagesOnInitialRun, expectedChangelogMessagesOnInitialRun); + List expectedChangelogMessagesAfterInitialRun = Arrays.asList("1", "2", "3", "2", "97", null, "98", "99"); + initialRun(inputMessagesOnInitialRun, expectedChangelogMessagesAfterInitialRun); // admin client delete topic doesn't seem to work, times out up to 60 seconds. // simulate delete topic by changing the changelog topic instead. String newChangelogTopic = "changelog2"; LOG.info("Changing changelog topic to: {}", newChangelogTopic); - secondRun(newChangelogTopic, ImmutableList.of("98", "99", "4", "5", "5"), Collections.emptyList()); + secondRun(newChangelogTopic, ImmutableList.of("98", "99", "4", "5", "5"), Collections.emptyList(), CONFIGS); } private void initialRun(List inputMessages, List expectedChangelogMessages) { @@ -162,7 +137,12 @@ private void initialRun(List inputMessages, List expectedChangel } // run the application - RunApplicationContext context = runApplication(new MyApplication(CHANGELOG_TOPIC), "myApp", CONFIGS); + RunApplicationContext context = runApplication( + new MyStatefulApplication(INPUT_SYSTEM, INPUT_TOPIC, Collections.singletonMap(STORE_NAME, CHANGELOG_TOPIC)), + "myApp", CONFIGS); + + // wait for the application to finish + context.getRunner().waitForFinish(); // consume and verify the changelog messages if (expectedChangelogMessages.size() > 0) { @@ -172,13 +152,11 @@ private void initialRun(List inputMessages, List expectedChangel Assert.assertEquals(expectedChangelogMessages, changelogMessages); } - // wait for the application to finish - context.getRunner().waitForFinish(); LOG.info("Finished initial run"); } private void secondRun(String changelogTopic, List expectedChangelogMessages, - List expectedInitialStoreContents) { + List expectedInitialStoreContents, Map overriddenConfigs) { // clear the local store directory if (!hostAffinity) { new FileUtil().rm(new File(LOGGED_STORE_BASE_DIR)); @@ -190,7 +168,9 @@ private void secondRun(String changelogTopic, List expectedChangelogMess inputMessages.forEach(m -> produceMessage(INPUT_TOPIC, 0, m, m)); // run the application - RunApplicationContext context = runApplication(new MyApplication(changelogTopic), "myApp", CONFIGS); + RunApplicationContext context = runApplication( + new MyStatefulApplication(INPUT_SYSTEM, INPUT_TOPIC, Collections.singletonMap(STORE_NAME, changelogTopic)), + "myApp", overriddenConfigs); // wait for the application to finish context.getRunner().waitForFinish(); @@ -202,76 +182,6 @@ private void secondRun(String changelogTopic, List expectedChangelogMess Assert.assertEquals(expectedChangelogMessages, changelogMessages); // verify the store contents during startup (this is after changelog verification to ensure init has completed) - Assert.assertEquals(expectedInitialStoreContents, actualInitialStoreContents); - } - - static class MyApplication implements TaskApplication { - private final String changelogTopic; - - public MyApplication(String changelogTopic) { - this.changelogTopic = changelogTopic; - } - - @Override - public void describe(TaskApplicationDescriptor appDescriptor) { - KafkaSystemDescriptor ksd = new KafkaSystemDescriptor(INPUT_SYSTEM); - KVSerde serde = KVSerde.of(new StringSerde(), new StringSerde()); - - KafkaInputDescriptor> isd = ksd.getInputDescriptor(INPUT_TOPIC, serde); - - RocksDbTableDescriptor td = new RocksDbTableDescriptor<>(STORE_NAME, serde) - .withChangelogStream(changelogTopic) - .withChangelogReplicationFactor(1); - - appDescriptor - .withInputStream(isd) - .withTaskFactory((StreamTaskFactory) () -> new MyTask()) - .withTable(td); - } - } - - static class MyTask implements StreamTask, InitableTask { - private KeyValueStore store; - - @Override - public void init(Context context) { - this.store = (KeyValueStore) context.getTaskContext().getStore(STORE_NAME); - KeyValueIterator storeEntries = store.all(); - while (storeEntries.hasNext()) { - actualInitialStoreContents.add(storeEntries.next().getValue()); - } - storeEntries.close(); - } - - @Override - public void process(IncomingMessageEnvelope envelope, - MessageCollector collector, TaskCoordinator coordinator) { - String key = (String) envelope.getKey(); - LOG.info("Received key: {}", key); - - if (key.endsWith("crash_once")) { // endsWith allows :crash_once and crash_once - if (!crashedOnce) { - crashedOnce = true; - coordinator.shutdown(TaskCoordinator.RequestScope.CURRENT_TASK); - } else { - return; - } - } else if (key.endsWith("shutdown")) { - coordinator.shutdown(TaskCoordinator.RequestScope.CURRENT_TASK); - } else if (key.startsWith("-")) { - store.delete(key.substring(1)); - } else if (key.startsWith(":")) { - // write the message and flush, but don't invoke commit later - String msg = key.substring(1); - store.put(msg, msg); - } else { - store.put(key, key); - } - store.flush(); - - if (!key.startsWith(":")) { - coordinator.commit(TaskCoordinator.RequestScope.CURRENT_TASK); - } - } + Assert.assertEquals(expectedInitialStoreContents, MyStatefulApplication.getInitialStoreContents().get(STORE_NAME)); } } \ No newline at end of file diff --git a/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateMultiStoreIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateMultiStoreIntegrationTest.java index d50d6bf72f..e0fbaca6e3 100644 --- a/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateMultiStoreIntegrationTest.java +++ b/samza-test/src/test/java/org/apache/samza/storage/kv/TransactionalStateMultiStoreIntegrationTest.java @@ -21,8 +21,8 @@ import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import java.io.File; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -31,25 +31,12 @@ import java.util.Map; import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.samza.application.TaskApplication; -import org.apache.samza.application.descriptors.TaskApplicationDescriptor; +import org.apache.samza.application.SamzaApplication; import org.apache.samza.config.JobConfig; import org.apache.samza.config.JobCoordinatorConfig; import org.apache.samza.config.KafkaConfig; import org.apache.samza.config.TaskConfig; -import org.apache.samza.context.Context; -import org.apache.samza.operators.KV; -import org.apache.samza.serializers.KVSerde; -import org.apache.samza.serializers.StringSerde; -import org.apache.samza.storage.kv.descriptors.RocksDbTableDescriptor; -import org.apache.samza.system.IncomingMessageEnvelope; -import org.apache.samza.system.kafka.descriptors.KafkaInputDescriptor; -import org.apache.samza.system.kafka.descriptors.KafkaSystemDescriptor; -import org.apache.samza.task.InitableTask; -import org.apache.samza.task.MessageCollector; -import org.apache.samza.task.StreamTask; -import org.apache.samza.task.StreamTaskFactory; -import org.apache.samza.task.TaskCoordinator; +import org.apache.samza.storage.MyStatefulApplication; import org.apache.samza.test.framework.StreamApplicationIntegrationTestHarness; import org.apache.samza.util.FileUtil; import org.junit.Assert; @@ -80,6 +67,7 @@ public static Collection data() { private static final String STORE_2_NAME = "store2"; private static final String STORE_1_CHANGELOG = "changelog1"; private static final String STORE_2_CHANGELOG = "changelog2"; + private static final String APP_NAME = "myApp"; private static final String LOGGED_STORE_BASE_DIR = new File(System.getProperty("java.io.tmpdir"), "logged-store").getAbsolutePath(); private static final Map CONFIGS = new HashMap() { { put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, "org.apache.samza.standalone.PassthroughJobCoordinatorFactory"); @@ -94,9 +82,6 @@ public static Collection data() { put(TaskConfig.COMMIT_MAX_DELAY_MS, "0"); // Ensure no commits are skipped due to in progress commits } }; - private static List actualInitialStoreContents = new ArrayList<>(); - private static boolean crashedOnce = false; - private final boolean hostAffinity; public TransactionalStateMultiStoreIntegrationTest(boolean hostAffinity) { @@ -108,8 +93,7 @@ public TransactionalStateMultiStoreIntegrationTest(boolean hostAffinity) { public void setUp() { super.setUp(); // reset static state shared with task between each parameterized iteration - crashedOnce = false; - actualInitialStoreContents = new ArrayList<>(); + MyStatefulApplication.resetTestState(); new FileUtil().rm(new File(LOGGED_STORE_BASE_DIR)); // always clear local store on startup } @@ -119,7 +103,7 @@ public void testStopAndRestart() { List expectedChangelogMessagesOnInitialRun = Arrays.asList("1", "2", "3", "2", "97", null, "98", "99"); initialRun(inputMessagesOnInitialRun, expectedChangelogMessagesOnInitialRun); - // first two are reverts for uncommitted messages from last run + // first two are reverts for uncommitted messages from last run for keys 98 and 99 List expectedChangelogMessagesOnSecondRun = Arrays.asList(null, null, "98", "99", "4", "5", "5"); List expectedInitialStoreContentsOnSecondRun = Arrays.asList("1", "2", "3"); @@ -160,8 +144,14 @@ private void initialRun(List inputMessages, List expectedChangel Assert.assertEquals(inputMessages, readInputMessages); } + SamzaApplication app = new MyStatefulApplication(INPUT_SYSTEM, INPUT_TOPIC, ImmutableMap.of( + STORE_1_NAME, STORE_1_CHANGELOG, + STORE_2_NAME, STORE_2_CHANGELOG + )); + // run the application - RunApplicationContext context = runApplication(new MyApplication(STORE_1_CHANGELOG), "myApp", CONFIGS); + RunApplicationContext context = runApplication(app, APP_NAME, CONFIGS); + // consume and verify the changelog messages if (expectedChangelogMessages.size() > 0) { @@ -188,8 +178,12 @@ private void secondRun(String changelogTopic, List expectedChangelogMess List inputMessages = Arrays.asList("4", "5", "5", ":shutdown"); inputMessages.forEach(m -> produceMessage(INPUT_TOPIC, 0, m, m)); + SamzaApplication app = new MyStatefulApplication(INPUT_SYSTEM, INPUT_TOPIC, ImmutableMap.of( + STORE_1_NAME, changelogTopic, + STORE_2_NAME, STORE_2_CHANGELOG + )); // run the application - RunApplicationContext context = runApplication(new MyApplication(changelogTopic), "myApp", CONFIGS); + RunApplicationContext context = runApplication(app, APP_NAME, CONFIGS); // wait for the application to finish context.getRunner().waitForFinish(); @@ -201,81 +195,6 @@ private void secondRun(String changelogTopic, List expectedChangelogMess Assert.assertEquals(expectedChangelogMessages, changelogMessages); // verify the store contents during startup (this is after changelog verification to ensure init has completed) - Assert.assertEquals(expectedInitialStoreContents, actualInitialStoreContents); - } - - static class MyApplication implements TaskApplication { - private final String changelogTopic; - - public MyApplication(String changelogTopic) { - this.changelogTopic = changelogTopic; - } - - @Override - public void describe(TaskApplicationDescriptor appDescriptor) { - KafkaSystemDescriptor ksd = new KafkaSystemDescriptor(INPUT_SYSTEM); - KVSerde serde = KVSerde.of(new StringSerde(), new StringSerde()); - - KafkaInputDescriptor> isd = ksd.getInputDescriptor(INPUT_TOPIC, serde); - - RocksDbTableDescriptor td1 = new RocksDbTableDescriptor<>(STORE_1_NAME, serde) - .withChangelogStream(changelogTopic) - .withChangelogReplicationFactor(1); - - RocksDbTableDescriptor td2 = new RocksDbTableDescriptor<>(STORE_2_NAME, serde) - .withChangelogStream(STORE_2_CHANGELOG) - .withChangelogReplicationFactor(1); - - appDescriptor - .withInputStream(isd) - .withTaskFactory((StreamTaskFactory) () -> new MyTask()) - .withTable(td1) - .withTable(td2); - } - } - - static class MyTask implements StreamTask, InitableTask { - private KeyValueStore store; - - @Override - public void init(Context context) { - this.store = (KeyValueStore) context.getTaskContext().getStore(STORE_1_NAME); - KeyValueIterator storeEntries = store.all(); - while (storeEntries.hasNext()) { - actualInitialStoreContents.add(storeEntries.next().getValue()); - } - storeEntries.close(); - } - - @Override - public void process(IncomingMessageEnvelope envelope, - MessageCollector collector, TaskCoordinator coordinator) { - String key = (String) envelope.getKey(); - LOG.info("Received key: {}", key); - - if (key.endsWith("crash_once")) { // endsWith allows :crash_once and crash_once - if (!crashedOnce) { - crashedOnce = true; - coordinator.shutdown(TaskCoordinator.RequestScope.CURRENT_TASK); - } else { - return; - } - } else if (key.endsWith("shutdown")) { - coordinator.shutdown(TaskCoordinator.RequestScope.CURRENT_TASK); - } else if (key.startsWith("-")) { - store.delete(key.substring(1)); - } else if (key.startsWith(":")) { - // write the message and flush, but don't invoke commit later - String msg = key.substring(1); - store.put(msg, msg); - } else { - store.put(key, key); - } - store.flush(); - - if (!key.startsWith(":")) { - coordinator.commit(TaskCoordinator.RequestScope.CURRENT_TASK); - } - } + Assert.assertEquals(expectedInitialStoreContents, MyStatefulApplication.getInitialStoreContents().get(STORE_1_NAME)); } } \ No newline at end of file From c85aade351a3e7dea2a699a5836968a9fc7ec016 Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Wed, 12 May 2021 16:47:09 -0700 Subject: [PATCH 03/20] SAMZA-2591: Async Commit [3/3]: Container restore lifecycle (#1491) - Refactor ContainerStorageManager lifecycle for restores --- .../apache/samza/checkpoint/CheckpointId.java | 2 +- .../org/apache/samza/job/model/JobModel.java | 4 + .../storage/KafkaChangelogRestoreParams.java | 85 +++++ .../samza/storage/StateBackendFactory.java | 2 +- .../apache/samza/storage/StorageEngine.java | 8 + .../apache/samza/storage/StoreProperties.java | 23 +- .../samza/storage/kv/KeyValueStore.java | 13 + .../samza/system/ChangelogSSPIterator.java | 2 +- .../apache/samza/config/StorageConfig.java | 5 +- .../coordinator/MetadataResourceUtil.java | 2 +- .../samza/serializers/CheckpointV2Serde.java | 6 +- .../model/JsonCheckpointV2Mixin.java | 3 - .../KafkaChangelogStateBackendFactory.java | 2 +- ...nTransactionalStateTaskRestoreManager.java | 80 +++-- .../apache/samza/storage/StorageRecovery.java | 15 +- .../samza/storage/TaskRestoreManager.java | 55 --- .../storage/TaskRestoreManagerFactory.java | 82 ----- .../storage/TaskSideInputStorageManager.java | 1 + .../TransactionalStateTaskRestoreManager.java | 127 ++++++- .../org/apache/samza/system/SystemAdmins.java | 8 + .../storage/ContainerStorageManager.java | 329 +++++++++--------- .../storage/TaskStorageManagerFactory.java | 46 --- .../apache/samza/job/model/TestJobModel.java | 2 +- ...tTransactionalStateTaskRestoreManager.java | 252 +++++++------- .../storage/TestContainerStorageManager.java | 34 +- .../InMemoryKeyValueStorageEngineFactory.java | 2 - .../RocksDbKeyValueStorageEngineFactory.scala | 2 - .../storage/kv/RocksDbKeyValueStore.scala | 9 +- .../storage/kv/TestRocksDbKeyValueStore.scala | 20 +- .../kv/BaseKeyValueStorageEngineFactory.java | 9 +- .../kv/MockKeyValueStorageEngineFactory.java | 4 +- .../TestBaseKeyValueStorageEngineFactory.java | 37 +- .../TestZkLocalApplicationRunner.java | 4 +- ...NonTransactionalStateIntegrationTest.scala | 5 +- .../test/integration/StreamTaskTestUtil.scala | 8 +- 35 files changed, 717 insertions(+), 571 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/storage/KafkaChangelogRestoreParams.java delete mode 100644 samza-core/src/main/java/org/apache/samza/storage/TaskRestoreManager.java delete mode 100644 samza-core/src/main/java/org/apache/samza/storage/TaskRestoreManagerFactory.java delete mode 100644 samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManagerFactory.java diff --git a/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointId.java b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointId.java index f2aa8e1f30..a4d4ca33bb 100644 --- a/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointId.java +++ b/samza-api/src/main/java/org/apache/samza/checkpoint/CheckpointId.java @@ -92,7 +92,7 @@ public int hashCode() { @Override public int compareTo(CheckpointId that) { - if(this.millis != that.millis) return Long.compare(this.millis, that.millis); + if (this.millis != that.millis) return Long.compare(this.millis, that.millis); else return Long.compare(this.nanoId, that.nanoId); } diff --git a/samza-api/src/main/java/org/apache/samza/job/model/JobModel.java b/samza-api/src/main/java/org/apache/samza/job/model/JobModel.java index d1f5e724fa..4bd995b27d 100644 --- a/samza-api/src/main/java/org/apache/samza/job/model/JobModel.java +++ b/samza-api/src/main/java/org/apache/samza/job/model/JobModel.java @@ -57,6 +57,10 @@ public JobModel(Config config, Map containers) { } } + public int getMaxChangeLogStreamPartitions() { + return maxChangeLogStreamPartitions; + } + public Config getConfig() { return config; } diff --git a/samza-api/src/main/java/org/apache/samza/storage/KafkaChangelogRestoreParams.java b/samza-api/src/main/java/org/apache/samza/storage/KafkaChangelogRestoreParams.java new file mode 100644 index 0000000000..961a85bb93 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/storage/KafkaChangelogRestoreParams.java @@ -0,0 +1,85 @@ +/* + * 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.samza.storage; + +import java.util.Map; +import java.util.Set; +import org.apache.samza.serializers.Serde; +import org.apache.samza.system.SystemAdmin; +import org.apache.samza.system.SystemConsumer; +import org.apache.samza.task.MessageCollector; + +/** + * Provides the required for Kafka Changelog restore managers + */ +public class KafkaChangelogRestoreParams { + private final Map storeConsumers; + private final Map inMemoryStores; + private final Map systemAdmins; + private final Map> storageEngineFactories; + private final Map> serdes; + private final MessageCollector collector; + private final Set storeNames; + + public KafkaChangelogRestoreParams( + Map storeConsumers, + Map inMemoryStores, + Map systemAdmins, + Map> storageEngineFactories, + Map> serdes, + MessageCollector collector, + Set storeNames) { + this.storeConsumers = storeConsumers; + this.inMemoryStores = inMemoryStores; + this.systemAdmins = systemAdmins; + this.storageEngineFactories = storageEngineFactories; + this.serdes = serdes; + this.collector = collector; + this.storeNames = storeNames; + } + + public Map getStoreConsumers() { + return storeConsumers; + } + + public Map getInMemoryStores() { + return inMemoryStores; + } + + public Map getSystemAdmins() { + return systemAdmins; + } + + public Map> getStorageEngineFactories() { + return storageEngineFactories; + } + + public Map> getSerdes() { + return serdes; + } + + public MessageCollector getCollector() { + return collector; + } + + public Set getStoreNames() { + return storeNames; + } +} diff --git a/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java b/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java index 9946d2a6ac..54a9a81b91 100644 --- a/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java +++ b/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java @@ -33,7 +33,7 @@ * Factory to build the Samza {@link TaskBackupManager}, {@link TaskRestoreManager} and {@link TaskStorageAdmin} * for a particular state storage backend, which are used to durably backup the Samza task state. */ - public interface StateBackendFactory { +public interface StateBackendFactory { TaskBackupManager getBackupManager(JobContext jobContext, ContainerContext containerContext, TaskModel taskModel, diff --git a/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java b/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java index 7b12c85eee..0d62ca4c10 100644 --- a/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java +++ b/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java @@ -24,6 +24,9 @@ import java.util.Optional; import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.context.ContainerContext; +import org.apache.samza.context.ExternalContext; +import org.apache.samza.context.JobContext; import org.apache.samza.system.ChangelogSSPIterator; /** @@ -38,6 +41,11 @@ */ public interface StorageEngine { + /** + * Initialize the storage engine + */ + default void init(ExternalContext externalContext, JobContext jobContext, ContainerContext containerContext) { }; + /** * Restore the content of this StorageEngine from the changelog. Messages are * provided in one {@link java.util.Iterator} and not deserialized for diff --git a/samza-api/src/main/java/org/apache/samza/storage/StoreProperties.java b/samza-api/src/main/java/org/apache/samza/storage/StoreProperties.java index a398271e4c..1244adfe17 100644 --- a/samza-api/src/main/java/org/apache/samza/storage/StoreProperties.java +++ b/samza-api/src/main/java/org/apache/samza/storage/StoreProperties.java @@ -24,12 +24,15 @@ public class StoreProperties { private final boolean persistedToDisk; private final boolean loggedStore; + private final boolean durable; private StoreProperties( final boolean persistedToDisk, - final boolean loggedStore) { + final boolean loggedStore, + final boolean durable) { this.persistedToDisk = persistedToDisk; this.loggedStore = loggedStore; + this.durable = durable; } /** @@ -50,9 +53,20 @@ public boolean isLoggedStore() { return loggedStore; } + /** + * Flag to indicate whether a store is durable, that is, it's contents are available cross container restarts + * or host reallocation. + * + * @return True, if the store is durable. False by default. + */ + public boolean isDurableStore() { + return durable; + } + public static class StorePropertiesBuilder { private boolean persistedToDisk = false; private boolean loggedStore = false; + private boolean durable = false; public StorePropertiesBuilder setPersistedToDisk(boolean persistedToDisk) { this.persistedToDisk = persistedToDisk; @@ -64,8 +78,13 @@ public StorePropertiesBuilder setLoggedStore(boolean loggedStore) { return this; } + public StorePropertiesBuilder setIsDurable(boolean durable) { + this.durable = durable; + return this; + } + public StoreProperties build() { - return new StoreProperties(persistedToDisk, loggedStore); + return new StoreProperties(persistedToDisk, loggedStore, durable); } } } diff --git a/samza-api/src/main/java/org/apache/samza/storage/kv/KeyValueStore.java b/samza-api/src/main/java/org/apache/samza/storage/kv/KeyValueStore.java index 41faac352e..a3552f08b9 100644 --- a/samza-api/src/main/java/org/apache/samza/storage/kv/KeyValueStore.java +++ b/samza-api/src/main/java/org/apache/samza/storage/kv/KeyValueStore.java @@ -26,6 +26,9 @@ import java.util.Optional; import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.context.ContainerContext; +import org.apache.samza.context.ExternalContext; +import org.apache.samza.context.JobContext; /** @@ -35,6 +38,16 @@ * @param the type of values maintained by this key-value store. */ public interface KeyValueStore { + + /** + * Initializes the KeyValueStore + * + * @param externalContext any external store required for initialization + * @param jobContext context of the job the KeyValueStore is in + * @param containerContext context of the KeyValueStore's container + */ + default void init(ExternalContext externalContext, JobContext jobContext, ContainerContext containerContext) { } + /** * Gets the value associated with the specified {@code key}. * diff --git a/samza-api/src/main/java/org/apache/samza/system/ChangelogSSPIterator.java b/samza-api/src/main/java/org/apache/samza/system/ChangelogSSPIterator.java index ea44b9d118..8e5bc93834 100644 --- a/samza-api/src/main/java/org/apache/samza/system/ChangelogSSPIterator.java +++ b/samza-api/src/main/java/org/apache/samza/system/ChangelogSSPIterator.java @@ -61,7 +61,7 @@ public ChangelogSSPIterator(SystemConsumer systemConsumer, SystemStreamPartition public IncomingMessageEnvelope next() { IncomingMessageEnvelope envelope = super.next(); - // if trimming changelog is enabled, then switch to trim mode if if we've consumed past the restore offset + // if trimming changelog is enabled, then switch to trim mode if we've consumed past the restore offset // (i.e., restoreOffset was null or current offset is > restoreOffset) if (this.trimEnabled && (restoreOffset == null || admin.offsetComparator(envelope.getOffset(), restoreOffset) > 0)) { mode = Mode.TRIM; diff --git a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java index f5e2055e3c..4dd753e370 100644 --- a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java @@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList; import java.util.ArrayList; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -72,6 +71,8 @@ public class StorageConfig extends MapConfig { public static final List DEFAULT_STATE_BACKEND_BACKUP_FACTORIES = ImmutableList.of( DEFAULT_STATE_BACKEND_FACTORY); public static final String STATE_BACKEND_RESTORE_FACTORY = STORE_PREFIX + "state.restore.backend"; + public static final String INMEMORY_KV_STORAGE_ENGINE_FACTORY = + "org.apache.samza.storage.kv.inmemory.InMemoryKeyValueStorageEngineFactory"; static final String CHANGELOG_SYSTEM = "job.changelog.system"; static final String CHANGELOG_DELETE_RETENTION_MS = STORE_PREFIX + "%s.changelog.delete.retention.ms"; @@ -83,8 +84,6 @@ public class StorageConfig extends MapConfig { static final String SIDE_INPUTS_PROCESSOR_FACTORY = STORE_PREFIX + "%s" + SIDE_INPUT_PROCESSOR_FACTORY_SUFFIX; static final String SIDE_INPUTS_PROCESSOR_SERIALIZED_INSTANCE = STORE_PREFIX + "%s.side.inputs.processor.serialized.instance"; - static final String INMEMORY_KV_STORAGE_ENGINE_FACTORY = - "org.apache.samza.storage.kv.inmemory.InMemoryKeyValueStorageEngineFactory"; // Internal config to clean storeDirs of a store on container start. This is used to benchmark bootstrap performance. static final String CLEAN_LOGGED_STOREDIRS_ON_START = STORE_PREFIX + "%s.clean.on.container.start"; diff --git a/samza-core/src/main/java/org/apache/samza/coordinator/MetadataResourceUtil.java b/samza-core/src/main/java/org/apache/samza/coordinator/MetadataResourceUtil.java index 1050662920..9e366ea3c6 100644 --- a/samza-core/src/main/java/org/apache/samza/coordinator/MetadataResourceUtil.java +++ b/samza-core/src/main/java/org/apache/samza/coordinator/MetadataResourceUtil.java @@ -61,7 +61,7 @@ public void createResources() { @VisibleForTesting void createChangelogStreams() { - ChangelogStreamManager.createChangelogStreams(config, jobModel.maxChangeLogStreamPartitions); + ChangelogStreamManager.createChangelogStreams(config, jobModel.getMaxChangeLogStreamPartitions()); } @VisibleForTesting diff --git a/samza-core/src/main/java/org/apache/samza/serializers/CheckpointV2Serde.java b/samza-core/src/main/java/org/apache/samza/serializers/CheckpointV2Serde.java index 10e6b3d0b9..48aa564e32 100644 --- a/samza-core/src/main/java/org/apache/samza/serializers/CheckpointV2Serde.java +++ b/samza-core/src/main/java/org/apache/samza/serializers/CheckpointV2Serde.java @@ -21,15 +21,13 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.samza.SamzaException; -import org.apache.samza.checkpoint.CheckpointId; import org.apache.samza.checkpoint.CheckpointV2; import org.apache.samza.serializers.model.SamzaObjectMapper; -import org.apache.samza.system.SystemStreamPartition; /** - * The {@link Serde} for {@link CheckpointV2} which includes {@link CheckpointId}s, state checkpoint markers - * and the input {@link SystemStreamPartition} offsets. + * The {@link Serde} for {@link CheckpointV2} which includes {@link org.apache.samza.checkpoint.CheckpointId}s, + * state checkpoint markers and the input {@link org.apache.samza.system.SystemStreamPartition} offsets. * * The overall payload is serde'd as JSON using {@link SamzaObjectMapper}. Since the Samza classes cannot be directly * serialized by Jackson using {@link org.apache.samza.serializers.model.JsonCheckpointV2Mixin}. diff --git a/samza-core/src/main/java/org/apache/samza/serializers/model/JsonCheckpointV2Mixin.java b/samza-core/src/main/java/org/apache/samza/serializers/model/JsonCheckpointV2Mixin.java index 8e26745a82..9edd234319 100644 --- a/samza-core/src/main/java/org/apache/samza/serializers/model/JsonCheckpointV2Mixin.java +++ b/samza-core/src/main/java/org/apache/samza/serializers/model/JsonCheckpointV2Mixin.java @@ -23,10 +23,7 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -import java.util.Set; -import org.apache.samza.Partition; import org.apache.samza.checkpoint.CheckpointId; -import org.apache.samza.container.TaskName; import org.apache.samza.system.SystemStreamPartition; @JsonIgnoreProperties(ignoreUnknown = true) diff --git a/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java b/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java index e3230f6306..36ea5b5923 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java +++ b/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java @@ -202,7 +202,7 @@ Map filterStandbySystemStreams(Map c Map changelogSSPToStore = new HashMap<>(); changelogSystemStreams.forEach((storeName, systemStream) -> containerModel.getTasks().forEach((taskName, taskModel) -> { - if (TaskMode.Standby.equals(taskModel.getTaskMode())) { + if (!TaskMode.Standby.equals(taskModel.getTaskMode())) { changelogSSPToStore.put(new SystemStreamPartition(systemStream, taskModel.getChangelogPartition()), storeName); } diff --git a/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java index 44dd59a93f..f648b0651d 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java @@ -28,9 +28,14 @@ import java.util.stream.Collectors; import org.apache.samza.Partition; import org.apache.samza.SamzaException; +import org.apache.samza.checkpoint.Checkpoint; import org.apache.samza.config.Config; import org.apache.samza.config.StorageConfig; +import org.apache.samza.context.ContainerContext; +import org.apache.samza.context.JobContext; import org.apache.samza.job.model.TaskModel; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.serializers.Serde; import org.apache.samza.system.ChangelogSSPIterator; import org.apache.samza.system.StreamMetadataCache; import org.apache.samza.system.StreamSpec; @@ -40,6 +45,7 @@ import org.apache.samza.system.SystemStream; import org.apache.samza.system.SystemStreamMetadata; import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.task.MessageCollector; import org.apache.samza.util.Clock; import org.apache.samza.util.FileUtil; import org.slf4j.Logger; @@ -64,44 +70,52 @@ class NonTransactionalStateTaskRestoreManager implements TaskRestoreManager { private final Clock clock; // Clock value used to validate base-directories for staleness. See isLoggedStoreValid. private Map changeLogOldestOffsets; // Map of changelog oldest known offsets private final Map fileOffsets; // Map of offsets read from offset file indexed by changelog SSP - private final Map changelogSystemStreams; // Map of change log system-streams indexed by store name + private final Map storeChangelogs; // Map of change log system-streams indexed by store name private final SystemAdmins systemAdmins; private final File loggedStoreBaseDirectory; private final File nonLoggedStoreBaseDirectory; private final StreamMetadataCache streamMetadataCache; private final Map storeConsumers; private final int maxChangeLogStreamPartitions; - private final StorageConfig storageConfig; + private final Config config; private final StorageManagerUtil storageManagerUtil; NonTransactionalStateTaskRestoreManager( + Set storeNames, + JobContext jobContext, + ContainerContext containerContext, TaskModel taskModel, - Map changelogSystemStreams, - Map taskStores, + Map storeChangelogs, + Map inMemoryStores, + Map> storageEngineFactories, + Map> serdes, SystemAdmins systemAdmins, StreamMetadataCache streamMetadataCache, Map storeConsumers, + MetricsRegistry metricsRegistry, + MessageCollector messageCollector, int maxChangeLogStreamPartitions, File loggedStoreBaseDirectory, File nonLoggedStoreBaseDirectory, Config config, Clock clock) { - this.taskStores = taskStores; this.taskModel = taskModel; this.clock = clock; - this.changelogSystemStreams = changelogSystemStreams; + this.storeChangelogs = storeChangelogs; this.systemAdmins = systemAdmins; this.fileOffsets = new HashMap<>(); - this.taskStoresToRestore = this.taskStores.entrySet().stream() - .filter(x -> x.getValue().getStoreProperties().isLoggedStore()) - .map(x -> x.getKey()).collect(Collectors.toSet()); this.loggedStoreBaseDirectory = loggedStoreBaseDirectory; this.nonLoggedStoreBaseDirectory = nonLoggedStoreBaseDirectory; this.streamMetadataCache = streamMetadataCache; this.storeConsumers = storeConsumers; this.maxChangeLogStreamPartitions = maxChangeLogStreamPartitions; - this.storageConfig = new StorageConfig(config); + this.config = config; this.storageManagerUtil = new StorageManagerUtil(); + this.taskStores = createStoreEngines(storeNames, jobContext, containerContext, + storageEngineFactories, serdes, metricsRegistry, messageCollector, inMemoryStores); + this.taskStoresToRestore = this.taskStores.entrySet().stream() + .filter(x -> x.getValue().getStoreProperties().isLoggedStore()) + .map(x -> x.getKey()).collect(Collectors.toSet()); } /** @@ -109,7 +123,7 @@ class NonTransactionalStateTaskRestoreManager implements TaskRestoreManager { * and registers SSPs with the respective consumers. */ @Override - public void init(Map checkpointedChangelogSSPOffsets) { + public void init(Checkpoint checkpoint) { cleanBaseDirsAndReadOffsetFiles(); setupBaseDirs(); validateChangelogStreams(); @@ -124,7 +138,7 @@ public void init(Map checkpointedChangelogSSPOffs */ private void cleanBaseDirsAndReadOffsetFiles() { LOG.debug("Cleaning base directories for stores."); - + StorageConfig storageConfig = new StorageConfig(config); FileUtil fileUtil = new FileUtil(); taskStores.forEach((storeName, storageEngine) -> { if (!storageEngine.getStoreProperties().isLoggedStore()) { @@ -147,7 +161,7 @@ private void cleanBaseDirsAndReadOffsetFiles() { fileUtil.rm(loggedStorePartitionDir); } else { - SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStreams.get(storeName), taskModel.getChangelogPartition()); + SystemStreamPartition changelogSSP = new SystemStreamPartition(storeChangelogs.get(storeName), taskModel.getChangelogPartition()); Map offset = storageManagerUtil.readOffsetFile(loggedStorePartitionDir, Collections.singleton(changelogSSP), false); LOG.info("Read offset {} for the store {} from logged storage partition directory {}", offset, storeName, loggedStorePartitionDir); @@ -170,10 +184,10 @@ private void cleanBaseDirsAndReadOffsetFiles() { * @return true if the logged store is valid, false otherwise. */ private boolean isLoggedStoreValid(String storeName, File loggedStoreDir) { - long changeLogDeleteRetentionInMs = storageConfig.getChangeLogDeleteRetentionInMs(storeName); + long changeLogDeleteRetentionInMs = new StorageConfig(config).getChangeLogDeleteRetentionInMs(storeName); - if (changelogSystemStreams.containsKey(storeName)) { - SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStreams.get(storeName), taskModel.getChangelogPartition()); + if (storeChangelogs.containsKey(storeName)) { + SystemStreamPartition changelogSSP = new SystemStreamPartition(storeChangelogs.get(storeName), taskModel.getChangelogPartition()); return this.taskStores.get(storeName).getStoreProperties().isPersistedToDisk() && storageManagerUtil.isOffsetFileValid(loggedStoreDir, Collections.singleton(changelogSSP), false) && !storageManagerUtil.isStaleStore(loggedStoreDir, changeLogDeleteRetentionInMs, clock.currentTimeMillis(), false); @@ -213,9 +227,9 @@ private void setupBaseDirs() { * Validates each changelog system-stream with its respective SystemAdmin. */ private void validateChangelogStreams() { - LOG.info("Validating change log streams: " + changelogSystemStreams); + LOG.info("Validating change log streams: " + storeChangelogs); - for (SystemStream changelogSystemStream : changelogSystemStreams.values()) { + for (SystemStream changelogSystemStream : storeChangelogs.values()) { SystemAdmin systemAdmin = systemAdmins.getSystemAdmin(changelogSystemStream.getSystem()); StreamSpec changelogSpec = StreamSpec.createChangeLogStreamSpec(changelogSystemStream.getStream(), changelogSystemStream.getSystem(), @@ -232,7 +246,7 @@ private void getOldestChangeLogOffsets() { Map changeLogMetadata = JavaConverters.mapAsJavaMapConverter( streamMetadataCache.getStreamMetadata( - JavaConverters.asScalaSetConverter(new HashSet<>(changelogSystemStreams.values())).asScala().toSet(), + JavaConverters.asScalaSetConverter(new HashSet<>(storeChangelogs.values())).asScala().toSet(), false)).asJava(); LOG.info("Got change log stream metadata: {}", changeLogMetadata); @@ -267,7 +281,7 @@ private Map getChangeLogOldestOffsetsForPartition(Partitio */ private void registerStartingOffsets() { - for (Map.Entry changelogSystemStreamEntry : changelogSystemStreams.entrySet()) { + for (Map.Entry changelogSystemStreamEntry : storeChangelogs.entrySet()) { SystemStreamPartition systemStreamPartition = new SystemStreamPartition(changelogSystemStreamEntry.getValue(), taskModel.getChangelogPartition()); SystemAdmin systemAdmin = systemAdmins.getSystemAdmin(changelogSystemStreamEntry.getValue().getSystem()); @@ -312,6 +326,28 @@ private String getStartingOffset(SystemStreamPartition systemStreamPartition, Sy return storageManagerUtil.getStartingOffset(systemStreamPartition, systemAdmin, fileOffset, oldestOffset); } + // TODO dchen put this in common code path for transactional and non-transactional + private Map createStoreEngines(Set storeNames, JobContext jobContext, + ContainerContext containerContext, Map> storageEngineFactories, + Map> serdes, MetricsRegistry metricsRegistry, + MessageCollector messageCollector, Map nonPersistedStores) { + Map storageEngines = new HashMap<>(); + // Put non persisted stores + nonPersistedStores.forEach(storageEngines::put); + // Create persisted stores + storeNames.forEach(storeName -> { + boolean isLogged = this.storeChangelogs.containsKey(storeName); + File storeBaseDir = isLogged ? this.loggedStoreBaseDirectory : this.nonLoggedStoreBaseDirectory; + File storeDirectory = storageManagerUtil.getTaskStoreDir(storeBaseDir, storeName, taskModel.getTaskName(), + taskModel.getTaskMode()); + StorageEngine engine = ContainerStorageManager.createStore(storeName, storeDirectory, taskModel, jobContext, containerContext, + storageEngineFactories, serdes, metricsRegistry, messageCollector, + StorageEngineFactory.StoreMode.BulkLoad, this.storeChangelogs, this.config); + storageEngines.put(storeName, engine); + }); + return storageEngines; + } + /** * Restore each store in taskStoresToRestore sequentially */ @@ -320,7 +356,7 @@ public void restore() throws InterruptedException { for (String storeName : taskStoresToRestore) { LOG.info("Restoring store: {} for task: {}", storeName, taskModel.getTaskName()); SystemConsumer systemConsumer = storeConsumers.get(storeName); - SystemStream systemStream = changelogSystemStreams.get(storeName); + SystemStream systemStream = storeChangelogs.get(storeName); SystemAdmin systemAdmin = systemAdmins.getSystemAdmin(systemStream.getSystem()); ChangelogSSPIterator changelogSSPIterator = new ChangelogSSPIterator(systemConsumer, new SystemStreamPartition(systemStream, taskModel.getChangelogPartition()), null, systemAdmin, false); @@ -333,7 +369,7 @@ public void restore() throws InterruptedException { * Stop only persistent stores. In case of certain stores and store mode (such as RocksDB), this * can invoke compaction. */ - public void stopPersistentStores() { + public void close() { Map persistentStores = this.taskStores.entrySet().stream().filter(e -> { return e.getValue().getStoreProperties().isPersistedToDisk(); diff --git a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java index 9d1896e3b2..0925949b67 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java +++ b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java @@ -20,13 +20,10 @@ package org.apache.samza.storage; import java.io.File; -import java.time.Duration; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; import org.apache.samza.SamzaException; import org.apache.samza.checkpoint.CheckpointManager; import org.apache.samza.config.Config; @@ -46,12 +43,10 @@ import org.apache.samza.metrics.MetricsRegistryMap; import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.SerdeFactory; -import org.apache.samza.system.SSPMetadataCache; import org.apache.samza.system.StreamMetadataCache; import org.apache.samza.system.SystemAdmins; import org.apache.samza.system.SystemFactory; import org.apache.samza.system.SystemStream; -import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.util.Clock; import org.apache.samza.util.CoordinatorStreamUtil; import org.apache.samza.util.ReflectionUtil; @@ -220,6 +215,7 @@ private Map> getSerdes() { */ @SuppressWarnings("rawtypes") private void getContainerStorageManagers() { + String factoryClass = new StorageConfig(jobConfig).getStateBackendRestoreFactory(); Clock clock = SystemClock.instance(); StreamMetadataCache streamMetadataCache = new StreamMetadataCache(systemAdmins, 5000, clock); // don't worry about prefetching for this; looks like the tool doesn't flush to offset files anyways @@ -230,18 +226,11 @@ private void getContainerStorageManagers() { for (ContainerModel containerModel : containers.values()) { ContainerContext containerContext = new ContainerContextImpl(containerModel, new MetricsRegistryMap()); - Set changelogSSPs = changeLogSystemStreams.values().stream() - .flatMap(ss -> containerModel.getTasks().values().stream() - .map(tm -> new SystemStreamPartition(ss, tm.getChangelogPartition()))) - .collect(Collectors.toSet()); - SSPMetadataCache sspMetadataCache = new SSPMetadataCache(systemAdmins, Duration.ofMillis(5000), clock, changelogSSPs); - ContainerStorageManager containerStorageManager = new ContainerStorageManager( checkpointManager, containerModel, streamMetadataCache, - sspMetadataCache, systemAdmins, changeLogSystemStreams, new HashMap<>(), @@ -253,10 +242,10 @@ private void getContainerStorageManagers() { new SamzaContainerMetrics(containerModel.getId(), new MetricsRegistryMap(), ""), JobContextImpl.fromConfigWithDefaults(jobConfig, jobModel), containerContext, + ReflectionUtil.getObj(factoryClass, StateBackendFactory.class), new HashMap<>(), storeBaseDir, storeBaseDir, - maxPartitionNumber, null, new SystemClock()); this.containerStorageManagers.put(containerModel.getId(), containerStorageManager); diff --git a/samza-core/src/main/java/org/apache/samza/storage/TaskRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/TaskRestoreManager.java deleted file mode 100644 index f60e14876b..0000000000 --- a/samza-core/src/main/java/org/apache/samza/storage/TaskRestoreManager.java +++ /dev/null @@ -1,55 +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.samza.storage; - -import java.util.Map; -import org.apache.samza.system.SystemStreamPartition; - - -/** - * The helper interface restores task state. - */ -public interface TaskRestoreManager { - - /** - * Init state resources such as file directories. - */ - void init(Map checkpointedChangelogSSPOffsets); - - /** - * Restore state from checkpoints, state snapshots and changelog. - * Currently, store restoration happens on a separate thread pool within {@code ContainerStorageManager}. In case of - * interrupt/shutdown signals from {@code SamzaContainer}, {@code ContainerStorageManager} may interrupt the restore - * thread. - * - * Note: Typically, interrupt signals don't bubble up as {@link InterruptedException} unless the restore thread is - * waiting on IO/network. In case of busy looping, implementors are expected to check the interrupt status of the - * thread periodically and shutdown gracefully before throwing {@link InterruptedException} upstream. - * {@code SamzaContainer} will not wait for clean up and the interrupt signal is the best effort by the container - * to notify that its shutting down. - */ - void restore() throws InterruptedException; - - /** - * Stop all persistent stores after restoring. - */ - void stopPersistentStores(); - -} diff --git a/samza-core/src/main/java/org/apache/samza/storage/TaskRestoreManagerFactory.java b/samza-core/src/main/java/org/apache/samza/storage/TaskRestoreManagerFactory.java deleted file mode 100644 index 9da9bc0ad6..0000000000 --- a/samza-core/src/main/java/org/apache/samza/storage/TaskRestoreManagerFactory.java +++ /dev/null @@ -1,82 +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.samza.storage; - -import java.io.File; -import java.util.Map; -import org.apache.samza.config.Config; -import org.apache.samza.config.TaskConfig; -import org.apache.samza.job.model.TaskModel; -import org.apache.samza.system.SSPMetadataCache; -import org.apache.samza.system.StreamMetadataCache; -import org.apache.samza.system.SystemAdmins; -import org.apache.samza.system.SystemConsumer; -import org.apache.samza.system.SystemStream; -import org.apache.samza.util.Clock; - -/** - * Factory class to create {@link TaskRestoreManager}. - */ -class TaskRestoreManagerFactory { - - public static TaskRestoreManager create( - TaskModel taskModel, - Map changelogSystemStreams, - Map taskStores, - SystemAdmins systemAdmins, - StreamMetadataCache streamMetadataCache, - SSPMetadataCache sspMetadataCache, - Map storeConsumers, - int maxChangeLogStreamPartitions, - File loggedStoreBaseDirectory, - File nonLoggedStoreBaseDirectory, - Config config, - Clock clock) { - - if (new TaskConfig(config).getTransactionalStateRestoreEnabled()) { - // Create checkpoint-snapshot based state restoration which is transactional. - return new TransactionalStateTaskRestoreManager( - taskModel, - taskStores, - changelogSystemStreams, - systemAdmins, - storeConsumers, - sspMetadataCache, - loggedStoreBaseDirectory, - nonLoggedStoreBaseDirectory, - config, - clock - ); - } else { - // Create legacy offset-file based state restoration which is NOT transactional. - return new NonTransactionalStateTaskRestoreManager( - taskModel, - changelogSystemStreams, - taskStores, - systemAdmins, - streamMetadataCache, - storeConsumers, - maxChangeLogStreamPartitions, - loggedStoreBaseDirectory, - nonLoggedStoreBaseDirectory, - config, - clock); - } - } -} diff --git a/samza-core/src/main/java/org/apache/samza/storage/TaskSideInputStorageManager.java b/samza-core/src/main/java/org/apache/samza/storage/TaskSideInputStorageManager.java index c93e0b3ea1..f407b1a7a6 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/TaskSideInputStorageManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/TaskSideInputStorageManager.java @@ -211,6 +211,7 @@ private boolean isPersistedStore(String storeName) { private void validateStoreConfiguration(Map stores) { stores.forEach((storeName, storageEngine) -> { + // Ensure that the side inputs store is NOT logged (they are durable) if (storageEngine.getStoreProperties().isLoggedStore()) { throw new SamzaException( String.format("Cannot configure both side inputs and a changelog for store: %s.", storeName)); diff --git a/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java index 4b6ac1f2a1..1329a0df84 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java @@ -31,24 +31,35 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import org.apache.commons.lang3.StringUtils; import org.apache.samza.Partition; import org.apache.samza.SamzaException; -import org.apache.samza.checkpoint.CheckpointedChangelogOffset; +import org.apache.samza.checkpoint.Checkpoint; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.checkpoint.CheckpointV1; +import org.apache.samza.checkpoint.CheckpointV2; +import org.apache.samza.checkpoint.kafka.KafkaChangelogSSPOffset; +import org.apache.samza.checkpoint.kafka.KafkaStateCheckpointMarker; import org.apache.samza.config.Config; import org.apache.samza.config.StorageConfig; import org.apache.samza.config.TaskConfig; import org.apache.samza.container.TaskName; +import org.apache.samza.context.ContainerContext; +import org.apache.samza.context.JobContext; import org.apache.samza.job.model.TaskMode; import org.apache.samza.job.model.TaskModel; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.serializers.Serde; import org.apache.samza.system.ChangelogSSPIterator; import org.apache.samza.system.SSPMetadataCache; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemAdmins; import org.apache.samza.system.SystemConsumer; import org.apache.samza.system.SystemStream; -import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata; +import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.task.MessageCollector; import org.apache.samza.util.Clock; import org.apache.samza.util.FileUtil; import org.slf4j.Logger; @@ -64,7 +75,7 @@ public class TransactionalStateTaskRestoreManager implements TaskRestoreManager private final Map storeEngines; // store name to storage engines private final Map storeChangelogs; // store name to changelog system stream private final SystemAdmins systemAdmins; - private final Map storeConsumers; + private final Map storeConsumers; // store name to system consumer private final SSPMetadataCache sspMetadataCache; private final File loggedStoreBaseDirectory; private final File nonLoggedStoreBaseDirectory; @@ -77,18 +88,24 @@ public class TransactionalStateTaskRestoreManager implements TaskRestoreManager private Map currentChangelogOffsets; public TransactionalStateTaskRestoreManager( + Set storeNames, // non-side input stores + JobContext jobContext, + ContainerContext containerContext, TaskModel taskModel, - Map storeEngines, Map storeChangelogs, + Map inMemoryStores, // in memory stores to be mutated during restore + Map> storageEngineFactories, + Map> serdes, SystemAdmins systemAdmins, Map storeConsumers, + MetricsRegistry metricsRegistry, + MessageCollector messageCollector, SSPMetadataCache sspMetadataCache, File loggedStoreBaseDirectory, File nonLoggedStoreBaseDirectory, Config config, Clock clock) { this.taskModel = taskModel; - this.storeEngines = storeEngines; this.storeChangelogs = storeChangelogs; this.systemAdmins = systemAdmins; this.storeConsumers = storeConsumers; @@ -101,14 +118,17 @@ public TransactionalStateTaskRestoreManager( this.clock = clock; this.storageManagerUtil = new StorageManagerUtil(); this.fileUtil = new FileUtil(); + this.storeEngines = createStoreEngines(storeNames, jobContext, containerContext, + storageEngineFactories, serdes, metricsRegistry, messageCollector, inMemoryStores); } @Override - public void init(Map checkpointedChangelogOffsets) { + public void init(Checkpoint checkpoint) { + Map storeStateCheckpointMarkers = getCheckpointedChangelogOffsets(checkpoint); currentChangelogOffsets = getCurrentChangelogOffsets(taskModel, storeChangelogs, sspMetadataCache); this.storeActions = getStoreActions(taskModel, storeEngines, storeChangelogs, - checkpointedChangelogOffsets, currentChangelogOffsets, systemAdmins, storageManagerUtil, + storeStateCheckpointMarkers, getCheckpointId(checkpoint), currentChangelogOffsets, systemAdmins, storageManagerUtil, loggedStoreBaseDirectory, nonLoggedStoreBaseDirectory, config, clock); setupStoreDirs(taskModel, storeEngines, storeActions, storageManagerUtil, fileUtil, @@ -143,7 +163,7 @@ public void restore() throws InterruptedException { * Stop only persistent stores. In case of certain stores and store mode (such as RocksDB), this * can invoke compaction. Persisted stores are recreated in read-write mode in {@link ContainerStorageManager}. */ - public void stopPersistentStores() { + public void close() { TaskName taskName = taskModel.getTaskName(); storeEngines.forEach((storeName, storeEngine) -> { if (storeEngine.getStoreProperties().isPersistedToDisk()) @@ -152,6 +172,27 @@ public void stopPersistentStores() { }); } + private Map createStoreEngines(Set storeNames, JobContext jobContext, + ContainerContext containerContext, Map> storageEngineFactories, + Map> serdes, MetricsRegistry metricsRegistry, + MessageCollector messageCollector, Map nonPersistedStores) { + Map storageEngines = new HashMap<>(); + // Put non persisted stores + nonPersistedStores.forEach(storageEngines::put); + // Create persisted stores + storeNames.forEach(storeName -> { + boolean isLogged = this.storeChangelogs.containsKey(storeName); + File storeBaseDir = isLogged ? this.loggedStoreBaseDirectory : this.nonLoggedStoreBaseDirectory; + File storeDirectory = storageManagerUtil.getTaskStoreDir(storeBaseDir, storeName, taskModel.getTaskName(), + taskModel.getTaskMode()); + StorageEngine engine = ContainerStorageManager.createStore(storeName, storeDirectory, taskModel, jobContext, containerContext, + storageEngineFactories, serdes, metricsRegistry, messageCollector, + StorageEngineFactory.StoreMode.BulkLoad, this.storeChangelogs, this.config); + storageEngines.put(storeName, engine); + }); + return storageEngines; + } + /** * Get offset metadata for each changelog SSP for this task. A task may have multiple changelog streams * (e.g., for different stores), but will have the same partition for all of them. @@ -194,7 +235,8 @@ static StoreActions getStoreActions( TaskModel taskModel, Map storeEngines, Map storeChangelogs, - Map checkpointedChangelogOffsets, + Map kafkaStateCheckpointMarkers, + CheckpointId checkpointId, Map currentChangelogOffsets, SystemAdmins systemAdmins, StorageManagerUtil storageManagerUtil, @@ -236,15 +278,12 @@ static StoreActions getStoreActions( String oldestOffset = changelogSSPMetadata.getOldestOffset(); String newestOffset = changelogSSPMetadata.getNewestOffset(); - String checkpointMessage = checkpointedChangelogOffsets.get(changelogSSP); - String checkpointedOffset = null; // can be null if no message, or message has null offset - long timeSinceLastCheckpointInMs = Long.MAX_VALUE; - if (StringUtils.isNotBlank(checkpointMessage)) { - CheckpointedChangelogOffset checkpointedChangelogOffset = CheckpointedChangelogOffset.fromString(checkpointMessage); - checkpointedOffset = checkpointedChangelogOffset.getOffset(); - timeSinceLastCheckpointInMs = System.currentTimeMillis() - - checkpointedChangelogOffset.getCheckpointId().getMillis(); + String checkpointedOffset = null; // can be null if no message, or message has null offset + if (kafkaStateCheckpointMarkers.containsKey(storeName) && + StringUtils.isNotBlank(kafkaStateCheckpointMarkers.get(storeName).getChangelogOffset())) { + checkpointedOffset = kafkaStateCheckpointMarkers.get(storeName).getChangelogOffset(); } + long timeSinceLastCheckpointInMs = checkpointId == null ? Long.MAX_VALUE : System.currentTimeMillis() - checkpointId.getMillis(); // if the clean.store.start config is set, delete current and checkpoint dirs, restore from oldest offset to checkpointed if (storageEngine.getStoreProperties().isPersistedToDisk() && new StorageConfig( @@ -557,6 +596,60 @@ private static void validateRestoreOffsets(RestoreOffsets restoreOffsets, System } } + private Map getCheckpointedChangelogOffsets(Checkpoint checkpoint) { + Map checkpointedChangelogOffsets = new HashMap<>(); + if (checkpoint == null) return checkpointedChangelogOffsets; + + if (checkpoint instanceof CheckpointV2) { + Map> factoryStoreSCMs = ((CheckpointV2) checkpoint).getStateCheckpointMarkers(); + if (factoryStoreSCMs.containsKey(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME)) { + factoryStoreSCMs.get(KafkaStateCheckpointMarker.KAFKA_STATE_BACKEND_FACTORY_NAME) + .forEach((storeName, scmString) -> { + KafkaStateCheckpointMarker kafkaSCM = KafkaStateCheckpointMarker.deserialize(scmString); + checkpointedChangelogOffsets.put(storeName, kafkaSCM); + }); + } // skip the non-KafkaStateCheckpointMarkers + } else if (checkpoint instanceof CheckpointV1) { + // If the checkpoint v1 is used, we need to fetch the changelog SSPs in the inputOffsets in order to get the + // store offset. + Map checkpointedOffsets = ((CheckpointV1) checkpoint).getOffsets(); + storeChangelogs.forEach((storeName, systemStream) -> { + Partition changelogPartition = taskModel.getChangelogPartition(); + SystemStreamPartition storeChangelogSSP = new SystemStreamPartition(systemStream, changelogPartition); + String checkpointedOffset = checkpointedOffsets.get(storeChangelogSSP); + if (StringUtils.isNotBlank(checkpointedOffset)) { + KafkaChangelogSSPOffset kafkaChangelogSSPOffset = KafkaChangelogSSPOffset.fromString(checkpointedOffset); + KafkaStateCheckpointMarker marker = new KafkaStateCheckpointMarker( + storeChangelogSSP, kafkaChangelogSSPOffset.getChangelogOffset()); + checkpointedChangelogOffsets.put(storeName, marker); + } + }); + } else { + throw new SamzaException("Unsupported checkpoint version: " + checkpoint.getVersion()); + } + + return checkpointedChangelogOffsets; + } + + private CheckpointId getCheckpointId(Checkpoint checkpoint) { + if (checkpoint == null) return null; + if (checkpoint instanceof CheckpointV1) { + for (Map.Entry storeNameSystemStream : storeChangelogs.entrySet()) { + SystemStreamPartition storeChangelogSSP = new SystemStreamPartition(storeNameSystemStream.getValue(), taskModel.getChangelogPartition()); + String checkpointMessage = checkpoint.getOffsets().get(storeChangelogSSP); + if (StringUtils.isNotBlank(checkpointMessage)) { + KafkaChangelogSSPOffset kafkaStateChanglogOffset = KafkaChangelogSSPOffset.fromString(checkpointMessage); + return kafkaStateChanglogOffset.getCheckpointId(); + } + } + } else if (checkpoint instanceof CheckpointV2) { + return ((CheckpointV2) checkpoint).getCheckpointId(); + } else { + throw new SamzaException("Unsupported checkpoint version: " + checkpoint.getVersion()); + } + return null; + } + @VisibleForTesting static class StoreActions { final Map storeDirsToRetain; diff --git a/samza-core/src/main/java/org/apache/samza/system/SystemAdmins.java b/samza-core/src/main/java/org/apache/samza/system/SystemAdmins.java index 2ca81caa7e..987e0c6481 100644 --- a/samza-core/src/main/java/org/apache/samza/system/SystemAdmins.java +++ b/samza-core/src/main/java/org/apache/samza/system/SystemAdmins.java @@ -43,6 +43,10 @@ public SystemAdmins(Config config, String adminLabel) { this.systemAdminMap = systemConfig.getSystemAdmins(adminLabel); } + public SystemAdmins(Map systemAdminMap) { + this.systemAdminMap = systemAdminMap; + } + /** * Creates a new instance of {@link SystemAdmins} with an empty admin mapping. * @return New empty instance of {@link SystemAdmins} @@ -70,6 +74,10 @@ public SystemAdmin getSystemAdmin(String systemName) { return systemAdminMap.get(systemName); } + public Map getSystemAdmins() { + return systemAdminMap; + } + public Set getSystemNames() { return systemAdminMap.keySet(); } diff --git a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java index c333748585..6838ec2f78 100644 --- a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java +++ b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java @@ -45,6 +45,7 @@ import org.apache.samza.checkpoint.Checkpoint; import org.apache.samza.checkpoint.CheckpointManager; import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; import org.apache.samza.config.StorageConfig; import org.apache.samza.config.TaskConfig; import org.apache.samza.container.RunLoop; @@ -65,7 +66,6 @@ import org.apache.samza.storage.kv.Entry; import org.apache.samza.storage.kv.KeyValueStore; import org.apache.samza.system.IncomingMessageEnvelope; -import org.apache.samza.system.SSPMetadataCache; import org.apache.samza.system.StreamMetadataCache; import org.apache.samza.system.SystemAdmins; import org.apache.samza.system.SystemConsumer; @@ -79,6 +79,7 @@ import org.apache.samza.system.chooser.MessageChooser; import org.apache.samza.system.chooser.RoundRobinChooserFactory; import org.apache.samza.table.utils.SerdeUtils; +import org.apache.samza.task.MessageCollector; import org.apache.samza.task.TaskInstanceCollector; import org.apache.samza.util.Clock; import org.apache.samza.util.ReflectionUtil; @@ -116,11 +117,14 @@ public class ContainerStorageManager { private static final int SIDE_INPUT_CHECK_TIMEOUT_SECONDS = 10; private static final int SIDE_INPUT_SHUTDOWN_TIMEOUT_SECONDS = 60; + private static final int RESTORE_THREAD_POOL_SHUTDOWN_TIMEOUT_SECONDS = 60; + /** Maps containing relevant per-task objects */ - private final Map> taskStores; private final Map taskRestoreManagers; private final Map taskInstanceMetrics; private final Map taskInstanceCollectors; + private final Map> inMemoryStores; // subset of taskStores after #start() + private Map> taskStores; // Will be available after #start() private final Map storeConsumers; // Mapping from store name to SystemConsumers private final Map> storageEngineFactories; // Map of storageEngineFactories indexed by store name @@ -129,7 +133,6 @@ public class ContainerStorageManager { private final SystemAdmins systemAdmins; private final StreamMetadataCache streamMetadataCache; - private final SSPMetadataCache sspMetadataCache; private final SamzaContainerMetrics samzaContainerMetrics; private final CheckpointManager checkpointManager; @@ -142,13 +145,12 @@ public class ContainerStorageManager { private final File nonLoggedStoreBaseDirectory; private final Set storeDirectoryPaths; // the set of store directory paths, used by SamzaContainer to initialize its disk-space-monitor - private final int parallelRestoreThreadPoolSize; - private final int maxChangeLogStreamPartitions; // The partition count of each changelog-stream topic. This is used for validating changelog streams before restoring. - /* Sideinput related parameters */ private final boolean hasSideInputs; + private final Map> sideInputStores; // subset of taskStores after #start() // side inputs indexed first by task, then store name private final Map>> taskSideInputStoreSSPs; + private final Set sideInputStoreNames; private final Map sspSideInputHandlers; private SystemConsumers sideInputSystemConsumers; private volatile Map sideInputTaskLatches; // Used by the sideInput-read thread to signal to the main thread @@ -157,17 +159,19 @@ public class ContainerStorageManager { private final ExecutorService sideInputsExecutor = Executors.newSingleThreadExecutor( new ThreadFactoryBuilder().setDaemon(true).setNameFormat(SIDEINPUTS_THREAD_NAME).build()); + private final ExecutorService restoreExecutor; private volatile Throwable sideInputException = null; private final Config config; private final StorageManagerUtil storageManagerUtil = new StorageManagerUtil(); + private boolean isStarted = false; + public ContainerStorageManager( CheckpointManager checkpointManager, ContainerModel containerModel, StreamMetadataCache streamMetadataCache, - SSPMetadataCache sspMetadataCache, SystemAdmins systemAdmins, Map changelogSystemStreams, Map> sideInputSystemStreams, @@ -179,22 +183,22 @@ public ContainerStorageManager( SamzaContainerMetrics samzaContainerMetrics, JobContext jobContext, ContainerContext containerContext, + StateBackendFactory stateBackendFactory, Map taskInstanceCollectors, File loggedStoreBaseDirectory, File nonLoggedStoreBaseDirectory, - int maxChangeLogStreamPartitions, SerdeManager serdeManager, Clock clock) { this.checkpointManager = checkpointManager; this.containerModel = containerModel; this.taskSideInputStoreSSPs = getTaskSideInputSSPs(containerModel, sideInputSystemStreams); + this.sideInputStoreNames = sideInputSystemStreams.keySet(); this.sideInputTaskLatches = new HashMap<>(); this.hasSideInputs = this.taskSideInputStoreSSPs.values().stream() .flatMap(m -> m.values().stream()) .flatMap(Collection::stream) .findAny() .isPresent(); - this.sspMetadataCache = sspMetadataCache; this.changelogSystemStreams = getChangelogSystemStreams(containerModel, changelogSystemStreams); // handling standby tasks LOG.info("Starting with changelogSystemStreams = {} taskSideInputStoreSSPs = {}", this.changelogSystemStreams, this.taskSideInputStoreSSPs); @@ -226,15 +230,22 @@ public ContainerStorageManager( // initializing the set of store directory paths this.storeDirectoryPaths = new HashSet<>(); - // Setting the restore thread pool size equal to the number of taskInstances - this.parallelRestoreThreadPoolSize = containerModel.getTasks().size(); - - this.maxChangeLogStreamPartitions = maxChangeLogStreamPartitions; this.streamMetadataCache = streamMetadataCache; this.systemAdmins = systemAdmins; - // create taskStores for all tasks in the containerModel and each store in storageEngineFactories - this.taskStores = createTaskStores(containerModel, jobContext, containerContext, storageEngineFactories, serdes, taskInstanceMetrics, taskInstanceCollectors); + // create side input taskStores for all tasks in the containerModel and each store in storageEngineFactories + this.sideInputStores = createTaskStores(sideInputStoreNames, containerModel, jobContext, containerContext, + storageEngineFactories, serdes, taskInstanceMetrics, taskInstanceCollectors); + StorageConfig storageConfig = new StorageConfig(config); + Set inMemoryStoreNames = storageEngineFactories.keySet().stream() + .filter(storeName -> { + Optional storeFactory = storageConfig.getStorageFactoryClassName(storeName); + return storeFactory.isPresent() && !storeFactory.get() + .equals(StorageConfig.INMEMORY_KV_STORAGE_ENGINE_FACTORY); + }) + .collect(Collectors.toSet()); + this.inMemoryStores = createTaskStores(inMemoryStoreNames, + this.containerModel, jobContext, containerContext, storageEngineFactories, serdes, taskInstanceMetrics, taskInstanceCollectors); Set containerChangelogSystems = this.changelogSystemStreams.values().stream() .map(SystemStream::getSystem) @@ -245,8 +256,19 @@ public ContainerStorageManager( containerChangelogSystems, systemFactories, config, this.samzaContainerMetrics.registry()); this.storeConsumers = createStoreIndexedMap(this.changelogSystemStreams, storeSystemConsumers); + // TODO HIGH dchen tune based on observed concurrency + JobConfig jobConfig = new JobConfig(config); + int restoreThreadPoolSize = + Math.min( + Math.max(containerModel.getTasks().size() * 2, jobConfig.getRestoreThreadPoolSize()), + jobConfig.getRestoreThreadPoolMaxSize() + ); + this.restoreExecutor = Executors.newFixedThreadPool(restoreThreadPoolSize, + new ThreadFactoryBuilder().setDaemon(true).setNameFormat(RESTORE_THREAD_NAME).build()); + // creating task restore managers - this.taskRestoreManagers = createTaskRestoreManagers(systemAdmins, clock, this.samzaContainerMetrics); + this.taskRestoreManagers = createTaskRestoreManagers(stateBackendFactory, clock, + this.samzaContainerMetrics); this.sspSideInputHandlers = createSideInputHandlers(clock); @@ -311,7 +333,8 @@ private Map>> getTaskSideInputS * @param changelogSystemStreams the passed in set of changelogSystemStreams * @return A map of changeLogSSP to storeName across all tasks, assuming no two stores have the same changelogSSP */ - private Map getChangelogSystemStreams(ContainerModel containerModel, Map changelogSystemStreams) { + private Map getChangelogSystemStreams(ContainerModel containerModel, + Map changelogSystemStreams) { if (MapUtils.invertMap(changelogSystemStreams).size() != changelogSystemStreams.size()) { throw new SamzaException("Two stores cannot have the same changelog system-stream"); @@ -319,20 +342,22 @@ private Map getChangelogSystemStreams(ContainerModel conta Map changelogSSPToStore = new HashMap<>(); changelogSystemStreams.forEach((storeName, systemStream) -> - containerModel.getTasks().forEach((taskName, taskModel) -> { changelogSSPToStore.put(new SystemStreamPartition(systemStream, taskModel.getChangelogPartition()), storeName); }) + containerModel.getTasks().forEach((taskName, taskModel) -> + changelogSSPToStore.put(new SystemStreamPartition(systemStream, taskModel.getChangelogPartition()), storeName)) ); getTasks(containerModel, TaskMode.Standby).forEach((taskName, taskModel) -> { - this.taskSideInputStoreSSPs.putIfAbsent(taskName, new HashMap<>()); + taskSideInputStoreSSPs.putIfAbsent(taskName, new HashMap<>()); changelogSystemStreams.forEach((storeName, systemStream) -> { SystemStreamPartition ssp = new SystemStreamPartition(systemStream, taskModel.getChangelogPartition()); changelogSSPToStore.remove(ssp); - this.taskSideInputStoreSSPs.get(taskName).put(storeName, Collections.singleton(ssp)); + taskSideInputStoreSSPs.get(taskName).put(storeName, Collections.singleton(ssp)); }); }); // changelogSystemStreams correspond only to active tasks (since those of standby-tasks moved to sideInputs above) - return MapUtils.invertMap(changelogSSPToStore).entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, x -> x.getValue().getSystemStream())); + return MapUtils.invertMap(changelogSSPToStore).entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, x -> x.getValue().getSystemStream())); } @@ -357,25 +382,35 @@ private static Map createConsumers(Set storeSyst } private static Map createStoreIndexedMap(Map changelogSystemStreams, - Map storeSystemConsumers) { + Map systemNameToSystemConsumers) { // Map of each storeName to its respective systemConsumer Map storeConsumers = new HashMap<>(); // Populate the map of storeName to its relevant systemConsumer for (String storeName : changelogSystemStreams.keySet()) { - storeConsumers.put(storeName, storeSystemConsumers.get(changelogSystemStreams.get(storeName).getSystem())); + storeConsumers.put(storeName, systemNameToSystemConsumers.get(changelogSystemStreams.get(storeName).getSystem())); } return storeConsumers; } - private Map createTaskRestoreManagers(SystemAdmins systemAdmins, Clock clock, SamzaContainerMetrics samzaContainerMetrics) { + private Map createTaskRestoreManagers(StateBackendFactory factory, Clock clock, + SamzaContainerMetrics samzaContainerMetrics) { Map taskRestoreManagers = new HashMap<>(); + containerModel.getTasks().forEach((taskName, taskModel) -> { + MetricsRegistry taskMetricsRegistry = + taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry() : new MetricsRegistryMap(); + Set nonSideInputStoreNames = storageEngineFactories.keySet().stream() + .filter(storeName -> !sideInputStoreNames.contains(storeName)) + .collect(Collectors.toSet()); + KafkaChangelogRestoreParams kafkaChangelogRestoreParams = new KafkaChangelogRestoreParams(storeConsumers, + inMemoryStores.get(taskName), systemAdmins.getSystemAdmins(), storageEngineFactories, serdes, + taskInstanceCollectors.get(taskName), nonSideInputStoreNames); + taskRestoreManagers.put(taskName, - TaskRestoreManagerFactory.create( - taskModel, changelogSystemStreams, getNonSideInputStores(taskName), systemAdmins, - streamMetadataCache, sspMetadataCache, storeConsumers, maxChangeLogStreamPartitions, - loggedStoreBaseDirectory, nonLoggedStoreBaseDirectory, config, clock)); + factory.getRestoreManager(jobContext, containerContext, taskModel, restoreExecutor, + taskMetricsRegistry, config, clock, loggedStoreBaseDirectory, nonLoggedStoreBaseDirectory, + kafkaChangelogRestoreParams)); samzaContainerMetrics.addStoresRestorationGauge(taskName); }); return taskRestoreManagers; @@ -388,106 +423,77 @@ private static Map getTasks(ContainerModel containerModel, } /** - * Create taskStores for all stores in storageEngineFactories. - * The store mode is chosen as bulk-load if its a non-sideinput store, and readWrite if its a sideInput store + * Create taskStores for all stores in storesToCreate. + * The store mode is chosen as read-write mode. */ - private Map> createTaskStores(ContainerModel containerModel, JobContext jobContext, ContainerContext containerContext, + private Map> createTaskStores(Set storesToCreate, + ContainerModel containerModel, JobContext jobContext, ContainerContext containerContext, Map> storageEngineFactories, Map> serdes, Map taskInstanceMetrics, Map taskInstanceCollectors) { - Map> taskStores = new HashMap<>(); + StorageConfig storageConfig = new StorageConfig(config); - // iterate over each task in the containerModel, and each store in storageEngineFactories + // iterate over each task and each storeName for (Map.Entry task : containerModel.getTasks().entrySet()) { TaskName taskName = task.getKey(); TaskModel taskModel = task.getValue(); - if (!taskStores.containsKey(taskName)) { taskStores.put(taskName, new HashMap<>()); } - for (String storeName : storageEngineFactories.keySet()) { - - StorageEngineFactory.StoreMode storeMode = this.taskSideInputStoreSSPs.get(taskName).containsKey(storeName) ? - StorageEngineFactory.StoreMode.ReadWrite : StorageEngineFactory.StoreMode.BulkLoad; + for (String storeName : storesToCreate) { + // A store is considered durable if it is backed by a changelog or another backupManager factory + boolean isDurable = changelogSystemStreams.containsKey(storeName) || + !storageConfig.getStoreBackupManagerClassName(storeName).isEmpty(); + boolean isSideInput = this.sideInputStoreNames.contains(storeName); + // Use the logged-store-base-directory for change logged stores and sideInput stores, and non-logged-store-base-dir + // for non logged stores + File storeBaseDir = isDurable || isSideInput ? this.loggedStoreBaseDirectory : this.nonLoggedStoreBaseDirectory; + File storeDirectory = storageManagerUtil.getTaskStoreDir(storeBaseDir, storeName, taskName, + taskModel.getTaskMode()); + this.storeDirectoryPaths.add(storeDirectory.toPath()); + + // if taskInstanceMetrics are specified use those for store metrics, + // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap + MetricsRegistry storeMetricsRegistry = + taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry() : new MetricsRegistryMap(); StorageEngine storageEngine = - createStore(storeName, taskName, taskModel, jobContext, containerContext, storageEngineFactories, serdes, taskInstanceMetrics, taskInstanceCollectors, storeMode); + createStore(storeName, storeDirectory, taskModel, jobContext, containerContext, storageEngineFactories, + serdes, storeMetricsRegistry, taskInstanceCollectors.get(taskName), + StorageEngineFactory.StoreMode.ReadWrite, this.changelogSystemStreams, this.config); // add created store to map taskStores.get(taskName).put(storeName, storageEngine); - LOG.info("Created store {} for task {} in mode {}", storeName, taskName, storeMode); + LOG.info("Created non side input store store {} in read-write mode for task {}", storeName, taskName); } } - return taskStores; } - /** - * Recreate all non-sideInput persistent stores in ReadWrite mode. - * - */ - private void recreatePersistentTaskStoresInReadWriteMode(ContainerModel containerModel, JobContext jobContext, - ContainerContext containerContext, Map> storageEngineFactories, - Map> serdes, Map taskInstanceMetrics, - Map taskInstanceCollectors) { - - // iterate over each task and each storeName - for (Map.Entry task : containerModel.getTasks().entrySet()) { - TaskName taskName = task.getKey(); - TaskModel taskModel = task.getValue(); - Map nonSideInputStores = getNonSideInputStores(taskName); - - for (String storeName : nonSideInputStores.keySet()) { - - // if this store has been already created then re-create and overwrite it only if it is a - // persistentStore and a non-sideInputStore, because sideInputStores are always created in RW mode - if (nonSideInputStores.get(storeName).getStoreProperties().isPersistedToDisk()) { - - StorageEngine storageEngine = - createStore(storeName, taskName, taskModel, jobContext, containerContext, storageEngineFactories, serdes, taskInstanceMetrics, taskInstanceCollectors, - StorageEngineFactory.StoreMode.ReadWrite); - - // add created store to map - this.taskStores.get(taskName).put(storeName, storageEngine); - - LOG.info("Re-created store {} in read-write mode for task {} because it a persistent store", storeName, taskName); - } else { - LOG.info("Skipping re-creation of store {} for task {}", storeName, taskName); - } - } - } - } - /** * Method to instantiate a StorageEngine with the given parameters, and populate the storeDirectory paths (used to monitor * disk space). */ - private StorageEngine createStore(String storeName, TaskName taskName, TaskModel taskModel, JobContext jobContext, - ContainerContext containerContext, Map> storageEngineFactories, - Map> serdes, Map taskInstanceMetrics, - Map taskInstanceCollectors, StorageEngineFactory.StoreMode storeMode) { + public static StorageEngine createStore( + String storeName, + File storeDirectory, + TaskModel taskModel, + JobContext jobContext, + ContainerContext containerContext, + Map> storageEngineFactories, + Map> serdes, + MetricsRegistry storeMetricsRegistry, + MessageCollector messageCollector, + StorageEngineFactory.StoreMode storeMode, + Map changelogSystemStreams, + Config config) { StorageConfig storageConfig = new StorageConfig(config); - - SystemStreamPartition changeLogSystemStreamPartition = - (changelogSystemStreams.containsKey(storeName)) ? new SystemStreamPartition( - changelogSystemStreams.get(storeName), taskModel.getChangelogPartition()) : null; - - // Use the logged-store-base-directory for change logged stores and sideInput stores, and non-logged-store-base-dir - // for non logged stores - File storeDirectory; - if (changeLogSystemStreamPartition != null || this.taskSideInputStoreSSPs.get(taskName).containsKey(storeName)) { - storeDirectory = storageManagerUtil.getTaskStoreDir(this.loggedStoreBaseDirectory, storeName, taskName, - taskModel.getTaskMode()); - } else { - storeDirectory = storageManagerUtil.getTaskStoreDir(this.nonLoggedStoreBaseDirectory, storeName, taskName, - taskModel.getTaskMode()); - } - - this.storeDirectoryPaths.add(storeDirectory.toPath()); + SystemStreamPartition changeLogSystemStreamPartition = changelogSystemStreams.containsKey(storeName) ? + new SystemStreamPartition(changelogSystemStreams.get(storeName), taskModel.getChangelogPartition()) : null; Optional storageKeySerde = storageConfig.getStorageKeySerde(storeName); Serde keySerde = null; @@ -500,14 +506,8 @@ private StorageEngine createStore(String storeName, TaskName taskName, TaskModel messageSerde = serdes.get(storageMsgSerde.get()); } - // if taskInstanceMetrics are specified use those for store metrics, - // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap - MetricsRegistry storeMetricsRegistry = - taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry() - : new MetricsRegistryMap(); - return storageEngineFactories.get(storeName) - .getStorageEngine(storeName, storeDirectory, keySerde, messageSerde, taskInstanceCollectors.get(taskName), + .getStorageEngine(storeName, storeDirectory, keySerde, messageSerde, messageCollector, storeMetricsRegistry, changeLogSystemStreamPartition, jobContext, containerContext, storeMode); } @@ -584,10 +584,10 @@ private Map createSideInputHandlers if (this.hasSideInputs) { containerModel.getTasks().forEach((taskName, taskModel) -> { - Map sideInputStores = getSideInputStores(taskName); + Map taskSideInputStores = sideInputStores.get(taskName); Map> sideInputStoresToSSPs = new HashMap<>(); boolean taskHasSideInputs = false; - for (String storeName : sideInputStores.keySet()) { + for (String storeName : taskSideInputStores.keySet()) { Set storeSSPs = this.taskSideInputStoreSSPs.get(taskName).get(storeName); taskHasSideInputs = taskHasSideInputs || !storeSSPs.isEmpty(); sideInputStoresToSSPs.put(storeName, storeSSPs); @@ -600,7 +600,7 @@ private Map createSideInputHandlers TaskSideInputHandler taskSideInputHandler = new TaskSideInputHandler(taskName, taskModel.getTaskMode(), loggedStoreBaseDirectory, - sideInputStores, + taskSideInputStores, sideInputStoresToSSPs, taskSideInputProcessors.get(taskName), this.systemAdmins, @@ -612,73 +612,60 @@ private Map createSideInputHandlers handlers.put(ssp, taskSideInputHandler); }); - LOG.info("Created TaskSideInputHandler for task {}, sideInputStores {} and loggedStoreBaseDirectory {}", - taskName, sideInputStores, loggedStoreBaseDirectory); + LOG.info("Created TaskSideInputHandler for task {}, taskSideInputStores {} and loggedStoreBaseDirectory {}", + taskName, taskSideInputStores, loggedStoreBaseDirectory); } }); } return handlers; } - private Map getSideInputStores(TaskName taskName) { - return taskStores.get(taskName).entrySet().stream(). - filter(e -> this.taskSideInputStoreSSPs.get(taskName).containsKey(e.getKey())).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - } - - private Map getNonSideInputStores(TaskName taskName) { - return taskStores.get(taskName).entrySet().stream(). - filter(e -> !this.taskSideInputStoreSSPs.get(taskName).containsKey(e.getKey())).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - } - private Set getSideInputHandlers() { return this.sspSideInputHandlers.values().stream().collect(Collectors.toSet()); } public void start() throws SamzaException, InterruptedException { - Map checkpointedChangelogSSPOffsets = new HashMap<>(); - if (new TaskConfig(config).getTransactionalStateRestoreEnabled()) { - getTasks(containerModel, TaskMode.Active).forEach((taskName, taskModel) -> { - if (checkpointManager != null) { - Set changelogSystemStreams = new HashSet<>(this.changelogSystemStreams.values()); - Checkpoint checkpoint = checkpointManager.readLastCheckpoint(taskName); - if (checkpoint != null) { - checkpoint.getOffsets().forEach((ssp, offset) -> { - if (changelogSystemStreams.contains(new SystemStream(ssp.getSystem(), ssp.getStream()))) { - checkpointedChangelogSSPOffsets.put(ssp, offset); - } - }); - } - } - }); + // Restores and recreates + restoreStores(); + // Shutdown restore executor since it will no longer be used + try { + restoreExecutor.shutdown(); + if (restoreExecutor.awaitTermination(RESTORE_THREAD_POOL_SHUTDOWN_TIMEOUT_SECONDS, TimeUnit.MILLISECONDS)) { + restoreExecutor.shutdownNow(); + } + } catch (Exception e) { + LOG.error(e.getMessage()); } - LOG.info("Checkpointed changelog ssp offsets: {}", checkpointedChangelogSSPOffsets); - restoreStores(checkpointedChangelogSSPOffsets); if (this.hasSideInputs) { startSideInputs(); } + isStarted = true; } // Restoration of all stores, in parallel across tasks - private void restoreStores(Map checkpointedChangelogSSPOffsets) - throws InterruptedException { + private void restoreStores() throws InterruptedException { LOG.info("Store Restore started"); + Set activeTasks = getTasks(containerModel, TaskMode.Active).keySet(); // initialize each TaskStorageManager - this.taskRestoreManagers.values().forEach(taskStorageManager -> - taskStorageManager.init(checkpointedChangelogSSPOffsets)); + this.taskRestoreManagers.forEach((taskName, taskRestoreManager) -> { + Checkpoint taskCheckpoint = null; + if (checkpointManager != null && activeTasks.contains(taskName)) { + // only pass in checkpoints for active tasks + taskCheckpoint = checkpointManager.readLastCheckpoint(taskName); + LOG.info("Obtained checkpoint: {} for state restore for taskName: {}", taskCheckpoint, taskName); + } + taskRestoreManager.init(taskCheckpoint); + }); // Start each store consumer once this.storeConsumers.values().stream().distinct().forEach(SystemConsumer::start); - // Create a thread pool for parallel restores (and stopping of persistent stores) - ExecutorService executorService = Executors.newFixedThreadPool(this.parallelRestoreThreadPoolSize, - new ThreadFactoryBuilder().setDaemon(true).setNameFormat(RESTORE_THREAD_NAME).build()); - List taskRestoreFutures = new ArrayList<>(this.taskRestoreManagers.entrySet().size()); // Submit restore callable for each taskInstance this.taskRestoreManagers.forEach((taskInstance, taskRestoreManager) -> { - taskRestoreFutures.add(executorService.submit( + taskRestoreFutures.add(restoreExecutor.submit( new TaskRestoreCallable(this.samzaContainerMetrics, taskInstance, taskRestoreManager))); }); @@ -690,7 +677,7 @@ private void restoreStores(Map checkpointedChange } catch (InterruptedException e) { LOG.warn("Received an interrupt during store restoration. Issuing interrupts to the store restoration workers to exit " + "prematurely without restoring full state."); - executorService.shutdownNow(); + restoreExecutor.shutdownNow(); throw e; } catch (Exception e) { LOG.error("Exception when restoring ", e); @@ -698,14 +685,29 @@ private void restoreStores(Map checkpointedChange } } - executorService.shutdown(); - // Stop each store consumer once this.storeConsumers.values().stream().distinct().forEach(SystemConsumer::stop); - // Now re-create persistent stores in read-write mode, leave non-persistent stores as-is - recreatePersistentTaskStoresInReadWriteMode(this.containerModel, jobContext, containerContext, + // Now create persistent non side input stores in read-write mode, leave non-persistent stores as-is + Set nonSideInputStoreNames = storageEngineFactories.keySet().stream() + .filter(storeName -> !sideInputStoreNames.contains(storeName)) + .collect(Collectors.toSet()); + this.taskStores = createTaskStores(nonSideInputStoreNames, this.containerModel, jobContext, containerContext, storageEngineFactories, serdes, taskInstanceMetrics, taskInstanceCollectors); + // Add in memory stores + this.inMemoryStores.forEach((taskName, stores) -> { + if (!this.taskStores.containsKey(taskName)) { + taskStores.put(taskName, new HashMap<>()); + } + taskStores.get(taskName).putAll(stores); + }); + // Add side input stores + this.sideInputStores.forEach((taskName, stores) -> { + if (!this.taskStores.containsKey(taskName)) { + taskStores.put(taskName, new HashMap<>()); + } + taskStores.get(taskName).putAll(stores); + }); LOG.info("Store Restore complete"); } @@ -838,15 +840,24 @@ private boolean awaitSideInputTasks() throws InterruptedException { * @return the task store. */ public Optional getStore(TaskName taskName, String storeName) { + if (!isStarted) { + throw new SamzaException(String.format( + "Attempting to access store %s for task %s before ContainerStorageManager is started.", + storeName, taskName)); + } return Optional.ofNullable(this.taskStores.get(taskName).get(storeName)); } /** - * Get all {@link StorageEngine} instance used by a given task. - * @param taskName the task name, all stores for which are desired. + * Get all {@link StorageEngine} instance used by a given task. + * @param taskName the task name, all stores for which are desired. * @return map of stores used by the given task, indexed by storename */ public Map getAllStores(TaskName taskName) { + if (!isStarted) { + throw new SamzaException(String.format( + "Attempting to access stores for task %s before ContainerStorageManager is started.", taskName)); + } return this.taskStores.get(taskName); } @@ -865,9 +876,13 @@ public void stopStores() { public void shutdown() { // stop all nonsideinputstores including persistent and non-persistent stores - this.containerModel.getTasks().forEach((taskName, taskModel) -> - getNonSideInputStores(taskName).forEach((storeName, store) -> store.stop()) - ); + if (taskStores != null) { + this.containerModel.getTasks() + .forEach((taskName, taskModel) -> taskStores.get(taskName) + .entrySet().stream() + .filter(e -> !sideInputStoreNames.contains(e.getKey())) + .forEach(e -> e.getValue().stop())); + } this.shouldShutdown = true; @@ -926,7 +941,7 @@ public Void call() { } finally { // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation). - taskRestoreManager.stopPersistentStores(); + taskRestoreManager.close(); long timeToRestore = System.currentTimeMillis() - startTime; if (this.samzaContainerMetrics != null) { diff --git a/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManagerFactory.java b/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManagerFactory.java deleted file mode 100644 index 97e45043b5..0000000000 --- a/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManagerFactory.java +++ /dev/null @@ -1,46 +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.samza.storage; - -import scala.collection.immutable.Map; - -import java.io.File; -import org.apache.samza.Partition; -import org.apache.samza.config.Config; -import org.apache.samza.config.TaskConfig; -import org.apache.samza.container.TaskName; -import org.apache.samza.job.model.TaskMode; -import org.apache.samza.system.SystemAdmins; -import org.apache.samza.system.SystemStream; - -public class TaskStorageManagerFactory { - public static TaskStorageManager create(TaskName taskName, ContainerStorageManager containerStorageManager, - Map storeChangelogs, SystemAdmins systemAdmins, - File loggedStoreBaseDir, Partition changelogPartition, - Config config, TaskMode taskMode) { - if (new TaskConfig(config).getTransactionalStateCheckpointEnabled()) { - return new TransactionalStateTaskStorageManager(taskName, containerStorageManager, storeChangelogs, systemAdmins, - loggedStoreBaseDir, changelogPartition, taskMode, new StorageManagerUtil()); - } else { - return new NonTransactionalStateTaskStorageManager(taskName, containerStorageManager, storeChangelogs, systemAdmins, - loggedStoreBaseDir, changelogPartition); - } - } -} diff --git a/samza-core/src/test/java/org/apache/samza/job/model/TestJobModel.java b/samza-core/src/test/java/org/apache/samza/job/model/TestJobModel.java index 77fe639f67..e66e213d6f 100644 --- a/samza-core/src/test/java/org/apache/samza/job/model/TestJobModel.java +++ b/samza-core/src/test/java/org/apache/samza/job/model/TestJobModel.java @@ -45,6 +45,6 @@ public void testMaxChangeLogStreamPartitions() { ContainerModel containerModel2 = new ContainerModel("1", tasksForContainer2); Map containers = ImmutableMap.of("0", containerModel1, "1", containerModel2); JobModel jobModel = new JobModel(config, containers); - assertEquals(jobModel.maxChangeLogStreamPartitions, 5); + assertEquals(jobModel.getMaxChangeLogStreamPartitions(), 5); } } \ No newline at end of file diff --git a/samza-core/src/test/java/org/apache/samza/storage/TestTransactionalStateTaskRestoreManager.java b/samza-core/src/test/java/org/apache/samza/storage/TestTransactionalStateTaskRestoreManager.java index 2bdd6c371e..6d5b45c66d 100644 --- a/samza-core/src/test/java/org/apache/samza/storage/TestTransactionalStateTaskRestoreManager.java +++ b/samza-core/src/test/java/org/apache/samza/storage/TestTransactionalStateTaskRestoreManager.java @@ -33,7 +33,7 @@ import java.util.Set; import org.apache.samza.Partition; import org.apache.samza.checkpoint.CheckpointId; -import org.apache.samza.checkpoint.CheckpointedChangelogOffset; +import org.apache.samza.checkpoint.kafka.KafkaStateCheckpointMarker; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.config.TaskConfig; @@ -134,7 +134,7 @@ public void testGetStoreActionsForNonLoggedPersistentStore_AlwaysClearStore() { Map mockStoreChangelogs = ImmutableMap.of(); - Map mockCheckpointedChangelogOffset = ImmutableMap.of(); + Map mockCheckpointedChangelogOffset = ImmutableMap.of(); Map mockCurrentChangelogOffsets = ImmutableMap.of(); SystemAdmins mockSystemAdmins = mock(SystemAdmins.class); @@ -149,7 +149,7 @@ public void testGetStoreActionsForNonLoggedPersistentStore_AlwaysClearStore() { .thenReturn(mockCurrentStoreDir); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, null, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -185,10 +185,11 @@ public void testStoreDeletedWhenCleanDirsFlagSet() { Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "5"; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - ImmutableMap mockCheckpointedChangelogOffset = - ImmutableMap.of(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + ImmutableMap mockCheckpointedChangelogOffset = + ImmutableMap.of(store1Name, kafkaStateCheckpointMarker); Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -218,7 +219,7 @@ public void testStoreDeletedWhenCleanDirsFlagSet() { .thenReturn(ImmutableList.of(dummyCheckpointDir)); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -255,10 +256,11 @@ public void testGetStoreActionsForLoggedNonPersistentStore_RestoreToCheckpointed Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "5"; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - ImmutableMap mockCheckpointedChangelogOffset = - ImmutableMap.of(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + ImmutableMap mockCheckpointedChangelogOffset = + ImmutableMap.of(store1Name, kafkaStateCheckpointMarker); Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -279,7 +281,7 @@ public void testGetStoreActionsForLoggedNonPersistentStore_RestoreToCheckpointed }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -319,11 +321,12 @@ public void testGetStoreActionsForLoggedNonPersistentStore_FullRestoreIfCheckpoi Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "21"; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - Map mockCheckpointedChangelogOffset = - new HashMap() { { - put(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + Map mockCheckpointedChangelogOffset = + new HashMap() { { + put(store1Name, kafkaStateCheckpointMarker); } }; Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -345,7 +348,7 @@ public void testGetStoreActionsForLoggedNonPersistentStore_FullRestoreIfCheckpoi }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -386,11 +389,12 @@ public void testGetStoreActionsForLoggedNonPersistentStore_FullRestoreIfCheckpoi Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "5"; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - Map mockCheckpointedChangelogOffset = - new HashMap() { { - put(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + Map mockCheckpointedChangelogOffset = + new HashMap() { { + put(store1Name, kafkaStateCheckpointMarker); } }; Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -412,7 +416,7 @@ public void testGetStoreActionsForLoggedNonPersistentStore_FullRestoreIfCheckpoi }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -453,12 +457,12 @@ public void testGetStoreActionsForLoggedNonPersistentStore_FullRestoreIfCheckpoi Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "5"; - CheckpointId checkpointId = CheckpointId.fromString("0-0"); // checkpoint id older than default min.compaction.lag.ms - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(checkpointId, changelog1CheckpointedOffset); - Map mockCheckpointedChangelogOffset = - new HashMap() { { - put(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.deserialize("0-0"); // checkpoint id older than default min.compaction.lag.ms + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + Map mockCheckpointedChangelogOffset = + new HashMap() { { + put(store1Name, kafkaStateCheckpointMarker); } }; Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -480,7 +484,7 @@ public void testGetStoreActionsForLoggedNonPersistentStore_FullRestoreIfCheckpoi }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -524,11 +528,12 @@ public void testGetStoreActionsForLoggedNonPersistentStore_FullTrimIfNullCheckpo Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = null; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - Map mockCheckpointedChangelogOffset = - new HashMap() { { - put(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + Map mockCheckpointedChangelogOffset = + new HashMap() { { + put(store1Name, kafkaStateCheckpointMarker); } }; Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -552,7 +557,7 @@ public void testGetStoreActionsForLoggedNonPersistentStore_FullTrimIfNullCheckpo }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -588,11 +593,12 @@ public void testGetStoreActionsForLoggedNonPersistentStore_FullRestoreIfNullChec Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = null; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - Map mockCheckpointedChangelogOffset = - new HashMap() { { - put(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + Map mockCheckpointedChangelogOffset = + new HashMap() { { + put(store1Name, kafkaStateCheckpointMarker); } }; Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -616,7 +622,7 @@ public void testGetStoreActionsForLoggedNonPersistentStore_FullRestoreIfNullChec }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -652,10 +658,11 @@ public void testGetStoreActionsForLoggedPersistentStore_RestoreToCheckpointedOff Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "5"; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - ImmutableMap mockCheckpointedChangelogOffset = - ImmutableMap.of(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + ImmutableMap mockCheckpointedChangelogOffset = + ImmutableMap.of(store1Name, kafkaStateCheckpointMarker); Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -679,7 +686,7 @@ public void testGetStoreActionsForLoggedPersistentStore_RestoreToCheckpointedOff }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -717,10 +724,11 @@ public void testGetStoreActionsForLoggedPersistentStore_RestoreToCheckpointedOff Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "5"; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - ImmutableMap mockCheckpointedChangelogOffset = - ImmutableMap.of(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + ImmutableMap mockCheckpointedChangelogOffset = + ImmutableMap.of(store1Name, kafkaStateCheckpointMarker); Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -760,7 +768,7 @@ public void testGetStoreActionsForLoggedPersistentStore_RestoreToCheckpointedOff }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -800,10 +808,11 @@ public void testGetStoreActionsForLoggedPersistentStore_RestoreDeltaIfStaleStore Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "5"; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - ImmutableMap mockCheckpointedChangelogOffset = - ImmutableMap.of(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + ImmutableMap mockCheckpointedChangelogOffset = + ImmutableMap.of(store1Name, kafkaStateCheckpointMarker); Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -843,7 +852,7 @@ public void testGetStoreActionsForLoggedPersistentStore_RestoreDeltaIfStaleStore }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -886,10 +895,11 @@ public void testGetStoreActionsForLoggedPersistentStore_NoRestoreButTrimIfUpToDa Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "5"; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - ImmutableMap mockCheckpointedChangelogOffset = - ImmutableMap.of(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + ImmutableMap mockCheckpointedChangelogOffset = + ImmutableMap.of(store1Name, kafkaStateCheckpointMarker); Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -927,7 +937,7 @@ public void testGetStoreActionsForLoggedPersistentStore_NoRestoreButTrimIfUpToDa }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -971,10 +981,11 @@ public void testGetStoreActionsForLoggedPersistentStore_DeleteStoreCheckpointIfL Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "5"; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - ImmutableMap mockCheckpointedChangelogOffset = - ImmutableMap.of(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + ImmutableMap mockCheckpointedChangelogOffset = + ImmutableMap.of(store1Name, kafkaStateCheckpointMarker); Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -1012,7 +1023,7 @@ public void testGetStoreActionsForLoggedPersistentStore_DeleteStoreCheckpointIfL }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -1056,10 +1067,11 @@ public void testGetStoreActionsForLoggedPersistentStore_RetainOneCheckpointIfMul Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "5"; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - ImmutableMap mockCheckpointedChangelogOffset = - ImmutableMap.of(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + ImmutableMap mockCheckpointedChangelogOffset = + ImmutableMap.of(store1Name, kafkaStateCheckpointMarker); Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -1097,7 +1109,7 @@ public void testGetStoreActionsForLoggedPersistentStore_RetainOneCheckpointIfMul }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -1144,11 +1156,12 @@ public void testGetStoreActionsForLoggedPersistentStore_FullRestoreIfNullCheckpo Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = null; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - Map mockCheckpointedChangelogOffset = - new HashMap() { { - put(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + Map mockCheckpointedChangelogOffset = + new HashMap() { { + put(store1Name, kafkaStateCheckpointMarker); } }; Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -1191,7 +1204,7 @@ public void testGetStoreActionsForLoggedPersistentStore_FullRestoreIfNullCheckpo }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -1239,11 +1252,12 @@ public void testGetStoreActionsForLoggedPersistentStore_FullTrimIfNullCheckpoint Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = null; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - Map mockCheckpointedChangelogOffset = - new HashMap() { { - put(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + Map mockCheckpointedChangelogOffset = + new HashMap() { { + put(store1Name, kafkaStateCheckpointMarker); } }; Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -1286,7 +1300,7 @@ public void testGetStoreActionsForLoggedPersistentStore_FullTrimIfNullCheckpoint }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -1332,11 +1346,12 @@ public void testGetStoreActionsForLoggedPersistentStore_FullRestoreIfEqualCheckp Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "5"; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - Map mockCheckpointedChangelogOffset = - new HashMap() { { - put(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + Map mockCheckpointedChangelogOffset = + new HashMap() { { + put(store1Name, kafkaStateCheckpointMarker); } }; Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -1376,7 +1391,7 @@ public void testGetStoreActionsForLoggedPersistentStore_FullRestoreIfEqualCheckp }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -1421,11 +1436,12 @@ public void testGetStoreActionsForLoggedPersistentStore_FullRestoreIfNullCheckpo Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = null; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - Map mockCheckpointedChangelogOffset = - new HashMap() { { - put(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + Map mockCheckpointedChangelogOffset = + new HashMap() { { + put(store1Name, kafkaStateCheckpointMarker); } }; Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -1471,7 +1487,7 @@ public void testGetStoreActionsForLoggedPersistentStore_FullRestoreIfNullCheckpo }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -1516,11 +1532,12 @@ public void testGetStoreActionsForLoggedPersistentStore_FullRestoreIfCheckpointe Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "5"; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - Map mockCheckpointedChangelogOffset = - new HashMap() { { - put(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + Map mockCheckpointedChangelogOffset = + new HashMap() { { + put(store1Name, kafkaStateCheckpointMarker); } }; Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -1561,7 +1578,7 @@ public void testGetStoreActionsForLoggedPersistentStore_FullRestoreIfCheckpointe }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -1606,12 +1623,12 @@ public void testGetStoreActionsForLoggedPersistentStore_RestoreFromLocalToNewest Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "5"; - CheckpointId checkpointId = CheckpointId.fromString("0-0"); // checkpoint timestamp older than default min compaction lag - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(checkpointId, changelog1CheckpointedOffset); - Map mockCheckpointedChangelogOffset = - new HashMap() { { - put(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.deserialize("0-0"); // checkpoint timestamp older than default min compaction lag + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + Map mockCheckpointedChangelogOffset = + new HashMap() { { + put(store1Name, kafkaStateCheckpointMarker); } }; Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -1652,7 +1669,7 @@ public void testGetStoreActionsForLoggedPersistentStore_RestoreFromLocalToNewest }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); @@ -1697,11 +1714,12 @@ public void testGetStoreActionsForLoggedPersistentStore_FullRestoreIfCheckpointe Map mockStoreChangelogs = ImmutableMap.of(store1Name, changelog1SystemStream); String changelog1CheckpointedOffset = "21"; - CheckpointedChangelogOffset changelog1CheckpointMessage = - new CheckpointedChangelogOffset(CheckpointId.create(), changelog1CheckpointedOffset); - Map mockCheckpointedChangelogOffset = - new HashMap() { { - put(changelog1SSP, changelog1CheckpointMessage.toString()); + CheckpointId checkpointId = CheckpointId.create(); + KafkaStateCheckpointMarker kafkaStateCheckpointMarker = + new KafkaStateCheckpointMarker(changelog1SSP, changelog1CheckpointedOffset); + Map mockCheckpointedChangelogOffset = + new HashMap() { { + put(store1Name, kafkaStateCheckpointMarker); } }; Map mockCurrentChangelogOffsets = ImmutableMap.of(changelog1SSP, changelog1SSPMetadata); @@ -1742,7 +1760,7 @@ public void testGetStoreActionsForLoggedPersistentStore_FullRestoreIfCheckpointe }); StoreActions storeActions = TransactionalStateTaskRestoreManager.getStoreActions( - mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, + mockTaskModel, mockStoreEngines, mockStoreChangelogs, mockCheckpointedChangelogOffset, checkpointId, mockCurrentChangelogOffsets, mockSystemAdmins, mockStorageManagerUtil, mockLoggedStoreBaseDir, mockNonLoggedStoreBaseDir, mockConfig, mockClock); diff --git a/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java b/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java index c36a3bebde..f381f3d497 100644 --- a/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java +++ b/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java @@ -24,8 +24,8 @@ import java.util.HashSet; import java.util.Map; import org.apache.samza.Partition; -import org.apache.samza.checkpoint.Checkpoint; import org.apache.samza.checkpoint.CheckpointManager; +import org.apache.samza.checkpoint.CheckpointV1; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.config.TaskConfig; @@ -205,24 +205,36 @@ public Void answer(InvocationOnMock invocation) { new scala.collection.immutable.Map.Map1(new SystemStream(SYSTEM_NAME, STREAM_NAME), systemStreamMetadata)); CheckpointManager checkpointManager = mock(CheckpointManager.class); - when(checkpointManager.readLastCheckpoint(any(TaskName.class))).thenReturn(new Checkpoint(new HashMap<>())); + when(checkpointManager.readLastCheckpoint(any(TaskName.class))).thenReturn(new CheckpointV1(new HashMap<>())); SSPMetadataCache mockSSPMetadataCache = mock(SSPMetadataCache.class); when(mockSSPMetadataCache.getMetadata(any(SystemStreamPartition.class))) .thenReturn(new SystemStreamMetadata.SystemStreamPartitionMetadata("0", "10", "11")); + ContainerContext mockContainerContext = mock(ContainerContext.class); + ContainerModel mockContainerModel = new ContainerModel("samza-container-test", tasks); + when(mockContainerContext.getContainerModel()).thenReturn(mockContainerModel); + // Reset the expected number of sysConsumer create, start and stop calls, and store.restore() calls this.systemConsumerCreationCount = 0; this.systemConsumerStartCount = 0; this.systemConsumerStopCount = 0; this.storeRestoreCallCount = 0; + StateBackendFactory backendFactory = mock(StateBackendFactory.class); + TaskRestoreManager restoreManager = mock(TaskRestoreManager.class); + when(backendFactory.getRestoreManager(any(), any(), any(), any(), any(), any(), any(), any(), any(), any())) + .thenReturn(restoreManager); + doAnswer(invocation -> { + storeRestoreCallCount++; + return null; + }).when(restoreManager).restore(); + // Create the container storage manager this.containerStorageManager = new ContainerStorageManager( checkpointManager, - new ContainerModel("samza-container-test", tasks), + mockContainerModel, mockStreamMetadataCache, - mockSSPMetadataCache, mockSystemAdmins, changelogSystemStreams, new HashMap<>(), @@ -233,11 +245,11 @@ public Void answer(InvocationOnMock invocation) { taskInstanceMetrics, samzaContainerMetrics, mock(JobContext.class), - mock(ContainerContext.class), + mockContainerContext, + backendFactory, mock(Map.class), DEFAULT_LOGGED_STORE_BASE_DIR, DEFAULT_STORE_BASE_DIR, - 2, null, new SystemClock()); } @@ -252,10 +264,10 @@ public void testParallelismAndMetrics() throws InterruptedException { mockingDetails(gauge).getInvocations().size() >= 1); } - Assert.assertTrue("Store restore count should be 2 because there are 2 tasks", this.storeRestoreCallCount == 2); - Assert.assertTrue("systemConsumerCreation count should be 1 (1 consumer per system)", - this.systemConsumerCreationCount == 1); - Assert.assertTrue("systemConsumerStopCount count should be 1", this.systemConsumerStopCount == 1); - Assert.assertTrue("systemConsumerStartCount count should be 1", this.systemConsumerStartCount == 1); + Assert.assertEquals("Store restore count should be 2 because there are 2 tasks", 2, this.storeRestoreCallCount); + Assert.assertEquals("systemConsumerCreation count should be 1 (1 consumer per system)", 1, + this.systemConsumerCreationCount); + Assert.assertEquals("systemConsumerStopCount count should be 1", 1, this.systemConsumerStopCount); + Assert.assertEquals("systemConsumerStartCount count should be 1", 1, this.systemConsumerStartCount); } } diff --git a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryKeyValueStorageEngineFactory.java b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryKeyValueStorageEngineFactory.java index 8cd4e3645a..883766b708 100644 --- a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryKeyValueStorageEngineFactory.java +++ b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryKeyValueStorageEngineFactory.java @@ -25,7 +25,6 @@ import org.apache.samza.storage.kv.BaseKeyValueStorageEngineFactory; import org.apache.samza.storage.kv.KeyValueStore; import org.apache.samza.storage.kv.KeyValueStoreMetrics; -import org.apache.samza.system.SystemStreamPartition; public class InMemoryKeyValueStorageEngineFactory extends BaseKeyValueStorageEngineFactory { @@ -33,7 +32,6 @@ public class InMemoryKeyValueStorageEngineFactory extends BaseKeyValueStor protected KeyValueStore getKVStore(String storeName, File storeDir, MetricsRegistry registry, - SystemStreamPartition changeLogSystemStreamPartition, JobContext jobContext, ContainerContext containerContext, StoreMode storeMode) { diff --git a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala index d02d6235f3..e1f44d218f 100644 --- a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala +++ b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala @@ -34,14 +34,12 @@ class RocksDbKeyValueStorageEngineFactory [K, V] extends BaseKeyValueStorageEngi * @param storeName Name of the store * @param storeDir The directory of the store * @param registry MetricsRegistry to which to publish store specific metrics. - * @param changeLogSystemStreamPartition Samza stream partition from which to receive the changelog. * @param containerContext Information about the container in which the task is executing. * @return A valid KeyValueStore instance */ override def getKVStore(storeName: String, storeDir: File, registry: MetricsRegistry, - changeLogSystemStreamPartition: SystemStreamPartition, jobContext: JobContext, containerContext: ContainerContext, storeMode: StoreMode): KeyValueStore[Array[Byte], Array[Byte]] = { val storageConfigSubset = jobContext.getConfig.subset("stores." + storeName + ".", true) diff --git a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala index 300177af6a..9d5ddfd6d2 100644 --- a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala +++ b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala @@ -24,11 +24,11 @@ import java.nio.file.{Path, Paths} import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantReadWriteLock import java.util.{Comparator, Optional} - import org.apache.samza.SamzaException import org.apache.samza.checkpoint.CheckpointId import org.apache.samza.config.Config -import org.apache.samza.util.Logging +import org.apache.samza.storage.StorageManagerUtil +import org.apache.samza.util.{FileUtil, Logging} import org.rocksdb.{TtlDB, _} object RocksDbKeyValueStore extends Logging { @@ -67,6 +67,9 @@ object RocksDbKeyValueStore extends Logging { } try { + // Create the path if it doesn't exist + new FileUtil().createDirectories(dir.toPath) + val rocksDb = if (useTTL) { info("Opening RocksDB store: %s in path: %s with TTL value: %s" format (storeName, dir.toString, ttl)) @@ -239,7 +242,7 @@ class RocksDbKeyValueStore( override def checkpoint(id: CheckpointId): Optional[Path] = { val checkpoint = Checkpoint.create(db) - val checkpointPath = dir.getPath + "-" + id.toString + val checkpointPath = StorageManagerUtil.getCheckpointDirPath(dir, id) checkpoint.createCheckpoint(checkpointPath) Optional.of(Paths.get(checkpointPath)) } diff --git a/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala b/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala index 2a4f44eafc..54dca8f4df 100644 --- a/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala +++ b/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala @@ -22,11 +22,10 @@ package org.apache.samza.storage.kv import java.io.File import java.util - import org.apache.samza.SamzaException import org.apache.samza.config.MapConfig import org.apache.samza.metrics.{Counter, Gauge, MetricsRegistryMap, MetricsVisitor, Timer} -import org.apache.samza.util.ExponentialSleepStrategy +import org.apache.samza.util.{ExponentialSleepStrategy, FileUtil} import org.junit.{Assert, Test} import org.rocksdb.{FlushOptions, Options, RocksDB, RocksIterator} @@ -123,6 +122,23 @@ class TestRocksDbKeyValueStore rocksDB.get(key) } + @Test + def testRocksDbCreatePathIfNotExist(): Unit = { + val map = new util.HashMap[String, String]() + val config = new MapConfig(map) + val options = new Options() + options.setCreateIfMissing(true) + + val dbDir = new File(System.getProperty("java.io.tmpdir") + File.separator + "samza-test2", "rocksDbFiles") + val rocksDB = new RocksDbKeyValueStore(dbDir, options, config, false, "dbStore") + val key = "key".getBytes("UTF-8") + rocksDB.put(key, "val".getBytes("UTF-8")) + + rocksDB.close() + + new FileUtil().rm(dbDir) + } + @Test def testIteratorWithRemoval(): Unit = { val lock = new Object diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java index d6c1196eb1..704e0cbd8a 100644 --- a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java @@ -19,6 +19,7 @@ package org.apache.samza.storage.kv; import java.io.File; +import java.util.List; import java.util.Optional; import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; @@ -60,7 +61,6 @@ public abstract class BaseKeyValueStorageEngineFactory implements StorageE * @param storeName Name of the store * @param storeDir The directory of the store * @param registry MetricsRegistry to which to publish store specific metrics. - * @param changeLogSystemStreamPartition Samza stream partition from which to receive the changelog. * @param jobContext Information about the job in which the task is executing. * @param containerContext Information about the container in which the task is executing. * @return A raw KeyValueStore instance @@ -68,7 +68,6 @@ public abstract class BaseKeyValueStorageEngineFactory implements StorageE protected abstract KeyValueStore getKVStore(String storeName, File storeDir, MetricsRegistry registry, - SystemStreamPartition changeLogSystemStreamPartition, JobContext jobContext, ContainerContext containerContext, StoreMode storeMode); @@ -106,6 +105,10 @@ public StorageEngine getStorageEngine(String storeName, if (!storeFactory.get().equals(INMEMORY_KV_STORAGE_ENGINE_FACTORY)) { storePropertiesBuilder.setPersistedToDisk(true); } + // The store is durable iff it is backed by the task backup manager + List storeBackupManager = storageConfig.getStoreBackupManagerClassName(storeName); + storePropertiesBuilder.setIsDurable(!storeBackupManager.isEmpty()); + int batchSize = storageConfigSubset.getInt(WRITE_BATCH_SIZE, DEFAULT_WRITE_BATCH_SIZE); int cacheSize = storageConfigSubset.getInt(OBJECT_CACHE_SIZE, Math.max(batchSize, DEFAULT_OBJECT_CACHE_SIZE)); if (cacheSize > 0 && cacheSize < batchSize) { @@ -123,7 +126,7 @@ public StorageEngine getStorageEngine(String storeName, } KeyValueStore rawStore = - getKVStore(storeName, storeDir, registry, changelogSSP, jobContext, containerContext, storeMode); + getKVStore(storeName, storeDir, registry, jobContext, containerContext, storeMode); KeyValueStore maybeLoggedStore = buildMaybeLoggedStore(changelogSSP, storeName, registry, storePropertiesBuilder, rawStore, changelogCollector); // this also applies serialization and caching layers diff --git a/samza-kv/src/test/java/org/apache/samza/storage/kv/MockKeyValueStorageEngineFactory.java b/samza-kv/src/test/java/org/apache/samza/storage/kv/MockKeyValueStorageEngineFactory.java index 3430ae951e..2c8251e962 100644 --- a/samza-kv/src/test/java/org/apache/samza/storage/kv/MockKeyValueStorageEngineFactory.java +++ b/samza-kv/src/test/java/org/apache/samza/storage/kv/MockKeyValueStorageEngineFactory.java @@ -22,7 +22,6 @@ import org.apache.samza.context.ContainerContext; import org.apache.samza.context.JobContext; import org.apache.samza.metrics.MetricsRegistry; -import org.apache.samza.system.SystemStreamPartition; /** @@ -38,8 +37,7 @@ public MockKeyValueStorageEngineFactory(KeyValueStore rawKeyValu @Override protected KeyValueStore getKVStore(String storeName, File storeDir, MetricsRegistry registry, - SystemStreamPartition changeLogSystemStreamPartition, JobContext jobContext, ContainerContext containerContext, - StoreMode storeMode) { + JobContext jobContext, ContainerContext containerContext, StoreMode storeMode) { return this.rawKeyValueStore; } } diff --git a/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseKeyValueStorageEngineFactory.java b/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseKeyValueStorageEngineFactory.java index 22a1b57027..05b106d28d 100644 --- a/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseKeyValueStorageEngineFactory.java +++ b/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseKeyValueStorageEngineFactory.java @@ -127,7 +127,23 @@ public void testInMemoryKeyValueStore() { "org.apache.samza.storage.kv.inmemory.InMemoryKeyValueStorageEngineFactory")); StorageEngine storageEngine = callGetStorageEngine(config, null); KeyValueStorageEngine keyValueStorageEngine = baseStorageEngineValidation(storageEngine); - assertStoreProperties(keyValueStorageEngine.getStoreProperties(), false, false); + assertStoreProperties(keyValueStorageEngine.getStoreProperties(), false, false, false); + NullSafeKeyValueStore nullSafeKeyValueStore = + assertAndCast(keyValueStorageEngine.getWrapperStore(), NullSafeKeyValueStore.class); + SerializedKeyValueStore serializedKeyValueStore = + assertAndCast(nullSafeKeyValueStore.getStore(), SerializedKeyValueStore.class); + // config has the in-memory key-value factory, but still calling the test factory, so store will be the test store + assertEquals(this.rawKeyValueStore, serializedKeyValueStore.getStore()); + } + + @Test + public void testDurableKeyValueStore() { + Config config = new MapConfig(BASE_CONFIG, DISABLE_CACHE, + ImmutableMap.of(String.format(StorageConfig.STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME), + "backendFactory,backendFactory2")); + StorageEngine storageEngine = callGetStorageEngine(config, null); + KeyValueStorageEngine keyValueStorageEngine = baseStorageEngineValidation(storageEngine); + assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, false, true); NullSafeKeyValueStore nullSafeKeyValueStore = assertAndCast(keyValueStorageEngine.getWrapperStore(), NullSafeKeyValueStore.class); SerializedKeyValueStore serializedKeyValueStore = @@ -141,7 +157,7 @@ public void testRawStoreOnly() { Config config = new MapConfig(BASE_CONFIG, DISABLE_CACHE); StorageEngine storageEngine = callGetStorageEngine(config, null); KeyValueStorageEngine keyValueStorageEngine = baseStorageEngineValidation(storageEngine); - assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, false); + assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, false, false); NullSafeKeyValueStore nullSafeKeyValueStore = assertAndCast(keyValueStorageEngine.getWrapperStore(), NullSafeKeyValueStore.class); SerializedKeyValueStore serializedKeyValueStore = @@ -154,7 +170,7 @@ public void testWithLoggedStore() { Config config = new MapConfig(BASE_CONFIG, DISABLE_CACHE); StorageEngine storageEngine = callGetStorageEngine(config, CHANGELOG_SSP); KeyValueStorageEngine keyValueStorageEngine = baseStorageEngineValidation(storageEngine); - assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, true); + assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, true, false); NullSafeKeyValueStore nullSafeKeyValueStore = assertAndCast(keyValueStorageEngine.getWrapperStore(), NullSafeKeyValueStore.class); SerializedKeyValueStore serializedKeyValueStore = @@ -170,7 +186,7 @@ public void testWithLoggedStoreAndCachedStore() { Config config = new MapConfig(BASE_CONFIG); StorageEngine storageEngine = callGetStorageEngine(config, CHANGELOG_SSP); KeyValueStorageEngine keyValueStorageEngine = baseStorageEngineValidation(storageEngine); - assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, true); + assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, true, false); NullSafeKeyValueStore nullSafeKeyValueStore = assertAndCast(keyValueStorageEngine.getWrapperStore(), NullSafeKeyValueStore.class); CachedStore cachedStore = assertAndCast(nullSafeKeyValueStore.getStore(), CachedStore.class); @@ -187,7 +203,7 @@ public void testDisallowLargeMessages() { Config config = new MapConfig(BASE_CONFIG, DISABLE_CACHE, DISALLOW_LARGE_MESSAGES); StorageEngine storageEngine = callGetStorageEngine(config, null); KeyValueStorageEngine keyValueStorageEngine = baseStorageEngineValidation(storageEngine); - assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, false); + assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, false, false); NullSafeKeyValueStore nullSafeKeyValueStore = assertAndCast(keyValueStorageEngine.getWrapperStore(), NullSafeKeyValueStore.class); SerializedKeyValueStore serializedKeyValueStore = @@ -202,7 +218,7 @@ public void testDisallowLargeMessagesWithCache() { Config config = new MapConfig(BASE_CONFIG, DISALLOW_LARGE_MESSAGES); StorageEngine storageEngine = callGetStorageEngine(config, null); KeyValueStorageEngine keyValueStorageEngine = baseStorageEngineValidation(storageEngine); - assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, false); + assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, false, false); NullSafeKeyValueStore nullSafeKeyValueStore = assertAndCast(keyValueStorageEngine.getWrapperStore(), NullSafeKeyValueStore.class); SerializedKeyValueStore serializedKeyValueStore = @@ -220,7 +236,7 @@ public void testDropLargeMessages() { Config config = new MapConfig(BASE_CONFIG, DISABLE_CACHE, DROP_LARGE_MESSAGES); StorageEngine storageEngine = callGetStorageEngine(config, null); KeyValueStorageEngine keyValueStorageEngine = baseStorageEngineValidation(storageEngine); - assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, false); + assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, false, false); NullSafeKeyValueStore nullSafeKeyValueStore = assertAndCast(keyValueStorageEngine.getWrapperStore(), NullSafeKeyValueStore.class); SerializedKeyValueStore serializedKeyValueStore = @@ -235,7 +251,7 @@ public void testDropLargeMessagesWithCache() { Config config = new MapConfig(BASE_CONFIG, DROP_LARGE_MESSAGES); StorageEngine storageEngine = callGetStorageEngine(config, null); KeyValueStorageEngine keyValueStorageEngine = baseStorageEngineValidation(storageEngine); - assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, false); + assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, false, false); NullSafeKeyValueStore nullSafeKeyValueStore = assertAndCast(keyValueStorageEngine.getWrapperStore(), NullSafeKeyValueStore.class); CachedStore cachedStore = assertAndCast(nullSafeKeyValueStore.getStore(), CachedStore.class); @@ -252,7 +268,7 @@ public void testAccessLogStore() { // AccessLoggedStore requires a changelog SSP StorageEngine storageEngine = callGetStorageEngine(config, CHANGELOG_SSP); KeyValueStorageEngine keyValueStorageEngine = baseStorageEngineValidation(storageEngine); - assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, true); + assertStoreProperties(keyValueStorageEngine.getStoreProperties(), true, true, false); NullSafeKeyValueStore nullSafeKeyValueStore = assertAndCast(keyValueStorageEngine.getWrapperStore(), NullSafeKeyValueStore.class); AccessLoggedStore accessLoggedStore = @@ -278,9 +294,10 @@ public void testAccessLogStore() { } private static void assertStoreProperties(StoreProperties storeProperties, boolean expectedPersistedToDisk, - boolean expectedLoggedStore) { + boolean expectedLoggedStore, boolean expectedDurable) { assertEquals(expectedPersistedToDisk, storeProperties.isPersistedToDisk()); assertEquals(expectedLoggedStore, storeProperties.isLoggedStore()); + assertEquals(expectedDurable, storeProperties.isDurableStore()); } /** diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java index 4e693720ae..e2e599da3d 100644 --- a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java +++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java @@ -856,7 +856,7 @@ public void testStatefulSamzaApplicationShouldRedistributeInputPartitionsToCorre } Assert.assertEquals(expectedTaskAssignments, actualTaskAssignments); - Assert.assertEquals(32, jobModel.maxChangeLogStreamPartitions); + Assert.assertEquals(32, jobModel.getMaxChangeLogStreamPartitions()); } /** @@ -958,7 +958,7 @@ public void testStatefulSamzaApplicationShouldRedistributeInputPartitionsToCorre // Validate that the new JobModel has the expected task assignments. actualTaskAssignments = getTaskAssignments(jobModel); Assert.assertEquals(expectedTaskAssignments, actualTaskAssignments); - Assert.assertEquals(32, jobModel.maxChangeLogStreamPartitions); + Assert.assertEquals(32, jobModel.getMaxChangeLogStreamPartitions()); } @Test diff --git a/samza-test/src/test/scala/org/apache/samza/test/integration/NonTransactionalStateIntegrationTest.scala b/samza-test/src/test/scala/org/apache/samza/test/integration/NonTransactionalStateIntegrationTest.scala index 00a530df5f..229e026375 100644 --- a/samza-test/src/test/scala/org/apache/samza/test/integration/NonTransactionalStateIntegrationTest.scala +++ b/samza-test/src/test/scala/org/apache/samza/test/integration/NonTransactionalStateIntegrationTest.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.samza.test.integration import org.apache.samza.config.TaskConfig @@ -78,7 +77,9 @@ class NonTransactionalStateIntegrationTest extends StreamTaskTestUtil { "stores.mystore.changelog.replication.factor" -> "1", // However, don't have the inputs use the checkpoint manager // since the second part of the test expects to replay the input streams. - "systems.kafka.streams.input.samza.reset.offset" -> "true")) + "systems.kafka.streams.input.samza.reset.offset" -> "true", + TaskConfig.COMMIT_MAX_DELAY_MS -> "0" // Ensure no commits are skipped due to in progress commits + )) @Test def testShouldStartAndRestore { diff --git a/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala b/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala index edcb159d9f..b375511037 100644 --- a/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala +++ b/samza-test/src/test/scala/org/apache/samza/test/integration/StreamTaskTestUtil.scala @@ -34,7 +34,7 @@ import org.apache.kafka.clients.consumer.{ConsumerRecord, KafkaConsumer} import org.apache.kafka.clients.producer.{KafkaProducer, Producer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.samza.Partition -import org.apache.samza.checkpoint.Checkpoint +import org.apache.samza.checkpoint.{Checkpoint, CheckpointV1} import org.apache.samza.config._ import org.apache.samza.container.TaskName import org.apache.samza.context.Context @@ -73,8 +73,8 @@ object StreamTaskTestUtil { var producer: Producer[Array[Byte], Array[Byte]] = null var adminClient: AdminClient = null - val cp1 = new Checkpoint(Map(new SystemStreamPartition("kafka", "topic", new Partition(0)) -> "123").asJava) - val cp2 = new Checkpoint(Map(new SystemStreamPartition("kafka", "topic", new Partition(0)) -> "12345").asJava) + val cp1 = new CheckpointV1(Map(new SystemStreamPartition("kafka", "topic", new Partition(0)) -> "123").asJava) + val cp2 = new CheckpointV1(Map(new SystemStreamPartition("kafka", "topic", new Partition(0)) -> "12345").asJava) // use a random store directory for each run. prevents test failures due to left over state from // previously aborted test runs @@ -291,7 +291,7 @@ class StreamTaskTestUtil { case _ => throw new ConfigException("No checkpoint manager factory configured") } - ChangelogStreamManager.createChangelogStreams(jobModel.getConfig, jobModel.maxChangeLogStreamPartitions) + ChangelogStreamManager.createChangelogStreams(jobModel.getConfig, jobModel.getMaxChangeLogStreamPartitions) } } From 7cc4eaa96fff244f6dce9c18af804917db7c3b2b Mon Sep 17 00:00:00 2001 From: shekhars-li <72765053+shekhars-li@users.noreply.github.com> Date: Tue, 25 May 2021 16:00:46 -0700 Subject: [PATCH 04/20] SAMZA-2657: Blob Store as backend for Samza State backup and restore (#1501) SAMZA-2657: Introduce blob store as backend for Samza state backup and restore --- build.gradle | 1 + .../samza/storage/BlobStoreAdminFactory.java | 37 + ...orageAdmin.java => StateBackendAdmin.java} | 11 +- .../samza/storage/StateBackendFactory.java | 8 +- .../storage/blobstore/BlobStoreManager.java | 83 ++ .../blobstore/BlobStoreManagerFactory.java | 31 + .../samza/storage/blobstore/Metadata.java | 114 +++ .../exceptions/DeletedException.java | 46 + .../exceptions/RetriableException.java | 42 + .../ClusterBasedJobCoordinator.java | 15 + .../apache/samza/config/BlobStoreConfig.java | 41 + .../apache/samza/config/StorageConfig.java | 74 +- .../standalone/PassthroughJobCoordinator.java | 14 +- .../KafkaChangelogStateBackendFactory.java | 70 +- ...nTransactionalStateTaskRestoreManager.java | 2 +- .../samza/storage/StorageManagerUtil.java | 25 +- .../apache/samza/storage/StorageRecovery.java | 2 +- .../storage/TaskSideInputStorageManager.java | 17 +- .../storage/TaskStorageCommitManager.java | 7 +- .../TransactionalStateTaskRestoreManager.java | 3 +- .../blobstore/BlobStoreBackupManager.java | 350 +++++++ .../blobstore/BlobStoreRestoreManager.java | 369 +++++++ .../BlobStoreStateBackendFactory.java | 98 ++ .../samza/storage/blobstore/diff/DirDiff.java | 244 +++++ .../storage/blobstore/index/DirIndex.java | 176 ++++ .../storage/blobstore/index/FileBlob.java | 85 ++ .../storage/blobstore/index/FileIndex.java | 114 +++ .../storage/blobstore/index/FileMetadata.java | 138 +++ .../blobstore/index/SnapshotIndex.java | 124 +++ .../blobstore/index/SnapshotMetadata.java | 115 +++ .../index/serde/JsonDirIndexMixin.java | 58 ++ .../index/serde/JsonFileBlobMixin.java | 41 + .../index/serde/JsonFileIndexMixin.java | 53 + .../index/serde/JsonFileMetadataMixin.java | 55 + .../index/serde/JsonSnapshotIndexMixin.java | 53 + .../serde/JsonSnapshotMetadataMixin.java | 53 + .../index/serde/SnapshotIndexSerde.java | 76 ++ .../BlobStoreBackupManagerMetrics.java | 136 +++ .../BlobStoreRestoreManagerMetrics.java | 94 ++ .../storage/blobstore/util/BlobStoreUtil.java | 622 ++++++++++++ .../storage/blobstore/util/DirDiffUtil.java | 403 ++++++++ .../org/apache/samza/util/FutureUtil.java | 8 +- .../samza/container/SamzaContainer.scala | 73 +- .../samza/job/local/ProcessJobFactory.scala | 15 +- .../storage/ContainerStorageManager.java | 4 +- .../samza/config/TestStorageConfig.java | 40 +- .../storage/TestTaskStorageCommitManager.java | 68 +- .../blobstore/TestBlobStoreBackupManager.java | 542 ++++++++++ .../TestBlobStoreRestoreManager.java | 361 +++++++ .../serde/TestSnapshotIndexSerde.java | 56 ++ .../blobstore/util/BlobStoreTestUtil.java | 296 ++++++ .../blobstore/util/TestBlobStoreUtil.java | 936 ++++++++++++++++++ .../blobstore/util/TestDirDiffUtil.java | 127 +++ .../blobstore/util/TestDirDiffUtilMisc.java | 71 ++ .../storage/kv/RocksDbOptionsHelper.java | 124 ++- .../RocksDbKeyValueStorageEngineFactory.scala | 8 +- .../storage/kv/RocksDbKeyValueStore.scala | 12 +- .../kv/BaseKeyValueStorageEngineFactory.java | 2 +- .../TestBaseKeyValueStorageEngineFactory.java | 2 +- 59 files changed, 6652 insertions(+), 193 deletions(-) create mode 100644 samza-api/src/main/java/org/apache/samza/storage/BlobStoreAdminFactory.java rename samza-api/src/main/java/org/apache/samza/storage/{TaskStorageAdmin.java => StateBackendAdmin.java} (76%) create mode 100644 samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManager.java create mode 100644 samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManagerFactory.java create mode 100644 samza-api/src/main/java/org/apache/samza/storage/blobstore/Metadata.java create mode 100644 samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/DeletedException.java create mode 100644 samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/RetriableException.java create mode 100644 samza-core/src/main/java/org/apache/samza/config/BlobStoreConfig.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreStateBackendFactory.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/diff/DirDiff.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/index/DirIndex.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileBlob.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileIndex.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileMetadata.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotIndex.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotMetadata.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonDirIndexMixin.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileBlobMixin.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileIndexMixin.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileMetadataMixin.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotIndexMixin.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotMetadataMixin.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/SnapshotIndexSerde.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreBackupManagerMetrics.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreRestoreManagerMetrics.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java create mode 100644 samza-core/src/main/java/org/apache/samza/storage/blobstore/util/DirDiffUtil.java create mode 100644 samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreBackupManager.java create mode 100644 samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java create mode 100644 samza-core/src/test/java/org/apache/samza/storage/blobstore/serde/TestSnapshotIndexSerde.java create mode 100644 samza-core/src/test/java/org/apache/samza/storage/blobstore/util/BlobStoreTestUtil.java create mode 100644 samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestBlobStoreUtil.java create mode 100644 samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtil.java create mode 100644 samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtilMisc.java diff --git a/build.gradle b/build.gradle index 9c48b2382d..a3512e07a5 100644 --- a/build.gradle +++ b/build.gradle @@ -191,6 +191,7 @@ project(":samza-core_$scalaSuffix") { compile "org.apache.commons:commons-lang3:$commonsLang3Version" compile "commons-io:commons-io:$commonsIoVersion" compile "com.fasterxml.jackson.core:jackson-databind:$jacksonVersion" + compile "com.fasterxml.jackson.datatype:jackson-datatype-jdk8:$jacksonVersion" compile "org.eclipse.jetty:jetty-webapp:$jettyVersion" compile "org.scala-lang:scala-library:$scalaVersion" compile "org.slf4j:slf4j-api:$slf4jVersion" diff --git a/samza-api/src/main/java/org/apache/samza/storage/BlobStoreAdminFactory.java b/samza-api/src/main/java/org/apache/samza/storage/BlobStoreAdminFactory.java new file mode 100644 index 0000000000..6070ef1be6 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/storage/BlobStoreAdminFactory.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.samza.storage; + +import org.apache.samza.config.Config; +import org.apache.samza.job.model.JobModel; + + +/** + * Factory to create instance of {@link StateBackendAdmin}s that needs to be implemented for every + * state backend + */ +public interface BlobStoreAdminFactory { + /** + * Returns an instance of {@link StateBackendAdmin} + * @param config job configuration + * @param jobModel Job Model + */ + StateBackendAdmin getStateBackendAdmin(Config config, JobModel jobModel); +} diff --git a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java b/samza-api/src/main/java/org/apache/samza/storage/StateBackendAdmin.java similarity index 76% rename from samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java rename to samza-api/src/main/java/org/apache/samza/storage/StateBackendAdmin.java index 205077b309..2cdb86f4e4 100644 --- a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java +++ b/samza-api/src/main/java/org/apache/samza/storage/StateBackendAdmin.java @@ -20,11 +20,16 @@ package org.apache.samza.storage; /** - * Creates and validate resources for the StateBackendFactory + * Admin responsible for loading any resources related to state backend */ -public interface TaskStorageAdmin { - +public interface StateBackendAdmin { + /** + * Create all the resources required per job per store state backend + */ void createResources(); + /** + * Validate all resources required per job per state for state backend + */ void validateResources(); } diff --git a/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java b/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java index 54a9a81b91..f51d414815 100644 --- a/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java +++ b/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java @@ -24,18 +24,20 @@ import org.apache.samza.config.Config; import org.apache.samza.context.ContainerContext; import org.apache.samza.context.JobContext; +import org.apache.samza.job.model.ContainerModel; +import org.apache.samza.job.model.JobModel; import org.apache.samza.job.model.TaskModel; import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.util.Clock; /** - * Factory to build the Samza {@link TaskBackupManager}, {@link TaskRestoreManager} and {@link TaskStorageAdmin} + * Factory to build the Samza {@link TaskBackupManager}, {@link TaskRestoreManager} and {@link StateBackendAdmin} * for a particular state storage backend, which are used to durably backup the Samza task state. */ public interface StateBackendFactory { TaskBackupManager getBackupManager(JobContext jobContext, - ContainerContext containerContext, + ContainerModel containerModel, TaskModel taskModel, ExecutorService backupExecutor, MetricsRegistry taskInstanceMetricsRegistry, @@ -55,5 +57,5 @@ TaskRestoreManager getRestoreManager(JobContext jobContext, File nonLoggedStoreBaseDir, KafkaChangelogRestoreParams kafkaChangelogRestoreParams); - TaskStorageAdmin getAdmin(); + StateBackendAdmin getAdmin(JobModel jobModel, Config config); } diff --git a/samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManager.java b/samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManager.java new file mode 100644 index 0000000000..ef3f0c6961 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManager.java @@ -0,0 +1,83 @@ +/* + * 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.samza.storage.blobstore; + +import java.io.InputStream; +import java.io.OutputStream; +import java.util.concurrent.CompletionStage; +import org.apache.samza.annotation.InterfaceStability; + + +/** + * Provides interface for common blob store operations: GET, PUT and DELETE + */ +@InterfaceStability.Unstable +public interface BlobStoreManager { + /** + * Initialize underlying blob store client, if necessary. + * + */ + void init(); + + /** + * Non-blocking PUT call to remote blob store with supplied metadata + * @param inputStream InputStream to read the file + * @param metadata user supplied {@link Metadata} of the request + * @return a future containing the blob ID of the uploaded blob if the upload is successful. + */ + CompletionStage put(InputStream inputStream, Metadata metadata); + + /** + * Non-blocking GET call to remote blob store + * @param id Blob ID of the blob to get + * @param outputStream OutputStream to write the downloaded blob + * @param metadata User supplied {@link Metadata} of the request + * @return A future that completes when all the chunks are downloaded and written successfully to the OutputStream + * @throws org.apache.samza.storage.blobstore.exceptions.DeletedException returned future should complete + * exceptionally with DeletedException on failure with the blob already deleted error. + */ + CompletionStage get(String id, OutputStream outputStream, Metadata metadata); + + /** + * Non-blocking call to mark a blob for deletion in the remote blob store + * @param id Blob ID of the blob to delete + * @param metadata User supplied {@link Metadata} of the request + * @return A future that completes when the blob is successfully deleted from the blob store. + * @throws org.apache.samza.storage.blobstore.exceptions.DeletedException returned future should complete + * exceptionally with DeletedException on failure with the blob already deleted error. This exception is + * caught and ignored by the caller of the delete method during initial cleanup and SnapshotIndex read. + */ + CompletionStage delete(String id, Metadata metadata); + + /** + * Non-blocking call to remove the Time-To-Live (TTL) for a blob and make it permanent. + * @param blobId Blob ID of blob to remove TTL for. + * @param metadata User supplied {@link Metadata} of the request + * @return a future that completes when the TTL for the blob is removed. + * @throws org.apache.samza.storage.blobstore.exceptions.DeletedException returned future should complete + * exceptionally with DeletedException on failure with the blob already deleted error. + */ + CompletionStage removeTTL(String blobId, Metadata metadata); + + /** + * Cleanly close resources like blob store client + */ + void close(); +} diff --git a/samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManagerFactory.java b/samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManagerFactory.java new file mode 100644 index 0000000000..5a913930a3 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManagerFactory.java @@ -0,0 +1,31 @@ +/* + * 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.samza.storage.blobstore; + +import java.util.concurrent.ExecutorService; +import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.config.Config; + +@InterfaceStability.Unstable +public interface BlobStoreManagerFactory { + BlobStoreManager getBackupBlobStoreManager(Config config, ExecutorService backupExecutor); + + BlobStoreManager getRestoreBlobStoreManager(Config config, ExecutorService restoreExecutor); +} diff --git a/samza-api/src/main/java/org/apache/samza/storage/blobstore/Metadata.java b/samza-api/src/main/java/org/apache/samza/storage/blobstore/Metadata.java new file mode 100644 index 0000000000..3423f0bed5 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/storage/blobstore/Metadata.java @@ -0,0 +1,114 @@ +/* + * 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.samza.storage.blobstore; + +import java.util.Optional; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; + + +/** + * Metadata associated with every BlobStore request. This class is used to trace a request and to determine the + * bucket/container informationof the blob. + */ +public class Metadata { + public static final String SNAPSHOT_INDEX_PAYLOAD_PATH = "snapshot-index"; + + private final String payloadPath; + private final long payloadSize; + private final String jobName; + private final String jobId; + private final String taskName; + private final String storeName; + + public Metadata(String payloadPath, Optional payloadSize, + String jobName, String jobId, String taskName, String storeName) { + this.payloadPath = payloadPath; + // Payload size may not be known in advance for requests like getSnapshotIndex, where only blob ID is known. Set -1. + this.payloadSize = payloadSize.orElse(-1L); + this.jobName = jobName; + this.jobId = jobId; + this.taskName = taskName; + this.storeName = storeName; + } + + public String getPayloadPath() { + return payloadPath; + } + + public long getPayloadSize() { + return payloadSize; + } + + public String getJobName() { + return jobName; + } + + public String getJobId() { + return jobId; + } + + public String getTaskName() { + return taskName; + } + + public String getStoreName() { + return storeName; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof Metadata)) { + return false; + } + + Metadata that = (Metadata) o; + + return new EqualsBuilder().append(getPayloadPath(), that.getPayloadPath()) + .append(getPayloadSize(), that.getPayloadSize()) + .append(getJobName(), that.getJobName()) + .append(getJobId(), that.getJobId()) + .append(getTaskName(), that.getTaskName()) + .append(getStoreName(), that.getStoreName()) + .isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(17, 37).append(getPayloadPath()) + .append(getPayloadSize()) + .append(getJobName()) + .append(getJobId()) + .append(getTaskName()) + .append(getStoreName()) + .toHashCode(); + } + + @Override + public String toString() { + return "Metadata{" + "payloadPath='" + payloadPath + '\'' + ", payloadSize='" + payloadSize + '\'' + + ", jobName='" + jobName + '\'' + ", jobId='" + jobId + '\'' + ", taskName='" + taskName + '\'' + + ", storeName='" + storeName + '\'' + '}'; + } +} diff --git a/samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/DeletedException.java b/samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/DeletedException.java new file mode 100644 index 0000000000..8fcd2309b0 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/DeletedException.java @@ -0,0 +1,46 @@ +/* + * 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.samza.storage.blobstore.exceptions; + +/** + * Future should complete with this exception to indicate that the exception occurred due to the request for an + * already deleted blob. This exception is caught and ignored by caller of the DELETE request during initial cleanup + * and snapshot index read. + * + */ +public class DeletedException extends RuntimeException { + + private static final long serialVersionUID = 1L; + + public DeletedException(String message, Throwable cause) { + super(message, cause); + } + + public DeletedException(String message) { + super(message); + } + + public DeletedException(Throwable cause) { + super(cause); + } + + public DeletedException() { + } +} diff --git a/samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/RetriableException.java b/samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/RetriableException.java new file mode 100644 index 0000000000..e4ff789957 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/RetriableException.java @@ -0,0 +1,42 @@ +/* + * 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.samza.storage.blobstore.exceptions; + + +public class RetriableException extends RuntimeException { + + private static final long serialVersionUID = 1L; + + public RetriableException(String message, Throwable cause) { + super(message, cause); + } + + public RetriableException(String message) { + super(message); + } + + public RetriableException(Throwable cause) { + super(cause); + } + + public RetriableException() { + } +} + diff --git a/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java b/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java index 08bcfda8fd..e956413884 100644 --- a/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java +++ b/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java @@ -59,10 +59,13 @@ import org.apache.samza.metrics.MetricsRegistryMap; import org.apache.samza.startpoint.StartpointManager; import org.apache.samza.storage.ChangelogStreamManager; +import org.apache.samza.storage.StateBackendAdmin; +import org.apache.samza.storage.StateBackendFactory; import org.apache.samza.system.StreamMetadataCache; import org.apache.samza.system.SystemAdmins; import org.apache.samza.system.SystemStream; import org.apache.samza.util.DiagnosticsUtil; +import org.apache.samza.util.ReflectionUtil; import org.apache.samza.util.SystemClock; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -267,6 +270,18 @@ public void run() { MetadataResourceUtil metadataResourceUtil = new MetadataResourceUtil(jobModel, this.metrics, config); metadataResourceUtil.createResources(); + // create all the resources required for state backend factories + StorageConfig storageConfig = new StorageConfig(config); + storageConfig.getBackupFactories().forEach(stateStorageBackendBackupFactory -> { + StateBackendFactory stateBackendFactory = + ReflectionUtil.getObj(stateStorageBackendBackupFactory, StateBackendFactory.class); + StateBackendAdmin stateBackendAdmin = stateBackendFactory.getAdmin(jobModel, config); + // Create resources required for state backend admin + stateBackendAdmin.createResources(); + // Validate resources required for state backend admin + stateBackendAdmin.validateResources(); + }); + /* * We fanout startpoint if and only if * 1. Startpoint is enabled in configuration diff --git a/samza-core/src/main/java/org/apache/samza/config/BlobStoreConfig.java b/samza-core/src/main/java/org/apache/samza/config/BlobStoreConfig.java new file mode 100644 index 0000000000..769ba29ca6 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/config/BlobStoreConfig.java @@ -0,0 +1,41 @@ +/* + * 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.samza.config; + +/** + * Config related helper methods for BlobStore. + */ +public class BlobStoreConfig extends MapConfig { + + public static final String BLOB_STORE_MANAGER_FACTORY = "blob.store.manager.factory"; + public static final String BLOB_STORE_ADMIN_FACTORY = "blob.store.admin.factory"; + public BlobStoreConfig(Config config) { + super(config); + } + + + public String getBlobStoreManagerFactory() { + return get(BLOB_STORE_MANAGER_FACTORY); + } + + public String getBlobStoreAdminFactory() { + return get(BLOB_STORE_ADMIN_FACTORY); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java index 4dd753e370..aa8df958fb 100644 --- a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java @@ -66,13 +66,15 @@ public class StorageConfig extends MapConfig { public static final String CHANGELOG_MIN_COMPACTION_LAG_MS = STORE_PREFIX + "%s.changelog." + MIN_COMPACTION_LAG_MS; public static final long DEFAULT_CHANGELOG_MIN_COMPACTION_LAG_MS = TimeUnit.HOURS.toMillis(4); - public static final String DEFAULT_STATE_BACKEND_FACTORY = "org.apache.samza.storage.KafkaChangelogStateBackendFactory"; - public static final String STORE_BACKEND_BACKUP_FACTORIES = STORE_PREFIX + "%s.state.backend.backup.factories"; - public static final List DEFAULT_STATE_BACKEND_BACKUP_FACTORIES = ImmutableList.of( - DEFAULT_STATE_BACKEND_FACTORY); - public static final String STATE_BACKEND_RESTORE_FACTORY = STORE_PREFIX + "state.restore.backend"; public static final String INMEMORY_KV_STORAGE_ENGINE_FACTORY = "org.apache.samza.storage.kv.inmemory.InMemoryKeyValueStorageEngineFactory"; + public static final String KAFKA_STATE_BACKEND_FACTORY = + "org.apache.samza.storage.KafkaChangelogStateBackendFactory"; + public static final List DEFAULT_BACKUP_FACTORIES = ImmutableList.of( + KAFKA_STATE_BACKEND_FACTORY); + public static final String STORE_BACKUP_FACTORIES = STORE_PREFIX + "%s.backup.factories"; + // TODO BLOCKER dchen make this per store + public static final String STORE_RESTORE_FACTORY = STORE_PREFIX + "restore.factory"; static final String CHANGELOG_SYSTEM = "job.changelog.system"; static final String CHANGELOG_DELETE_RETENTION_MS = STORE_PREFIX + "%s.changelog.delete.retention.ms"; @@ -138,15 +140,6 @@ public Optional getChangelogStream(String storeName) { return Optional.ofNullable(systemStreamRes); } - public List getStoreBackupManagerClassName(String storeName) { - List storeBackupManagers = getList(String.format(STORE_BACKEND_BACKUP_FACTORIES, storeName), new ArrayList<>()); - // For backwards compatibility if the changelog is enabled, we use default kafka backup factory - if (storeBackupManagers.isEmpty() && getChangelogStream(storeName).isPresent()) { - storeBackupManagers = DEFAULT_STATE_BACKEND_BACKUP_FACTORIES; - } - return storeBackupManagers; - } - public boolean getAccessLogEnabled(String storeName) { return getBoolean(String.format(ACCESSLOG_ENABLED, storeName), false); } @@ -265,24 +258,6 @@ public long getChangelogMinCompactionLagMs(String storeName) { return getLong(minCompactLagConfigName, getDefaultChangelogMinCompactionLagMs()); } - - public Set getStateBackendBackupFactories() { - return getStoreNames().stream() - .flatMap((storeName) -> getStoreBackupManagerClassName(storeName).stream()) - .collect(Collectors.toSet()); - } - - public List getBackupStoreNamesForStateBackupFactory(String backendFactoryName) { - return getStoreNames().stream() - .filter((storeName) -> getStoreBackupManagerClassName(storeName) - .contains(backendFactoryName)) - .collect(Collectors.toList()); - } - - public String getStateBackendRestoreFactory() { - return get(STATE_BACKEND_RESTORE_FACTORY, DEFAULT_STATE_BACKEND_FACTORY); - } - /** * Helper method to check if a system has a changelog attached to it. */ @@ -313,11 +288,44 @@ public int getNumPersistentStores() { .count(); } + public List getStoreBackupFactory(String storeName) { + List storeBackupManagers = getList(String.format(STORE_BACKUP_FACTORIES, storeName), new ArrayList<>()); + // For backwards compatibility if the changelog is enabled, we use default kafka backup factory + if (storeBackupManagers.isEmpty() && getChangelogStream(storeName).isPresent()) { + storeBackupManagers = DEFAULT_BACKUP_FACTORIES; + } + return storeBackupManagers; + } + + public Set getBackupFactories() { + return getStoreNames().stream() + .flatMap((storeName) -> getStoreBackupFactory(storeName).stream()) + .collect(Collectors.toSet()); + } + + public String getRestoreFactory() { + return get(STORE_RESTORE_FACTORY, KAFKA_STATE_BACKEND_FACTORY); + } + + public List getStoresWithBackupFactory(String backendFactoryName) { + return getStoreNames().stream() + .filter((storeName) -> getStoreBackupFactory(storeName) + .contains(backendFactoryName)) + .collect(Collectors.toList()); + } + + // TODO BLOCKER dchen update when making restore managers per store + public List getStoresWithRestoreFactory(String backendFactoryName) { + return getStoreNames().stream() + .filter((storeName) -> getRestoreFactory().equals(backendFactoryName)) + .collect(Collectors.toList()); + } + /** * Helper method to get if logged store dirs should be deleted regardless of their contents. * @return */ - public boolean getCleanLoggedStoreDirsOnStart(String storeName) { + public boolean cleanLoggedStoreDirsOnStart(String storeName) { return getBoolean(String.format(CLEAN_LOGGED_STOREDIRS_ON_START, storeName), false); } } diff --git a/samza-core/src/main/java/org/apache/samza/standalone/PassthroughJobCoordinator.java b/samza-core/src/main/java/org/apache/samza/standalone/PassthroughJobCoordinator.java index cc51732fb9..7bf29fe6af 100644 --- a/samza-core/src/main/java/org/apache/samza/standalone/PassthroughJobCoordinator.java +++ b/samza-core/src/main/java/org/apache/samza/standalone/PassthroughJobCoordinator.java @@ -19,25 +19,27 @@ package org.apache.samza.standalone; import com.google.common.collect.ImmutableMap; +import java.util.Collections; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.container.grouper.task.GrouperMetadata; import org.apache.samza.container.grouper.task.GrouperMetadataImpl; import org.apache.samza.coordinator.JobCoordinator; +import org.apache.samza.coordinator.JobCoordinatorListener; import org.apache.samza.coordinator.JobModelManager; import org.apache.samza.coordinator.MetadataResourceUtil; import org.apache.samza.job.model.JobModel; -import org.apache.samza.coordinator.JobCoordinatorListener; +import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.runtime.LocationId; import org.apache.samza.runtime.LocationIdProvider; import org.apache.samza.runtime.LocationIdProviderFactory; -import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.system.StreamMetadataCache; import org.apache.samza.system.SystemAdmins; -import org.apache.samza.util.*; +import org.apache.samza.util.ReflectionUtil; +import org.apache.samza.util.SystemClock; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; + /** * Standalone Job Coordinator does not implement any leader elector module or cluster manager @@ -125,7 +127,9 @@ public JobModel getJobModel() { systemAdmins.start(); try { String containerId = Integer.toString(config.getInt(JobConfig.PROCESSOR_ID)); - GrouperMetadata grouperMetadata = new GrouperMetadataImpl(ImmutableMap.of(String.valueOf(containerId), locationId), Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap()); + GrouperMetadata grouperMetadata = + new GrouperMetadataImpl(ImmutableMap.of(String.valueOf(containerId), locationId), Collections.emptyMap(), + Collections.emptyMap(), Collections.emptyMap()); return JobModelManager.readJobModel(this.config, Collections.emptyMap(), streamMetadataCache, grouperMetadata); } finally { systemAdmins.stop(); diff --git a/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java b/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java index 36ea5b5923..54f20da719 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java +++ b/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java @@ -28,13 +28,13 @@ import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; import org.apache.commons.collections4.MapUtils; -import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.config.StorageConfig; import org.apache.samza.config.TaskConfig; import org.apache.samza.context.ContainerContext; import org.apache.samza.context.JobContext; import org.apache.samza.job.model.ContainerModel; +import org.apache.samza.job.model.JobModel; import org.apache.samza.job.model.TaskMode; import org.apache.samza.job.model.TaskModel; import org.apache.samza.metrics.MetricsRegistry; @@ -46,29 +46,13 @@ import org.apache.samza.util.Clock; -/** - * Class used the provide the {@link TaskRestoreManager} and the {@link TaskBackupManager} for the Kafka changelog - * state backend. - */ public class KafkaChangelogStateBackendFactory implements StateBackendFactory { private StreamMetadataCache streamCache; - /* - * This keeps track of the changelog SSPs that are associated with the whole container. This is used so that we can - * prefetch the metadata about the all of the changelog SSPs associated with the container whenever we need the - * metadata about some of the changelog SSPs. - * An example use case is when Samza writes offset files for stores ({@link TaskStorageManager}). Each task is - * responsible for its own offset file, but if we can do prefetching, then most tasks will already have cached - * metadata by the time they need the offset metadata. - * Note: By using all changelog streams to build the sspsToPrefetch, any fetches done for persisted stores will - * include the ssps for non-persisted stores, so this is slightly suboptimal. However, this does not increase the - * actual number of calls to the {@link SystemAdmin}, and we can decouple this logic from the per-task objects (e.g. - * {@link TaskStorageManager}). - */ private SSPMetadataCache sspCache; @Override public TaskBackupManager getBackupManager(JobContext jobContext, - ContainerContext containerContext, + ContainerModel containerModel, TaskModel taskModel, ExecutorService backupExecutor, MetricsRegistry metricsRegistry, @@ -101,7 +85,10 @@ public TaskRestoreManager getRestoreManager(JobContext jobContext, File nonLoggedStoreBaseDir, KafkaChangelogRestoreParams kafkaChangelogRestoreParams) { Map storeChangelogs = new StorageConfig(config).getStoreChangelogs(); - Set changelogSSPs = getChangelogSSPForContainer(storeChangelogs, containerContext); + Set changelogSSPs = storeChangelogs.values().stream() + .flatMap(ss -> containerContext.getContainerModel().getTasks().values().stream() + .map(tm -> new SystemStreamPartition(ss, tm.getChangelogPartition()))) + .collect(Collectors.toSet()); // filter out standby store-ssp pairs Map filteredStoreChangelogs = filterStandbySystemStreams(storeChangelogs, containerContext.getContainerModel()); @@ -152,12 +139,12 @@ public TaskRestoreManager getRestoreManager(JobContext jobContext, } @Override - public TaskStorageAdmin getAdmin() { - throw new SamzaException("getAdmin() method not supported for KafkaStateBackendFactory"); + //TODO HIGH snjain implement this + public StateBackendAdmin getAdmin(JobModel jobModel, Config config) { + return new NoOpKafkaChangelogStateBackendAdmin(); } - @VisibleForTesting - Set getChangelogSSPForContainer(Map storeChangelogs, + public Set getChangelogSSPForContainer(Map storeChangelogs, ContainerContext containerContext) { return storeChangelogs.values().stream() .flatMap(ss -> containerContext.getContainerModel().getTasks().values().stream() @@ -188,8 +175,7 @@ StreamMetadataCache getStreamCache(SystemAdmins admins, Clock clock) { * @param ssps SSPs to prefetch * @return SSPMetadataCache containing the partition metadata */ - @VisibleForTesting - SSPMetadataCache getSspCache(SystemAdmins admins, Clock clock, Set ssps) { + private SSPMetadataCache getSspCache(SystemAdmins admins, Clock clock, Set ssps) { if (sspCache == null) { sspCache = new SSPMetadataCache(admins, Duration.ofSeconds(5), clock, ssps); } @@ -201,15 +187,37 @@ Map filterStandbySystemStreams(Map c ContainerModel containerModel) { Map changelogSSPToStore = new HashMap<>(); changelogSystemStreams.forEach((storeName, systemStream) -> - containerModel.getTasks().forEach((taskName, taskModel) -> { - if (!TaskMode.Standby.equals(taskModel.getTaskMode())) { - changelogSSPToStore.put(new SystemStreamPartition(systemStream, taskModel.getChangelogPartition()), - storeName); - } - }) + containerModel.getTasks().forEach((taskName, taskModel) -> + changelogSSPToStore.put(new SystemStreamPartition(systemStream, taskModel.getChangelogPartition()), storeName)) ); + + Set standbyTaskModels = containerModel.getTasks().values().stream() + .filter(taskModel -> taskModel.getTaskMode().equals(TaskMode.Standby)) + .collect(Collectors.toSet()); + + // remove all standby task changelog ssps + standbyTaskModels.forEach((taskModel) -> { + changelogSystemStreams.forEach((storeName, systemStream) -> { + SystemStreamPartition ssp = new SystemStreamPartition(systemStream, taskModel.getChangelogPartition()); + changelogSSPToStore.remove(ssp); + }); + }); + // changelogSystemStreams correspond only to active tasks (since those of standby-tasks moved to sideInputs above) return MapUtils.invertMap(changelogSSPToStore).entrySet().stream() .collect(Collectors.toMap(Map.Entry::getKey, x -> x.getValue().getSystemStream())); } + + public class NoOpKafkaChangelogStateBackendAdmin implements StateBackendAdmin { + + @Override + public void createResources() { + // all the changelog creations are handled by {@link ChangelogStreamManager} + } + + @Override + public void validateResources() { + // all the changelog validations are handled by {@link ChangelogStreamManager} + } + } } diff --git a/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java index f648b0651d..35fa375dfd 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java @@ -156,7 +156,7 @@ private void cleanBaseDirsAndReadOffsetFiles() { LOG.info("Got logged storage partition directory as " + loggedStorePartitionDir.toPath().toString()); // Delete the logged store if it is not valid. - if (!isLoggedStoreValid(storeName, loggedStorePartitionDir) || storageConfig.getCleanLoggedStoreDirsOnStart(storeName)) { + if (!isLoggedStoreValid(storeName, loggedStorePartitionDir) || storageConfig.cleanLoggedStoreDirsOnStart(storeName)) { LOG.info("Deleting logged storage partition directory " + loggedStorePartitionDir.toPath().toString()); fileUtil.rm(loggedStorePartitionDir); } else { diff --git a/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java b/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java index 91c0c7cf91..057b248ae9 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java +++ b/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java @@ -20,12 +20,11 @@ package org.apache.samza.storage; import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; -import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableMap; - import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -70,17 +69,6 @@ public class StorageManagerUtil { private static final String SST_FILE_SUFFIX = ".sst"; private static final CheckpointV2Serde CHECKPOINT_V2_SERDE = new CheckpointV2Serde(); - /** - * Returns the path for a storage engine to create its checkpoint based on the current checkpoint id. - * - * @param taskStoreDir directory of the store as returned by {@link #getTaskStoreDir} - * @param checkpointId current checkpoint id - * @return String denoting the file path of the store with the given checkpoint id - */ - public static String getCheckpointDirPath(File taskStoreDir, CheckpointId checkpointId) { - return taskStoreDir.getPath() + "-" + checkpointId.serialize(); - } - /** * Fetch the starting offset for the input {@link SystemStreamPartition} * @@ -405,6 +393,17 @@ public List getTaskStoreCheckpointDirs(File storeBaseDir, String storeName } } + /** + * Returns the path for a storage engine to create its checkpoint based on the current checkpoint id. + * + * @param taskStoreDir directory of the store as returned by {@link #getTaskStoreDir} + * @param checkpointId current checkpoint id + * @return String denoting the file path of the store with the given checkpoint id + */ + public String getStoreCheckpointDir(File taskStoreDir, CheckpointId checkpointId) { + return taskStoreDir.getPath() + "-" + checkpointId.serialize(); + } + public void restoreCheckpointFiles(File checkpointDir, File storeDir) { // the current task store dir should already be deleted for restore assert !storeDir.exists(); diff --git a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java index 0925949b67..8091e482ef 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java +++ b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java @@ -215,7 +215,7 @@ private Map> getSerdes() { */ @SuppressWarnings("rawtypes") private void getContainerStorageManagers() { - String factoryClass = new StorageConfig(jobConfig).getStateBackendRestoreFactory(); + String factoryClass = new StorageConfig(jobConfig).getRestoreFactory(); Clock clock = SystemClock.instance(); StreamMetadataCache streamMetadataCache = new StreamMetadataCache(systemAdmins, 5000, clock); // don't worry about prefetching for this; looks like the tool doesn't flush to offset files anyways diff --git a/samza-core/src/main/java/org/apache/samza/storage/TaskSideInputStorageManager.java b/samza-core/src/main/java/org/apache/samza/storage/TaskSideInputStorageManager.java index f407b1a7a6..61d30c3fd0 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/TaskSideInputStorageManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/TaskSideInputStorageManager.java @@ -20,15 +20,6 @@ package org.apache.samza.storage; import com.google.common.annotations.VisibleForTesting; -import org.apache.samza.SamzaException; -import org.apache.samza.container.TaskName; -import org.apache.samza.job.model.TaskMode; -import org.apache.samza.system.SystemStreamPartition; -import org.apache.samza.util.Clock; -import org.apache.samza.util.FileUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.util.HashMap; import java.util.Map; @@ -37,6 +28,14 @@ import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; +import org.apache.samza.SamzaException; +import org.apache.samza.container.TaskName; +import org.apache.samza.job.model.TaskMode; +import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.util.Clock; +import org.apache.samza.util.FileUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** diff --git a/samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java b/samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java index cc80a48f42..09a8807104 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java @@ -116,8 +116,9 @@ public Map> snapshot(CheckpointId checkpointId) { LOG.debug("Flushed all storage engines for taskName: {}, checkpoint id: {}", taskName, checkpointId); - long checkpointStartNs = System.nanoTime(); + // Checkpoint all persisted and durable stores + long checkpointStartNs = System.nanoTime(); storageEngines.forEach((storeName, storageEngine) -> { if (storageEngine.getStoreProperties().isPersistedToDisk() && storageEngine.getStoreProperties().isDurableStore()) { @@ -212,7 +213,7 @@ public void writeCheckpointToStoreDirectories(Checkpoint checkpoint) { storageManagerUtil.writeCheckpointV2File(storeDir, checkpointV2); CheckpointId checkpointId = checkpointV2.getCheckpointId(); - File checkpointDir = Paths.get(StorageManagerUtil.getCheckpointDirPath(storeDir, checkpointId)).toFile(); + File checkpointDir = Paths.get(storageManagerUtil.getStoreCheckpointDir(storeDir, checkpointId)).toFile(); storageManagerUtil.writeCheckpointV2File(checkpointDir, checkpointV2); } catch (Exception e) { throw new SamzaException( @@ -334,7 +335,7 @@ void writeChangelogOffsetFiles(Map checkpointOffs writeChangelogOffsetFile(storeName, changelogSSP, newestOffset, currentStoreDir); // Write changelog SSP offset to the OFFSET files in the store checkpoint directory - File checkpointDir = Paths.get(StorageManagerUtil.getCheckpointDirPath( + File checkpointDir = Paths.get(storageManagerUtil.getStoreCheckpointDir( currentStoreDir, kafkaChangelogSSPOffset.getCheckpointId())).toFile(); writeChangelogOffsetFile(storeName, changelogSSP, newestOffset, checkpointDir); } else { diff --git a/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java index 1329a0df84..20f653c76e 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java @@ -24,7 +24,6 @@ import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ListMultimap; - import java.io.File; import java.util.Collections; import java.util.HashMap; @@ -287,7 +286,7 @@ static StoreActions getStoreActions( // if the clean.store.start config is set, delete current and checkpoint dirs, restore from oldest offset to checkpointed if (storageEngine.getStoreProperties().isPersistedToDisk() && new StorageConfig( - config).getCleanLoggedStoreDirsOnStart(storeName)) { + config).cleanLoggedStoreDirsOnStart(storeName)) { File currentDir = storageManagerUtil.getTaskStoreDir(loggedStoreBaseDirectory, storeName, taskName, taskMode); LOG.info("Marking current directory: {} for store: {} in task: {} for deletion due to clean.on.container.start config.", currentDir, storeName, taskName); diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java new file mode 100644 index 0000000000..15cc87b6d8 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java @@ -0,0 +1,350 @@ +/* + * 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.samza.storage.blobstore; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableMap; +import java.io.File; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.samza.SamzaException; +import org.apache.samza.checkpoint.Checkpoint; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.StorageConfig; +import org.apache.samza.job.model.ContainerModel; +import org.apache.samza.job.model.JobModel; +import org.apache.samza.job.model.TaskModel; +import org.apache.samza.storage.StorageManagerUtil; +import org.apache.samza.storage.TaskBackupManager; +import org.apache.samza.storage.blobstore.diff.DirDiff; +import org.apache.samza.storage.blobstore.index.DirIndex; +import org.apache.samza.storage.blobstore.index.SnapshotIndex; +import org.apache.samza.storage.blobstore.index.SnapshotMetadata; +import org.apache.samza.storage.blobstore.metrics.BlobStoreBackupManagerMetrics; +import org.apache.samza.storage.blobstore.util.BlobStoreUtil; +import org.apache.samza.storage.blobstore.util.DirDiffUtil; +import org.apache.samza.util.Clock; +import org.apache.samza.util.FutureUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class BlobStoreBackupManager implements TaskBackupManager { + private static final Logger LOG = LoggerFactory.getLogger(BlobStoreBackupManager.class); + + private final JobModel jobModel; + private final ExecutorService executor; + private final String jobName; + private final String jobId; + private final ContainerModel containerModel; + private final TaskModel taskModel; + private final String taskName; + private final Config config; + private final Clock clock; + private final StorageManagerUtil storageManagerUtil; + private final List storesToBackup; + private final File loggedStoreBaseDir; + private final BlobStoreManager blobStoreManager; + private final BlobStoreUtil blobStoreUtil; + private final BlobStoreBackupManagerMetrics metrics; + + /** + * Map of store name to a Pair of blob id of {@link SnapshotIndex} and the corresponding {@link SnapshotIndex} from + * last successful task checkpoint or {@link #upload}. + * + * After {@link #init}, the map reflects the contents of the last completed checkpoint for the task from the previous + * deployment, if any. + * + * During regular processing, this map is updated after each successful {@link #upload} with the blob id of + * {@link SnapshotIndex} and the corresponding {@link SnapshotIndex} of the upload. + * + * The contents of this map are used to calculate the diff for local state between the last and the current checkpoint + * during {@link #upload}. + * + * Since the task commit process guarantees that the async stage of the previous commit is complete before another + * commit can start, this future is guaranteed to be complete in the call to {@link #upload} during the next commit. + * + * This field is non-final, since the future itself is replaced in its entirety after init/upload. + * The internal map contents are never directly modified (e.g. using puts). It's volatile to ensure visibility + * across threads since the map assignment may happen on a different thread than the one reading the contents. + */ + private volatile CompletableFuture>> + prevStoreSnapshotIndexesFuture; + + public BlobStoreBackupManager(JobModel jobModel, ContainerModel containerModel, TaskModel taskModel, + ExecutorService backupExecutor, BlobStoreBackupManagerMetrics blobStoreTaskBackupMetrics, Config config, + Clock clock, File loggedStoreBaseDir, StorageManagerUtil storageManagerUtil, BlobStoreManager blobStoreManager) { + this.jobModel = jobModel; + this.jobName = new JobConfig(config).getName().get(); + this.jobId = new JobConfig(config).getJobId(); + this.containerModel = containerModel; + this.taskModel = taskModel; + this.taskName = taskModel.getTaskName().getTaskName(); + this.executor = backupExecutor; + this.config = config; + this.clock = clock; + this.storageManagerUtil = storageManagerUtil; + StorageConfig storageConfig = new StorageConfig(config); + this.storesToBackup = + storageConfig.getStoresWithBackupFactory(BlobStoreStateBackendFactory.class.getName()); + this.loggedStoreBaseDir = loggedStoreBaseDir; + this.blobStoreManager = blobStoreManager; + this.blobStoreUtil = createBlobStoreUtil(blobStoreManager, executor, blobStoreTaskBackupMetrics); + this.prevStoreSnapshotIndexesFuture = CompletableFuture.completedFuture(ImmutableMap.of()); + this.metrics = blobStoreTaskBackupMetrics; + metrics.initStoreMetrics(storesToBackup); + } + + @Override + public void init(Checkpoint checkpoint) { + long startTime = System.nanoTime(); + LOG.debug("Initializing blob store backup manager for task: {}", taskName); + + blobStoreManager.init(); + + // Note: blocks the caller thread. + // TODO LOW shesharma exclude stores that are no longer configured during init + Map> prevStoreSnapshotIndexes = + blobStoreUtil.getStoreSnapshotIndexes(jobName, jobId, taskName, checkpoint); + this.prevStoreSnapshotIndexesFuture = + CompletableFuture.completedFuture(ImmutableMap.copyOf(prevStoreSnapshotIndexes)); + metrics.initNs.set(System.nanoTime() - startTime); + } + + @Override + public Map snapshot(CheckpointId checkpointId) { + // No-op. Stores are flushed and checkpoints are created by commit manager + return Collections.emptyMap(); + } + + @Override + public CompletableFuture> upload(CheckpointId checkpointId, Map storeSCMs) { + long uploadStartTime = System.nanoTime(); + + // reset gauges for each upload + metrics.filesToUpload.getValue().set(0L); + metrics.bytesToUpload.getValue().set(0L); + metrics.filesUploaded.getValue().set(0L); + metrics.bytesUploaded.getValue().set(0L); + metrics.filesRemaining.getValue().set(0L); + metrics.bytesRemaining.getValue().set(0L); + metrics.filesToRetain.getValue().set(0L); + metrics.bytesToRetain.getValue().set(0L); + + // This map is used to atomically replace the prevStoreSnapshotIndexesFuture map at the end of the task commit + Map>> + storeToSCMAndSnapshotIndexPairFutures = new HashMap<>(); + // This map is used to return serialized State Checkpoint Markers to the caller + Map> storeToSerializedSCMFuture = new HashMap<>(); + + storesToBackup.forEach((storeName) -> { + long storeUploadStartTime = System.nanoTime(); + try { + // metadata for the current store snapshot to upload + SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName); + + // get the local store dir corresponding to the current checkpointId + File storeDir = storageManagerUtil.getTaskStoreDir(loggedStoreBaseDir, storeName, + taskModel.getTaskName(), taskModel.getTaskMode()); + String checkpointDirPath = storageManagerUtil.getStoreCheckpointDir(storeDir, checkpointId); + File checkpointDir = new File(checkpointDirPath); + + LOG.debug("Got task: {} store: {} storeDir: {} and checkpointDir: {}", + taskName, storeName, storeDir, checkpointDir); + + // guaranteed to be available since a new task commit may not start until the previous one is complete + Map> prevStoreSnapshotIndexes = + prevStoreSnapshotIndexesFuture.get(0, TimeUnit.MILLISECONDS); + + // get the previous store directory contents + DirIndex prevDirIndex; + + if (prevStoreSnapshotIndexes.containsKey(storeName)) { + prevDirIndex = prevStoreSnapshotIndexes.get(storeName).getRight().getDirIndex(); + } else { + // no previous SnapshotIndex means that this is the first commit for this store. Create an empty DirIndex. + prevDirIndex = new DirIndex(checkpointDir.getName(), Collections.emptyList(), Collections.emptyList(), + Collections.emptyList(), Collections.emptyList()); + } + + long dirDiffStartTime = System.nanoTime(); + // get the diff between previous and current store directories + DirDiff dirDiff = DirDiffUtil.getDirDiff(checkpointDir, prevDirIndex, DirDiffUtil.areSameFile(false)); + metrics.storeDirDiffNs.get(storeName).update(System.nanoTime() - dirDiffStartTime); + + DirDiff.Stats stats = DirDiff.getStats(dirDiff); + updateStoreDiffMetrics(storeName, stats); + metrics.filesToUpload.getValue().addAndGet(stats.filesAdded); + metrics.bytesToUpload.getValue().addAndGet(stats.bytesAdded); + // Note: FilesRemaining metric is set to FilesAdded in the beginning of the current upload and then counted down + // for each upload. + metrics.filesRemaining.getValue().addAndGet(stats.filesAdded); + metrics.bytesRemaining.getValue().addAndGet(stats.bytesAdded); + metrics.filesToRetain.getValue().addAndGet(stats.filesRetained); + metrics.bytesToRetain.getValue().addAndGet(stats.bytesRetained); + + // upload the diff to the blob store and get the new directory index + CompletionStage dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata); + + CompletionStage snapshotIndexFuture = + dirIndexFuture.thenApplyAsync(dirIndex -> { + LOG.trace("Dir upload complete. Returning new SnapshotIndex for task: {} store: {}.", taskName, storeName); + Optional prevSnapshotIndexBlobId = + Optional.ofNullable(prevStoreSnapshotIndexes.get(storeName)).map(Pair::getLeft); + return new SnapshotIndex(clock.currentTimeMillis(), snapshotMetadata, dirIndex, prevSnapshotIndexBlobId); + }, executor); + + // upload the new snapshot index to the blob store and get its blob id + CompletionStage snapshotIndexBlobIdFuture = + snapshotIndexFuture + .thenComposeAsync(si -> { + LOG.trace("Uploading Snapshot index for task: {} store: {}", taskName, storeName); + return blobStoreUtil.putSnapshotIndex(si); + }, executor); + + // save store name and it's SnapshotIndex blob id and SnapshotIndex pair. At the end of the upload, atomically + // update previous snapshot index map with this. + CompletableFuture> scmAndSnapshotIndexPairFuture = + FutureUtil.toFutureOfPair( + Pair.of(snapshotIndexBlobIdFuture.toCompletableFuture(), snapshotIndexFuture.toCompletableFuture())); + + scmAndSnapshotIndexPairFuture.whenComplete((res, ex) -> { + long uploadTimeNs = System.nanoTime() - storeUploadStartTime; + metrics.storeUploadNs.get(storeName).update(uploadTimeNs); + }); + + storeToSCMAndSnapshotIndexPairFutures.put(storeName, scmAndSnapshotIndexPairFuture); + storeToSerializedSCMFuture.put(storeName, snapshotIndexBlobIdFuture.toCompletableFuture()); + } catch (Exception e) { + throw new SamzaException( + String.format("Error uploading store snapshot to blob store for task: %s, store: %s, checkpointId: %s", + taskName, storeName, checkpointId), e); + } + }); + + // replace the previous storeName to snapshot index mapping with the new mapping. + this.prevStoreSnapshotIndexesFuture = + FutureUtil.toFutureOfMap(storeToSCMAndSnapshotIndexPairFutures); + + return FutureUtil.toFutureOfMap(storeToSerializedSCMFuture) + .whenComplete((res, ex) -> metrics.uploadNs.update(System.nanoTime() - uploadStartTime)); + } + + /** + * Clean up would be called at the end of every commit as well as on a container start/restart. + * Clean up involves the following steps: + * 1. Remove TTL of the snapshot index blob and for any associated files and sub-dirs marked for retention. + * 2. Delete the files/subdirs marked for deletion in the snapshot index. + * 3. Delete the remote {@link SnapshotIndex} blob for the previous checkpoint. + * @param checkpointId the {@link CheckpointId} of the last successfully committed checkpoint. + * @param storeSCMs store name to state checkpoint markers for the last successfully committed checkpoint + */ + @Override + public CompletableFuture cleanUp(CheckpointId checkpointId, Map storeSCMs) { + long startTime = System.nanoTime(); + List> removeTTLFutures = new ArrayList<>(); + List> cleanupRemoteSnapshotFutures = new ArrayList<>(); + List> removePrevRemoteSnapshotFutures = new ArrayList<>(); + + // SCM, in case of blob store backup and restore, is just the blob id of SnapshotIndex representing the remote snapshot + storeSCMs.forEach((storeName, snapshotIndexBlobId) -> { + // Only perform cleanup for stores configured with BlobStore State Backend Factory + if (storesToBackup.contains(storeName)) { + Metadata requestMetadata = + new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), jobName, jobId, taskName, storeName); + CompletionStage snapshotIndexFuture = + blobStoreUtil.getSnapshotIndex(snapshotIndexBlobId, requestMetadata); + + // 1. remove TTL of index blob and all of its files and sub-dirs marked for retention + CompletionStage removeTTLFuture = + snapshotIndexFuture.thenComposeAsync(snapshotIndex -> { + LOG.debug("Removing TTL for index blob: {} and all of its files and sub-dirs for task: {} store :{}", + snapshotIndexBlobId, taskName, storeName); + return blobStoreUtil.removeTTL(snapshotIndexBlobId, snapshotIndex, requestMetadata); + }, executor); + removeTTLFutures.add(removeTTLFuture); + + // 2. delete the files/subdirs marked for deletion in the snapshot index. + CompletionStage cleanupRemoteSnapshotFuture = + snapshotIndexFuture.thenComposeAsync(snapshotIndex -> { + LOG.debug("Deleting files and dirs to remove for current index blob: {} for task: {} store: {}", + snapshotIndexBlobId, taskName, storeName); + return blobStoreUtil.cleanUpDir(snapshotIndex.getDirIndex(), requestMetadata); + }, executor); + + cleanupRemoteSnapshotFutures.add(cleanupRemoteSnapshotFuture); + + // 3. delete the remote {@link SnapshotIndex} blob for the previous checkpoint. + CompletionStage removePrevRemoteSnapshotFuture = + snapshotIndexFuture.thenComposeAsync(snapshotIndex -> { + if (snapshotIndex.getPrevSnapshotIndexBlobId().isPresent()) { + String blobId = snapshotIndex.getPrevSnapshotIndexBlobId().get(); + LOG.debug("Removing previous snapshot index blob: {} from blob store for task: {} store: {}.", + blobId, taskName, storeName); + return blobStoreUtil.deleteSnapshotIndexBlob(blobId, requestMetadata); + } else { + // complete future immediately. There are no previous snapshots index blobs to delete. + return CompletableFuture.completedFuture(null); + } + }, executor); + removePrevRemoteSnapshotFutures.add(removePrevRemoteSnapshotFuture); + } + }); + + return FutureUtil.allOf(removeTTLFutures, cleanupRemoteSnapshotFutures, removePrevRemoteSnapshotFutures) + .whenComplete((res, ex) -> metrics.cleanupNs.update(System.nanoTime() - startTime)); + } + + @Override + public void close() { + blobStoreManager.close(); + } + + @VisibleForTesting + protected BlobStoreUtil createBlobStoreUtil(BlobStoreManager blobStoreManager, ExecutorService executor, + BlobStoreBackupManagerMetrics metrics) { + return new BlobStoreUtil(blobStoreManager, executor, metrics, null); + } + + private void updateStoreDiffMetrics(String storeName, DirDiff.Stats stats) { + metrics.storeFilesToUpload.get(storeName).set((long) stats.filesAdded); + metrics.storeFilesToRetain.get(storeName).set((long) stats.filesRetained); + metrics.storeFilesToRemove.get(storeName).set((long) stats.filesRemoved); + + metrics.storeSubDirsToUpload.get(storeName).set((long) stats.subDirsAdded); + metrics.storeSubDirsToRetain.get(storeName).set((long) stats.subDirsRetained); + metrics.storeSubDirsToRemove.get(storeName).set((long) stats.subDirsRemoved); + + metrics.storeBytesToUpload.get(storeName).set(stats.bytesAdded); + metrics.storeBytesToRetain.get(storeName).set(stats.bytesRetained); + metrics.storeBytesToRemove.get(storeName).set(stats.bytesRemoved); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java new file mode 100644 index 0000000000..052e49da89 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java @@ -0,0 +1,369 @@ +/* + * 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.samza.storage.blobstore; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableSet; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.ExecutorService; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.samza.SamzaException; +import org.apache.samza.checkpoint.Checkpoint; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.config.BlobStoreConfig; +import org.apache.samza.config.Config; +import org.apache.samza.config.JobConfig; +import org.apache.samza.config.StorageConfig; +import org.apache.samza.container.TaskName; +import org.apache.samza.job.model.TaskMode; +import org.apache.samza.job.model.TaskModel; +import org.apache.samza.storage.StorageManagerUtil; +import org.apache.samza.storage.TaskRestoreManager; +import org.apache.samza.storage.blobstore.index.DirIndex; +import org.apache.samza.storage.blobstore.index.SnapshotIndex; +import org.apache.samza.storage.blobstore.metrics.BlobStoreRestoreManagerMetrics; +import org.apache.samza.storage.blobstore.util.BlobStoreUtil; +import org.apache.samza.storage.blobstore.util.DirDiffUtil; +import org.apache.samza.util.FileUtil; +import org.apache.samza.util.FutureUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class BlobStoreRestoreManager implements TaskRestoreManager { + private static final Logger LOG = LoggerFactory.getLogger(BlobStoreRestoreManager.class); + // when checking if checkpoint dir is the same as remote snapshot, exclude the "OFFSET" family of files files + // that are written to the checkpoint dir after the remote upload is complete as part of + // TaskStorageCommitManager#writeCheckpointToStoreDirectories. + private static final Set FILES_TO_IGNORE = ImmutableSet.of( + StorageManagerUtil.OFFSET_FILE_NAME_LEGACY, + StorageManagerUtil.OFFSET_FILE_NAME_NEW, + StorageManagerUtil.SIDE_INPUT_OFFSET_FILE_NAME_LEGACY, + StorageManagerUtil.CHECKPOINT_FILE_NAME); + + private final TaskModel taskModel; + private final String jobName; + private final String jobId; + private final ExecutorService executor; + private final Config config; + private final StorageConfig storageConfig; + private final BlobStoreConfig blobStoreConfig; + private final StorageManagerUtil storageManagerUtil; + private final BlobStoreUtil blobStoreUtil; + private final DirDiffUtil dirDiffUtil; + private final File loggedBaseDir; + private final File nonLoggedBaseDir; + private final String taskName; + private final List storesToRestore; + private final BlobStoreRestoreManagerMetrics metrics; + + private BlobStoreManager blobStoreManager; + + /** + * Map of store name and Pair of blob id of SnapshotIndex and the corresponding SnapshotIndex from last successful + * task checkpoint + */ + private Map> prevStoreSnapshotIndexes; + + public BlobStoreRestoreManager(TaskModel taskModel, ExecutorService restoreExecutor, + BlobStoreRestoreManagerMetrics metrics, Config config, File loggedBaseDir, File nonLoggedBaseDir, + StorageManagerUtil storageManagerUtil, BlobStoreManager blobStoreManager) { + this.taskModel = taskModel; + this.jobName = new JobConfig(config).getName().get(); + this.jobId = new JobConfig(config).getJobId(); + this.executor = restoreExecutor; // TODO BLOCKER dchen1 dont block on restore executor + this.config = config; + this.storageConfig = new StorageConfig(config); + this.blobStoreConfig = new BlobStoreConfig(config); + this.storageManagerUtil = storageManagerUtil; + this.blobStoreManager = blobStoreManager; + this.blobStoreUtil = createBlobStoreUtil(blobStoreManager, executor, metrics); + this.dirDiffUtil = new DirDiffUtil(); + this.prevStoreSnapshotIndexes = new HashMap<>(); + this.loggedBaseDir = loggedBaseDir; + this.nonLoggedBaseDir = nonLoggedBaseDir; + this.taskName = taskModel.getTaskName().getTaskName(); + StorageConfig storageConfig = new StorageConfig(config); + this.storesToRestore = + storageConfig.getStoresWithRestoreFactory(BlobStoreStateBackendFactory.class.getName()); + this.metrics = metrics; + } + + @Override + public void init(Checkpoint checkpoint) { + long startTime = System.nanoTime(); + LOG.debug("Initializing blob store restore manager for task: {}", taskName); + + blobStoreManager.init(); + + // get previous SCMs from checkpoint + prevStoreSnapshotIndexes = blobStoreUtil.getStoreSnapshotIndexes(jobName, jobId, taskName, checkpoint); + metrics.getSnapshotIndexNs.set(System.nanoTime() - startTime); + LOG.trace("Found previous snapshot index during blob store restore manager init for task: {} to be: {}", + taskName, prevStoreSnapshotIndexes); + + metrics.initStoreMetrics(storesToRestore); + + // Note: blocks the caller thread. + deleteUnusedStoresFromBlobStore(jobName, jobId, taskName, storageConfig, blobStoreConfig, prevStoreSnapshotIndexes, + blobStoreUtil, executor); + metrics.initNs.set(System.nanoTime() - startTime); + } + + /** + * Restore state from checkpoints and state snapshots. + * State restore is performed by first retrieving the SnapshotIndex of the previous commit for every store from the + * prevStoreSnapshotIndexes map. Local store is deleted to perform a restore from local checkpoint directory or remote + * directory. If no local state checkpoint directory is found, or if the local checkpoint directory is different from + * the remote snapshot, local checkpoint directory is deleted and a restore from the remote store is done by + * downloading the state asynchronously and in parallel. + * + */ + @Override + public void restore() { + restoreStores(jobName, jobId, taskModel.getTaskName(), storesToRestore, prevStoreSnapshotIndexes, loggedBaseDir, + storageConfig, metrics, storageManagerUtil, blobStoreUtil, dirDiffUtil, executor); + } + + @Override + public void close() { + blobStoreManager.close(); + } + + @VisibleForTesting + protected BlobStoreUtil createBlobStoreUtil(BlobStoreManager blobStoreManager, ExecutorService executor, + BlobStoreRestoreManagerMetrics metrics) { + return new BlobStoreUtil(blobStoreManager, executor, null, metrics); + } + + /** + * Deletes blob store contents for stores that were present in the last checkpoint but are either no longer + * present in job configs (removed by user since last deploymetn) or are no longer configured to be backed + * up using blob stores. + * + * This method blocks until all the necessary store contents and snapshot index blobs have been marked for deletion. + */ + @VisibleForTesting + static void deleteUnusedStoresFromBlobStore(String jobName, String jobId, String taskName, StorageConfig storageConfig, + BlobStoreConfig blobStoreConfig, Map> initialStoreSnapshotIndexes, + BlobStoreUtil blobStoreUtil, ExecutorService executor) { + + List storesToBackup = + storageConfig.getStoresWithBackupFactory(BlobStoreStateBackendFactory.class.getName()); + List storesToRestore = + storageConfig.getStoresWithRestoreFactory(BlobStoreStateBackendFactory.class.getName()); + + List> storeDeletionFutures = new ArrayList<>(); + initialStoreSnapshotIndexes.forEach((storeName, scmAndSnapshotIndex) -> { + if (!storesToBackup.contains(storeName) && !storesToRestore.contains(storeName)) { + LOG.debug("Removing task: {} store: {} from blob store. It is either no longer used, " + + "or is no longer configured to be backed up or restored with blob store.", taskName, storeName); + DirIndex dirIndex = scmAndSnapshotIndex.getRight().getDirIndex(); + Metadata requestMetadata = + new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), jobName, jobId, taskName, storeName); + CompletionStage storeDeletionFuture = + blobStoreUtil.cleanUpDir(dirIndex, requestMetadata) // delete files and sub-dirs previously marked for removal + .thenComposeAsync(v -> + blobStoreUtil.deleteDir(dirIndex, requestMetadata), executor) // deleted files and dirs still present + .thenComposeAsync(v -> blobStoreUtil.deleteSnapshotIndexBlob( + scmAndSnapshotIndex.getLeft(), requestMetadata), + executor); // delete the snapshot index blob + storeDeletionFutures.add(storeDeletionFuture); + } + }); + + FutureUtil.allOf(storeDeletionFutures).join(); + } + + /** + * Restores all eligible stores in the task. + */ + @VisibleForTesting + static void restoreStores(String jobName, String jobId, TaskName taskName, List storesToRestore, + Map> prevStoreSnapshotIndexes, + File loggedBaseDir, StorageConfig storageConfig, BlobStoreRestoreManagerMetrics metrics, + StorageManagerUtil storageManagerUtil, BlobStoreUtil blobStoreUtil, DirDiffUtil dirDiffUtil, + ExecutorService executor) { + long restoreStartTime = System.nanoTime(); + List> restoreFutures = new ArrayList<>(); + + LOG.debug("Starting restore for task: {} stores: {}", taskName, storesToRestore); + storesToRestore.forEach(storeName -> { + if (!prevStoreSnapshotIndexes.containsKey(storeName)) { + LOG.info("No checkpointed snapshot index found for task: {} store: {}. Skipping restore.", taskName, storeName); + // TODO HIGH shesharm what should we do with the local state already present on disk, if any? + // E.g. this will be the case if user changes a store from changelog based backup and restore to + // blob store based backup and restore, both at the same time. + return; + } + + Pair scmAndSnapshotIndex = prevStoreSnapshotIndexes.get(storeName); + + long storeRestoreStartTime = System.nanoTime(); + SnapshotIndex snapshotIndex = scmAndSnapshotIndex.getRight(); + DirIndex dirIndex = snapshotIndex.getDirIndex(); + + DirIndex.Stats stats = DirIndex.getStats(dirIndex); + metrics.filesToRestore.getValue().addAndGet(stats.filesPresent); + metrics.bytesToRestore.getValue().addAndGet(stats.bytesPresent); + metrics.filesRemaining.getValue().addAndGet(stats.filesPresent); + metrics.bytesRemaining.getValue().addAndGet(stats.bytesPresent); + + CheckpointId checkpointId = snapshotIndex.getSnapshotMetadata().getCheckpointId(); + File storeDir = storageManagerUtil.getTaskStoreDir(loggedBaseDir, storeName, taskName, TaskMode.Active); + Path storeCheckpointDir = Paths.get(storageManagerUtil.getStoreCheckpointDir(storeDir, checkpointId)); + LOG.trace("Got task: {} store: {} local store directory: {} and local store checkpoint directory: {}", + taskName, storeName, storeDir, storeCheckpointDir); + + // we always delete the store dir to preserve transactional state guarantees. + try { + LOG.debug("Deleting local store directory: {}. Will be restored from local store checkpoint directory " + + "or remote snapshot.", storeDir); + FileUtils.deleteDirectory(storeDir); + } catch (IOException e) { + throw new SamzaException(String.format("Error deleting store directory: %s", storeDir), e); + } + + boolean shouldRestore = shouldRestore(taskName.getTaskName(), storeName, dirIndex, + storeCheckpointDir, storageConfig, dirDiffUtil); + + if (shouldRestore) { // restore the store from the remote blob store + // delete all store checkpoint directories. if we only delete the store directory and don't + // delete the checkpoint directories, the store size on disk will grow to 2x after restore + // until the first commit is completed and older checkpoint dirs are deleted. This is + // because the hard-linked checkpoint dir files will no longer be de-duped with the + // now-deleted main store directory contents and will take up additional space of their + // own during the restore. + deleteCheckpointDirs(taskName, storeName, loggedBaseDir, storageManagerUtil); + + metrics.storePreRestoreNs.get(storeName).set(System.nanoTime() - storeRestoreStartTime); + enqueueRestore(jobName, jobId, taskName.toString(), storeName, storeDir, dirIndex, storeRestoreStartTime, + restoreFutures, blobStoreUtil, dirDiffUtil, metrics, executor); + } else { + LOG.debug("Renaming store checkpoint directory: {} to store directory: {} since its contents are identical " + + "to the remote snapshot.", storeCheckpointDir, storeDir); + // atomically rename the checkpoint dir to the store dir + new FileUtil().move(storeCheckpointDir.toFile(), storeDir); + + // delete any other checkpoint dirs. + deleteCheckpointDirs(taskName, storeName, loggedBaseDir, storageManagerUtil); + } + }); + + // wait for all restores to finish + FutureUtil.allOf(restoreFutures).whenComplete((res, ex) -> { + LOG.info("Restore completed for task: {} stores", taskName); + metrics.restoreNs.set(System.nanoTime() - restoreStartTime); + }).join(); // TODO BLOCKER dchen1 make non-blocking. + } + + /** + * Determines if the store needs to be restored from remote snapshot based on local and remote state. + */ + @VisibleForTesting + static boolean shouldRestore(String taskName, String storeName, DirIndex dirIndex, + Path storeCheckpointDir, StorageConfig storageConfig, DirDiffUtil dirDiffUtil) { + // if a store checkpoint directory exists for the last successful task checkpoint, try to use it. + boolean restoreStore; + if (Files.exists(storeCheckpointDir)) { + if (storageConfig.cleanLoggedStoreDirsOnStart(storeName)) { + LOG.debug("Restoring task: {} store: {} from remote snapshot since the store is configured to be " + + "restored on each restart.", taskName, storeName); + restoreStore = true; + } else if (dirDiffUtil.areSameDir(FILES_TO_IGNORE, false).test(storeCheckpointDir.toFile(), dirIndex)) { + restoreStore = false; // no restore required for this store. + } else { + // we don't optimize for the case when the local host doesn't contain the most recent store checkpoint + // directory but contains an older checkpoint directory which could have partial overlap with the remote + // snapshot. we also don't try to optimize for any edge cases where the most recent checkpoint directory + // contents could be partially different than the remote store (afaik, there is no known valid scenario + // where this could happen right now, except for the offset file handling above). + // it's simpler and fast enough for now to restore the entire store instead. + + LOG.error("Local store checkpoint directory: {} contents are not the same as the remote snapshot. " + + "Queuing for restore from remote snapshot.", storeCheckpointDir); + restoreStore = true; + } + } else { // did not find last checkpoint dir, restore the store from the remote blob store + LOG.debug("No local store checkpoint directory found at: {}. " + + "Queuing for restore from remote snapshot.", storeCheckpointDir); + restoreStore = true; + } + + return restoreStore; + } + + /** + * Starts the restore for the store, enqueuing all restore-completion futures into {@param restoreFutures}. + */ + @VisibleForTesting + static void enqueueRestore(String jobName, String jobId, String taskName, String storeName, File storeDir, DirIndex dirIndex, + long storeRestoreStartTime, List> restoreFutures, BlobStoreUtil blobStoreUtil, + DirDiffUtil dirDiffUtil, BlobStoreRestoreManagerMetrics metrics, ExecutorService executor) { + + Metadata requestMetadata = new Metadata(storeDir.getAbsolutePath(), Optional.empty(), jobName, jobId, taskName, storeName); + CompletableFuture restoreFuture = + blobStoreUtil.restoreDir(storeDir, dirIndex, requestMetadata).thenRunAsync(() -> { + metrics.storeRestoreNs.get(storeName).set(System.nanoTime() - storeRestoreStartTime); + + long postRestoreStartTime = System.nanoTime(); + LOG.trace("Comparing restored store directory: {} and remote directory to verify restore.", storeDir); + if (!dirDiffUtil.areSameDir(FILES_TO_IGNORE, true).test(storeDir, dirIndex)) { + metrics.storePostRestoreNs.get(storeName).set(System.nanoTime() - postRestoreStartTime); + throw new SamzaException( + String.format("Restored store directory: %s contents " + "are not the same as the remote snapshot.", + storeDir.getAbsolutePath())); + } else { + metrics.storePostRestoreNs.get(storeName).set(System.nanoTime() - postRestoreStartTime); + LOG.info("Restore from remote snapshot completed for store: {}", storeDir); + } + }, executor); + + restoreFutures.add(restoreFuture); + } + + private static void deleteCheckpointDirs(TaskName taskName, String storeName, File loggedBaseDir, StorageManagerUtil storageManagerUtil) { + try { + List checkpointDirs = storageManagerUtil.getTaskStoreCheckpointDirs( + loggedBaseDir, storeName, taskName, TaskMode.Active); + for (File checkpointDir: checkpointDirs) { + LOG.debug("Deleting local store checkpoint directory: {} before restore.", checkpointDir); + FileUtils.deleteDirectory(checkpointDir); + } + } catch (Exception e) { + throw new SamzaException( + String.format("Error deleting checkpoint directory for task: %s store: %s.", + taskName, storeName), e); + } + } +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreStateBackendFactory.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreStateBackendFactory.java new file mode 100644 index 0000000000..23482c7b44 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreStateBackendFactory.java @@ -0,0 +1,98 @@ +/* + * 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.samza.storage.blobstore; + +import com.google.common.base.Preconditions; +import java.io.File; +import java.util.concurrent.ExecutorService; +import org.apache.commons.lang3.StringUtils; +import org.apache.samza.config.BlobStoreConfig; +import org.apache.samza.config.Config; +import org.apache.samza.context.ContainerContext; +import org.apache.samza.context.JobContext; +import org.apache.samza.job.model.ContainerModel; +import org.apache.samza.job.model.JobModel; +import org.apache.samza.job.model.TaskModel; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.storage.BlobStoreAdminFactory; +import org.apache.samza.storage.KafkaChangelogRestoreParams; +import org.apache.samza.storage.StateBackendAdmin; +import org.apache.samza.storage.StateBackendFactory; +import org.apache.samza.storage.StorageManagerUtil; +import org.apache.samza.storage.TaskBackupManager; +import org.apache.samza.storage.TaskRestoreManager; +import org.apache.samza.storage.blobstore.metrics.BlobStoreBackupManagerMetrics; +import org.apache.samza.storage.blobstore.metrics.BlobStoreRestoreManagerMetrics; +import org.apache.samza.util.Clock; +import org.apache.samza.util.ReflectionUtil; + + +public class BlobStoreStateBackendFactory implements StateBackendFactory { + @Override + public TaskBackupManager getBackupManager( + JobContext jobContext, + ContainerModel containerModel, + TaskModel taskModel, + ExecutorService backupExecutor, + MetricsRegistry metricsRegistry, + Config config, + Clock clock, + File loggedStoreBaseDir, + File nonLoggedStoreBaseDir) { + BlobStoreConfig blobStoreConfig = new BlobStoreConfig(config); + String blobStoreManagerFactory = blobStoreConfig.getBlobStoreManagerFactory(); + Preconditions.checkState(StringUtils.isNotBlank(blobStoreManagerFactory)); + BlobStoreManagerFactory factory = ReflectionUtil.getObj(blobStoreManagerFactory, BlobStoreManagerFactory.class); + BlobStoreManager blobStoreManager = factory.getBackupBlobStoreManager(config, backupExecutor); + BlobStoreBackupManagerMetrics metrics = new BlobStoreBackupManagerMetrics(metricsRegistry); + return new BlobStoreBackupManager(jobContext.getJobModel(), containerModel, taskModel, backupExecutor, + metrics, config, clock, loggedStoreBaseDir, new StorageManagerUtil(), blobStoreManager); + } + + @Override + public TaskRestoreManager getRestoreManager( + JobContext jobContext, + ContainerContext containerContext, + TaskModel taskModel, + ExecutorService restoreExecutor, + MetricsRegistry metricsRegistry, + Config config, + Clock clock, + File loggedStoreBaseDir, + File nonLoggedStoreBaseDir, + KafkaChangelogRestoreParams kafkaChangelogRestoreParams) { + BlobStoreConfig blobStoreConfig = new BlobStoreConfig(config); + String blobStoreManagerFactory = blobStoreConfig.getBlobStoreManagerFactory(); + Preconditions.checkState(StringUtils.isNotBlank(blobStoreManagerFactory)); + BlobStoreManagerFactory factory = ReflectionUtil.getObj(blobStoreManagerFactory, BlobStoreManagerFactory.class); + BlobStoreManager blobStoreManager = factory.getRestoreBlobStoreManager(config, restoreExecutor); + BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(metricsRegistry); + return new BlobStoreRestoreManager(taskModel, restoreExecutor, metrics, config, loggedStoreBaseDir, + nonLoggedStoreBaseDir, new StorageManagerUtil(), blobStoreManager); + } + + @Override + public StateBackendAdmin getAdmin(JobModel jobModel, Config config) { + BlobStoreConfig blobStoreConfig = new BlobStoreConfig(config); + String stateBackendAdminFactory = blobStoreConfig.getBlobStoreAdminFactory(); + BlobStoreAdminFactory factory = ReflectionUtil.getObj(stateBackendAdminFactory, BlobStoreAdminFactory.class); + return factory.getStateBackendAdmin(config, jobModel); + } +} \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/diff/DirDiff.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/diff/DirDiff.java new file mode 100644 index 0000000000..76fdb08986 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/diff/DirDiff.java @@ -0,0 +1,244 @@ +/* + * 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.samza.storage.blobstore.diff; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Sets; +import java.io.File; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.samza.storage.blobstore.index.DirIndex; +import org.apache.samza.storage.blobstore.index.FileIndex; + +/** + * Representation of the diff between a local directory and a remote directory contents. + */ +public class DirDiff { + + private final String dirName; + + /** + * New files in this directory that needs to be uploaded to the blob store. + */ + private final List filesAdded; + + /** + * Files that have already been uploaded to the blob store in a previous snapshot and haven't changed. + */ + private final List filesRetained; + + /** + * Files that have already been uploaded to the blob store in a previous snapshot and need to be removed. + */ + private final List filesRemoved; + + /** + * Subdirectories of this directory that are not already present in the previous snapshot and all of their contents + * need to be recursively added. + */ + private final List subDirsAdded; + + /** + * Subdirectories of this directory that are already present in the previous snapshot, but whose contents + * may have changed and may need to be recursively added or removed. + */ + private final List subDirsRetained; + + /** + * Subdirectories that are already present in the previous snapshot, but don't exist in the local snapshot, + * and hence all of their contents need to be recursively removed. + */ + private final List subDirsRemoved; + + public DirDiff(String dirName, + List filesAdded, List filesRetained, List filesRemoved, + List subDirsAdded, List subDirsRetained, List subDirsRemoved) { + Preconditions.checkNotNull(dirName); // may be empty for root dirs + Preconditions.checkNotNull(filesAdded); + Preconditions.checkNotNull(filesRetained); + Preconditions.checkNotNull(filesRemoved); + Preconditions.checkNotNull(subDirsAdded); + Preconditions.checkNotNull(subDirsRetained); + Preconditions.checkNotNull(subDirsRemoved); + + // validate that a file is not present in multiple lists + Set addedFilesSet = filesAdded.stream().map(File::getName).collect(Collectors.toSet()); + Set retainedFilesSet = filesRetained.stream().map(FileIndex::getFileName).collect(Collectors.toSet()); + Set removedFilesSet = filesRemoved.stream().map(FileIndex::getFileName).collect(Collectors.toSet()); + Sets.SetView addedAndRetainedFilesSet = Sets.intersection(addedFilesSet, retainedFilesSet); + Preconditions.checkState(addedAndRetainedFilesSet.isEmpty(), + String.format("Files present in both added and retained sets: %s", addedAndRetainedFilesSet.toString())); + Sets.SetView retainedAndRemovedFilesSet = Sets.intersection(retainedFilesSet, removedFilesSet); + Preconditions.checkState(retainedAndRemovedFilesSet.isEmpty(), + String.format("Files present in both retained and removed sets: %s", retainedAndRemovedFilesSet.toString())); + + // validate that a subDir is not present in multiple lists + Set addedSubDirsSet = subDirsAdded.stream().map(DirDiff::getDirName).collect(Collectors.toSet()); + Set retainedSubDirsSet = subDirsRetained.stream().map(DirDiff::getDirName).collect(Collectors.toSet()); + Set removedSubDirsSet = subDirsRemoved.stream().map(DirIndex::getDirName).collect(Collectors.toSet()); + Sets.SetView addedAndRetainedSubDirsSet = Sets.intersection(addedSubDirsSet, retainedSubDirsSet); + Preconditions.checkState(addedAndRetainedSubDirsSet.isEmpty(), + String.format("Sub-dirs present in both added and retained sets: %s", addedAndRetainedSubDirsSet.toString())); + Sets.SetView retainedAndRemovedSubDirsSet = Sets.intersection(retainedSubDirsSet, removedSubDirsSet); + Preconditions.checkState(retainedAndRemovedSubDirsSet.isEmpty(), + String.format("Sub-dirs present in both retained and removed sets: %s", retainedAndRemovedSubDirsSet.toString())); + + this.dirName = dirName; + this.filesAdded = filesAdded; + this.filesRetained = filesRetained; + this.filesRemoved = filesRemoved; + this.subDirsAdded = subDirsAdded; + this.subDirsRetained = subDirsRetained; + this.subDirsRemoved = subDirsRemoved; + } + + public String getDirName() { + return dirName; + } + + public List getFilesAdded() { + return filesAdded; + } + + public List getFilesRetained() { + return filesRetained; + } + + public List getFilesRemoved() { + return filesRemoved; + } + + public List getSubDirsAdded() { + return subDirsAdded; + } + + public List getSubDirsRetained() { + return subDirsRetained; + } + + public List getSubDirsRemoved() { + return subDirsRemoved; + } + + public static Stats getStats(DirDiff dirDiff) { + Stats stats = new Stats(); + updateStats(dirDiff, stats); + return stats; + } + + private static void updateStats(DirDiff dirDiff, Stats stats) { + stats.filesAdded += dirDiff.getFilesAdded().size(); + stats.filesRetained += dirDiff.getFilesRetained().size(); + stats.filesRemoved += dirDiff.getFilesRemoved().size(); + + stats.bytesAdded += dirDiff.getFilesAdded().stream().mapToLong(File::length).sum(); + stats.bytesRetained += dirDiff.getFilesRetained().stream().mapToLong(f -> f.getFileMetadata().getSize()).sum(); + stats.bytesRemoved += dirDiff.getFilesRemoved().stream().mapToLong(f -> f.getFileMetadata().getSize()).sum(); + + for (DirDiff subDirAdded: dirDiff.getSubDirsAdded()) { + stats.subDirsAdded += 1; + updateStats(subDirAdded, stats); + } + for (DirDiff subDirRetained: dirDiff.getSubDirsRetained()) { + stats.subDirsRetained += 1; + updateStats(subDirRetained, stats); + } + for (DirIndex subDirRemoved: dirDiff.getSubDirsRemoved()) { + stats.subDirsRemoved += 1; + updateStatsForDirRemoved(subDirRemoved, stats); + } + } + + private static void updateStatsForDirRemoved(DirIndex dirIndex, Stats stats) { + // every file and sub-dir present in a removed parent dir are to be removed as well + // files and sub-dirs to be removed don't matter since they would have already been + // cleaned up after the previous commit + stats.filesRemoved += dirIndex.getFilesRemoved().size(); + stats.bytesRemoved += dirIndex.getFilesPresent().stream().mapToLong(f -> f.getFileMetadata().getSize()).sum(); + for (DirIndex subDirRemoved: dirIndex.getSubDirsPresent()) { + stats.subDirsRemoved += 1; + updateStatsForDirRemoved(subDirRemoved, stats); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + + if (o == null || getClass() != o.getClass()) return false; + + DirDiff dirDiff = (DirDiff) o; + + return new EqualsBuilder() + .append(getDirName(), dirDiff.getDirName()) + .append(getFilesAdded(), dirDiff.getFilesAdded()) + .append(getFilesRetained(), dirDiff.getFilesRetained()) + .append(getFilesRemoved(), dirDiff.getFilesRemoved()) + .append(getSubDirsAdded(), dirDiff.getSubDirsAdded()) + .append(getSubDirsRetained(), dirDiff.getSubDirsRetained()) + .append(getSubDirsRemoved(), dirDiff.getSubDirsRemoved()) + .isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(17, 37) + .append(getDirName()) + .append(getFilesAdded()) + .append(getFilesRetained()) + .append(getFilesRemoved()) + .append(getSubDirsAdded()) + .append(getSubDirsRetained()) + .append(getSubDirsRemoved()) + .toHashCode(); + } + + public static class Stats { + public int filesAdded; + public int filesRetained; + public int filesRemoved; + + public int subDirsAdded; + public int subDirsRetained; + public int subDirsRemoved; + + public long bytesAdded; + public long bytesRetained; + public long bytesRemoved; + + @Override + public String toString() { + return "Stats{" + + "filesAdded=" + filesAdded + + ", filesRetained=" + filesRetained + + ", filesRemoved=" + filesRemoved + + ", subDirsAdded=" + subDirsAdded + + ", subDirsRetained=" + subDirsRetained + + ", subDirsRemoved=" + subDirsRemoved + + ", bytesAdded=" + bytesAdded + + ", bytesRetained=" + bytesRetained + + ", bytesRemoved=" + bytesRemoved + + '}'; + } + } +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/DirIndex.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/DirIndex.java new file mode 100644 index 0000000000..31819be54e --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/DirIndex.java @@ -0,0 +1,176 @@ +/* + * 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.samza.storage.blobstore.index; + +import com.google.common.base.Preconditions; +import java.util.List; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; + +/** + * Representation of a directory in the blob store + */ +public class DirIndex { + public static final String ROOT_DIR_NAME = ""; + private static final short SCHEMA_VERSION = 1; + + private final String dirName; + + private final List filesPresent; + private final List filesRemoved; + + // Note: subDirsPresent can also have filesRemoved and subDirsRemoved within them. + private final List subDirsPresent; + private final List subDirsRemoved; + + public DirIndex(String dirName, + List filesPresent, List filesRemoved, + List subDirsPresent, List subDirsRemoved) { + Preconditions.checkNotNull(dirName); // may be empty for root dirs + Preconditions.checkNotNull(filesPresent); + Preconditions.checkNotNull(filesRemoved); + Preconditions.checkNotNull(subDirsPresent); + Preconditions.checkNotNull(subDirsRemoved); + this.dirName = dirName; + this.filesPresent = filesPresent; + this.filesRemoved = filesRemoved; + this.subDirsPresent = subDirsPresent; + this.subDirsRemoved = subDirsRemoved; + } + + public static short getSchemaVersion() { + return SCHEMA_VERSION; + } + + public String getDirName() { + return dirName; + } + + public List getFilesPresent() { + return filesPresent; + } + + public List getFilesRemoved() { + return filesRemoved; + } + + public List getSubDirsPresent() { + return subDirsPresent; + } + + public List getSubDirsRemoved() { + return subDirsRemoved; + } + + public static Stats getStats(DirIndex dirIndex) { + Stats stats = new Stats(); + updateStats(dirIndex, stats); + return stats; + } + + private static void updateStats(DirIndex dirIndex, Stats stats) { + stats.filesPresent += dirIndex.getFilesPresent().size(); + stats.filesRemoved += dirIndex.getFilesRemoved().size(); + + stats.subDirsPresent += dirIndex.getSubDirsPresent().size(); + stats.subDirsRemoved += dirIndex.getSubDirsRemoved().size(); + + stats.bytesPresent += dirIndex.getFilesPresent().stream().mapToLong(fi -> fi.getFileMetadata().getSize()).sum(); + stats.bytesRemoved += dirIndex.getFilesRemoved().stream().mapToLong(fi -> fi.getFileMetadata().getSize()).sum(); + + for (DirIndex subDirPresent : dirIndex.getSubDirsPresent()) { + updateStats(subDirPresent, stats); + } + + for (DirIndex subDirsRemoved : dirIndex.getSubDirsRemoved()) { + updateStatsForSubDirsRemoved(subDirsRemoved, stats); + } + } + + private static void updateStatsForSubDirsRemoved(DirIndex dirIndex, Stats stats) { + stats.filesRemoved += dirIndex.getFilesPresent().size(); + stats.bytesRemoved += dirIndex.getFilesPresent().stream().mapToLong(fi -> fi.getFileMetadata().getSize()).sum(); + for (DirIndex subDirToRemove : dirIndex.getSubDirsPresent()) { + updateStatsForSubDirsRemoved(subDirToRemove, stats); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + + if (o == null || getClass() != o.getClass()) return false; + + DirIndex that = (DirIndex) o; + + return new EqualsBuilder() + .append(getDirName(), that.getDirName()) + .append(getFilesPresent(), that.getFilesPresent()) + .append(getFilesRemoved(), that.getFilesRemoved()) + .append(getSubDirsPresent(), that.getSubDirsPresent()) + .append(getSubDirsRemoved(), that.getSubDirsRemoved()) + .isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(17, 37) + .append(getDirName()).append(getFilesPresent()) + .append(getFilesRemoved()) + .append(getSubDirsPresent()) + .append(getSubDirsRemoved()) + .toHashCode(); + } + + @Override + public String toString() { + return "DirIndex{" + + "dirName='" + + dirName + '\'' + + ", filesPresent=" + filesPresent + + ", filesRemoved=" + filesRemoved + + ", subDirsPresent=" + subDirsPresent + + ", subDirsRemoved=" + subDirsRemoved + + '}'; + } + + public static class Stats { + public int filesPresent; + public int filesRemoved; + + public int subDirsPresent; + public int subDirsRemoved; + + public long bytesPresent; + public long bytesRemoved; + + @Override + public String toString() { + return "Stats{" + + "filesAdded=" + filesPresent + + ", filesRemoved=" + filesRemoved + + ", subDirsAdded=" + subDirsPresent + + ", subDirsRemoved=" + subDirsRemoved + + ", bytesAdded=" + bytesPresent + + ", bytesRemoved=" + bytesRemoved + + '}'; + } + } +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileBlob.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileBlob.java new file mode 100644 index 0000000000..8a289aaf59 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileBlob.java @@ -0,0 +1,85 @@ +/* + * 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.samza.storage.blobstore.index; + +import com.google.common.base.Preconditions; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; + + +/** + * Representation of a File in a Blob store + */ +public class FileBlob { + + private final String blobId; + /** + * Offset of this blob in the file. A file can be uploaded multiple chunks, and can have + * multiple blobs associated with it. Each blob then has its own ID and an offset in the file. + */ + private final int offset; + + public FileBlob(String blobId, int offset) { + Preconditions.checkState(StringUtils.isNotBlank(blobId)); + Preconditions.checkState(offset >= 0); + this.blobId = blobId; + this.offset = offset; + } + + public String getBlobId() { + return blobId; + } + + public int getOffset() { + return offset; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof FileBlob)) { + return false; + } + + FileBlob fileBlob = (FileBlob) o; + + return new EqualsBuilder() + .append(blobId, fileBlob.blobId) + .append(offset, fileBlob.offset) + .isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(17, 37) + .append(blobId) + .append(offset) + .toHashCode(); + } + + @Override + public String toString() { + return "FileBlob{" + "blobId='" + blobId + '\'' + ", offset=" + offset + '}'; + } +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileIndex.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileIndex.java new file mode 100644 index 0000000000..b0666cb1ee --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileIndex.java @@ -0,0 +1,114 @@ +/* + * 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.samza.storage.blobstore.index; + +import com.google.common.base.Preconditions; +import java.util.List; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; + + +/** + * Representation of a file in blob store + */ +public class FileIndex { + private final String fileName; + /** + * Chunks of file uploaded to blob store as {@link FileBlob}s + */ + private final List fileBlobs; + /** + * Metadata (e.g. POSIX file attributes) associated with the file. + */ + private final FileMetadata fileMetadata; + /** + * Checksum of the file for verifying integrity. + */ + private final long checksum; + + + public FileIndex(String fileName, List fileBlobs, FileMetadata fileMetadata, long checksum) { + Preconditions.checkState(StringUtils.isNotBlank(fileName)); + Preconditions.checkNotNull(fileBlobs); + // fileBlobs can be empty list for a file of size 0 bytes. + Preconditions.checkNotNull(fileMetadata); + this.fileName = fileName; + this.fileBlobs = fileBlobs; + this.fileMetadata = fileMetadata; + this.checksum = checksum; + } + + public String getFileName() { + return fileName; + } + + public List getBlobs() { + return fileBlobs; + } + + public FileMetadata getFileMetadata() { + return fileMetadata; + } + + public long getChecksum() { + return checksum; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof FileIndex)) { + return false; + } + + FileIndex that = (FileIndex) o; + + return new EqualsBuilder() + .append(getFileName(), that.getFileName()) + .append(getBlobs(), that.getBlobs()) + .append(fileMetadata, that.fileMetadata) + .append(getChecksum(), that.getChecksum()) + .isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(17, 37) + .append(getFileName()) + .append(getBlobs()) + .append(fileMetadata) + .append(getChecksum()) + .toHashCode(); + } + + @Override + public String toString() { + return "FileIndex{" + + "fileName='" + fileName + '\'' + + ", fileBlobs=" + fileBlobs + + ", fileMetadata=" + fileMetadata + + ", checksum='" + checksum + '\'' + + '}'; + } +} \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileMetadata.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileMetadata.java new file mode 100644 index 0000000000..41db1264d6 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileMetadata.java @@ -0,0 +1,138 @@ +/* + * 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.samza.storage.blobstore.index; + +import com.google.common.base.Preconditions; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.attribute.PosixFileAttributes; +import java.nio.file.attribute.PosixFilePermissions; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; + + +/** + * Representation of metadata associated with a File. + */ +public class FileMetadata { + + private final long creationTimeMillis; + private final long lastModifiedTimeMillis; + private final long size; + private final String owner; + private final String group; + private final String permissions; + + public FileMetadata(long creationTimeMillis, long lastModifiedTimeMillis, long size, + String owner, String group, String permissions) { + Preconditions.checkState(creationTimeMillis >= 0); + Preconditions.checkState(lastModifiedTimeMillis >= 0); + Preconditions.checkState(size >= 0); + Preconditions.checkState(StringUtils.isNotBlank(owner)); + Preconditions.checkState(StringUtils.isNotBlank(group)); + Preconditions.checkState(StringUtils.isNotBlank(permissions)); + this.creationTimeMillis = creationTimeMillis; + this.lastModifiedTimeMillis = lastModifiedTimeMillis; + this.size = size; + this.owner = owner; + this.group = group; + this.permissions = permissions; + } + + public static FileMetadata fromFile(File file) throws IOException { + PosixFileAttributes attributes = Files.readAttributes(file.toPath(), PosixFileAttributes.class); + + return new FileMetadata(attributes.creationTime().toMillis(), attributes.lastModifiedTime().toMillis(), + attributes.size(), attributes.owner().toString(), attributes.group().toString(), + PosixFilePermissions.toString(attributes.permissions())); + } + + public long getCreationTimeMillis() { + return creationTimeMillis; + } + + public long getLastModifiedTimeMillis() { + return lastModifiedTimeMillis; + } + + public long getSize() { + return size; + } + + public String getOwner() { + return owner; + } + + public String getGroup() { + return group; + } + + public String getPermissions() { + return permissions; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof FileMetadata)) { + return false; + } + + FileMetadata that = (FileMetadata) o; + + return new EqualsBuilder() + .append(getCreationTimeMillis(), that.getCreationTimeMillis()) + .append(getLastModifiedTimeMillis(), that.getLastModifiedTimeMillis()) + .append(getSize(), that.getSize()) + .append(getOwner(), that.getOwner()) + .append(getGroup(), that.getGroup()) + .append(getPermissions(), that.getPermissions()) + .isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(17, 37) + .append(getCreationTimeMillis()) + .append(getLastModifiedTimeMillis()) + .append(getSize()) + .append(getOwner()) + .append(getGroup()) + .append(getPermissions()) + .toHashCode(); + } + + @Override + public String toString() { + return "FileMetadata{" + + "creationTimeMillis=" + creationTimeMillis + + ", lastModifiedTimeMillis=" + lastModifiedTimeMillis + + ", size=" + size + + ", owner='" + owner + '\'' + + ", group='" + group + '\'' + + ", permissions=" + permissions + + '}'; + } +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotIndex.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotIndex.java new file mode 100644 index 0000000000..b3f68dc31d --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotIndex.java @@ -0,0 +1,124 @@ +/* + * 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.samza.storage.blobstore.index; + +import com.google.common.base.Preconditions; +import java.util.Optional; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; + + +/** + * A {@link SnapshotIndex} contains all the information necessary for recreating the local store by + * downloading its contents from the remote blob store. The {@link SnapshotIndex} is itself serialized + * and stored as a blob in the remote store, and its blob id tracked in the Task checkpoint. + */ +public class SnapshotIndex { + private static final short SCHEMA_VERSION = 1; + + private final long creationTimeMillis; + /** + * Metadata for a snapshot like job name, job Id, store name etc. + */ + private final SnapshotMetadata snapshotMetadata; + private final DirIndex dirIndex; + + /** + * Blob ID of previous snapshot index blob. Tracked here to be cleaned up + * in cleanup phase of commit lifecycle. + */ + private final Optional prevSnapshotIndexBlobId; + + public SnapshotIndex(long creationTimeMillis, SnapshotMetadata snapshotMetadata, DirIndex dirIndex, + Optional prevSnapshotIndexBlobId) { + Preconditions.checkState(creationTimeMillis >= 0); + Preconditions.checkNotNull(snapshotMetadata); + Preconditions.checkNotNull(dirIndex); + Preconditions.checkNotNull(prevSnapshotIndexBlobId); + Preconditions.checkState( + !(prevSnapshotIndexBlobId.isPresent() && StringUtils.isBlank(prevSnapshotIndexBlobId.get()))); + this.creationTimeMillis = creationTimeMillis; + this.snapshotMetadata = snapshotMetadata; + this.dirIndex = dirIndex; + // if no previous snapshot index blob was present, this can be null + this.prevSnapshotIndexBlobId = prevSnapshotIndexBlobId; + } + + public static short getSchemaVersion() { + return SCHEMA_VERSION; + } + + public long getCreationTimeMillis() { + return creationTimeMillis; + } + + public SnapshotMetadata getSnapshotMetadata() { + return snapshotMetadata; + } + + public DirIndex getDirIndex() { + return dirIndex; + } + + public Optional getPrevSnapshotIndexBlobId() { + return prevSnapshotIndexBlobId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof SnapshotIndex)) { + return false; + } + + SnapshotIndex that = (SnapshotIndex) o; + + return new EqualsBuilder() + .append(getCreationTimeMillis(), that.getCreationTimeMillis()) + .append(getSnapshotMetadata(), that.getSnapshotMetadata()) + .append(getDirIndex(), that.getDirIndex()) + .append(getPrevSnapshotIndexBlobId(), that.getPrevSnapshotIndexBlobId()) + .isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(17, 37) + .append(getCreationTimeMillis()) + .append(getSnapshotMetadata()) + .append(getDirIndex()) + .append(prevSnapshotIndexBlobId) + .toHashCode(); + } + + @Override + public String toString() { + return "SnapshotIndex{" + + "creationTimeMillis=" + creationTimeMillis + + ", snapshotMetadata=" + snapshotMetadata + + ", dirIndex=" + dirIndex + + ", prevSnapshotIndexBlobId" + prevSnapshotIndexBlobId + + '}'; + } +} \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotMetadata.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotMetadata.java new file mode 100644 index 0000000000..7f059b9645 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotMetadata.java @@ -0,0 +1,115 @@ +/* + * 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.samza.storage.blobstore.index; + +import com.google.common.base.Preconditions; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.samza.checkpoint.CheckpointId; + + +/** + * Represents metadata associated with a remote snapshot. + */ +public class SnapshotMetadata { + private final CheckpointId checkpointId; + private final String jobName; + private final String jobId; + private final String taskName; + private final String storeName; + + public SnapshotMetadata(CheckpointId checkpointId, String jobName, String jobId, String taskName, + String storeName) { + Preconditions.checkNotNull(checkpointId); + Preconditions.checkState(StringUtils.isNotBlank(jobName)); + Preconditions.checkState(StringUtils.isNotBlank(jobId)); + Preconditions.checkState(StringUtils.isNotBlank(taskName)); + Preconditions.checkState(StringUtils.isNotBlank(storeName)); + this.checkpointId = checkpointId; + this.jobName = jobName; + this.jobId = jobId; + this.taskName = taskName; + this.storeName = storeName; + } + + public CheckpointId getCheckpointId() { + return checkpointId; + } + + public String getJobName() { + return jobName; + } + + public String getJobId() { + return jobId; + } + + public String getTaskName() { + return taskName; + } + + public String getStoreName() { + return storeName; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof SnapshotMetadata)) { + return false; + } + + SnapshotMetadata that = (SnapshotMetadata) o; + + return new EqualsBuilder() + .append(getCheckpointId(), that.getCheckpointId()) + .append(getJobName(), that.getJobName()) + .append(getJobId(), that.getJobId()) + .append(getTaskName(), that.getTaskName()) + .append(getStoreName(), that.getStoreName()) + .isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(17, 37) + .append(getCheckpointId()) + .append(getJobName()) + .append(getJobId()) + .append(getTaskName()) + .append(getStoreName()) + .toHashCode(); + } + + @Override + public String toString() { + return "SnapshotMetadata{" + + "checkpointId=" + checkpointId + + ", jobName='" + jobName + '\'' + + ", jobId='" + jobId + '\'' + + ", taskName='" + taskName + '\'' + + ", storeName='" + storeName + '\'' + + '}'; + } +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonDirIndexMixin.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonDirIndexMixin.java new file mode 100644 index 0000000000..7a29c83f98 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonDirIndexMixin.java @@ -0,0 +1,58 @@ +/* + * 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.samza.storage.blobstore.index.serde; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; +import org.apache.samza.storage.blobstore.index.DirIndex; +import org.apache.samza.storage.blobstore.index.FileIndex; + + +/** + * A mix-in Jackson class to convert {@link DirIndex} to/from JSON. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public abstract class JsonDirIndexMixin { + + @JsonCreator + public JsonDirIndexMixin(@JsonProperty("dir-name") String dirName, + @JsonProperty("files-present") List filesPresent, + @JsonProperty("files-removed") List filesRemoved, + @JsonProperty("sub-dirs-present") List subDirsPresent, + @JsonProperty("sub-dirs-removed") List subDirsRemoved) { + } + + @JsonProperty("dir-name") + abstract String getDirName(); + + @JsonProperty("files-present") + abstract List getFilesPresent(); + + @JsonProperty("files-removed") + abstract List getFilesRemoved(); + + @JsonProperty("sub-dirs-present") + abstract List getSubDirsPresent(); + + @JsonProperty("sub-dirs-removed") + abstract List getSubDirsRemoved(); +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileBlobMixin.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileBlobMixin.java new file mode 100644 index 0000000000..03a91d1c66 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileBlobMixin.java @@ -0,0 +1,41 @@ +/* + * 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.samza.storage.blobstore.index.serde; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + + +/** + * A mix-in Jackson class to convert FileBlob to/from JSON. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public abstract class JsonFileBlobMixin { + @JsonCreator + public JsonFileBlobMixin(@JsonProperty("blob-id") String blobId, @JsonProperty("offset") int offset) { + } + + @JsonProperty("blob-id") + abstract String getBlobId(); + + @JsonProperty("offset") + abstract int getOffset(); +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileIndexMixin.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileIndexMixin.java new file mode 100644 index 0000000000..d8ee10a901 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileIndexMixin.java @@ -0,0 +1,53 @@ +/* + * 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.samza.storage.blobstore.index.serde; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; +import org.apache.samza.storage.blobstore.index.FileBlob; +import org.apache.samza.storage.blobstore.index.FileMetadata; + + +/** + * A mix-in Jackson class to convert FileIndex to/from JSON. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public abstract class JsonFileIndexMixin { + @JsonCreator + public JsonFileIndexMixin(@JsonProperty("file-name") String fileName, + @JsonProperty("blobs") List blobs, @JsonProperty("file-metadata") FileMetadata fileMetadata, + @JsonProperty("checksum") long checksum) { + + } + + @JsonProperty("file-name") + abstract String getFileName(); + + @JsonProperty("blobs") + abstract List getBlobs(); + + @JsonProperty("file-metadata") + abstract FileMetadata getFileMetadata(); + + @JsonProperty("checksum") + abstract long getChecksum(); +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileMetadataMixin.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileMetadataMixin.java new file mode 100644 index 0000000000..71eea611d1 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileMetadataMixin.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.samza.storage.blobstore.index.serde; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * A mix-in Jackson class to convert FileMetadata to/from JSON. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public abstract class JsonFileMetadataMixin { + @JsonCreator + public JsonFileMetadataMixin(@JsonProperty("creation-time-millis") long creationTimeMillis, + @JsonProperty("last-modified-time-millis") long lastModifiedTimeMillis, @JsonProperty("size") long size, + @JsonProperty("owner") String owner, @JsonProperty("group") String group, + @JsonProperty("permissions") String permissions) { + } + + @JsonProperty("creation-time-millis") + abstract long getCreationTimeMillis(); + + @JsonProperty("last-modified-time-millis") + abstract long getLastModifiedTimeMillis(); + + @JsonProperty("size") + abstract long getSize(); + + @JsonProperty("owner") + abstract String getOwner(); + + @JsonProperty("group") + abstract String getGroup(); + + @JsonProperty("permissions") + abstract String getPermissions(); +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotIndexMixin.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotIndexMixin.java new file mode 100644 index 0000000000..3cce2538fd --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotIndexMixin.java @@ -0,0 +1,53 @@ +/* + * 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.samza.storage.blobstore.index.serde; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Optional; +import org.apache.samza.storage.blobstore.index.DirIndex; +import org.apache.samza.storage.blobstore.index.SnapshotMetadata; + + +/** + * A mix-in Jackson class to convert SnapshotIndex to/from JSON. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public abstract class JsonSnapshotIndexMixin { + @JsonCreator + public JsonSnapshotIndexMixin(@JsonProperty("creation-time-millis") long creationTimeMillis, + @JsonProperty("snapshot-metadata") SnapshotMetadata snapshotMetadata, + @JsonProperty("dir-index") DirIndex dirIndex, + @JsonProperty("prev-snapshot-index-blob-id") Optional prevSnapshotIndexBlobId) { + } + + @JsonProperty("creation-time-millis") + abstract long getCreationTimeMillis(); + + @JsonProperty("snapshot-metadata") + abstract SnapshotMetadata getSnapshotMetadata(); + + @JsonProperty("dir-index") + abstract DirIndex getDirIndex(); + + @JsonProperty("prev-snapshot-index-blob-id") + abstract Optional getPrevSnapshotIndexBlobId(); +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotMetadataMixin.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotMetadataMixin.java new file mode 100644 index 0000000000..804e3304a3 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotMetadataMixin.java @@ -0,0 +1,53 @@ +/* + * 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.samza.storage.blobstore.index.serde; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.samza.checkpoint.CheckpointId; + +/** + * A mix-in Jackson class to convert SnapshotMetadata to/from JSON. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public abstract class JsonSnapshotMetadataMixin { + @JsonCreator + public JsonSnapshotMetadataMixin(@JsonProperty("checkpoint-id") CheckpointId checkpointId, + @JsonProperty("job-name") String jobName, @JsonProperty("job-id") String jobId, + @JsonProperty("task-name") String taskName, @JsonProperty("store-name") String storeName) { + + } + + @JsonProperty("checkpoint-id") + abstract CheckpointId getCheckpointId(); + + @JsonProperty("job-name") + abstract String getJobName(); + + @JsonProperty("job-id") + abstract String getJobId(); + + @JsonProperty("task-name") + abstract String getTaskName(); + + @JsonProperty("store-name") + abstract String getStoreName(); +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/SnapshotIndexSerde.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/SnapshotIndexSerde.java new file mode 100644 index 0000000000..2e0e50c678 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/SnapshotIndexSerde.java @@ -0,0 +1,76 @@ +/* + * 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.samza.storage.blobstore.index.serde; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; +import com.fasterxml.jackson.datatype.jdk8.Jdk8Module; +import org.apache.samza.SamzaException; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.serializers.JsonCheckpointIdMixin; +import org.apache.samza.serializers.Serde; +import org.apache.samza.storage.blobstore.index.DirIndex; +import org.apache.samza.storage.blobstore.index.FileBlob; +import org.apache.samza.storage.blobstore.index.FileIndex; +import org.apache.samza.storage.blobstore.index.FileMetadata; +import org.apache.samza.storage.blobstore.index.SnapshotIndex; +import org.apache.samza.storage.blobstore.index.SnapshotMetadata; + + +public class SnapshotIndexSerde implements Serde { + + private final static ObjectMapper MAPPER = new ObjectMapper(); + private TypeReference typeReference; + private final ObjectWriter objectWriter; + + public SnapshotIndexSerde() { + MAPPER.registerModule(new Jdk8Module()); + MAPPER.addMixIn(SnapshotIndex.class, JsonSnapshotIndexMixin.class) + .addMixIn(SnapshotMetadata.class, JsonSnapshotMetadataMixin.class) + .addMixIn(DirIndex.class, JsonDirIndexMixin.class) + .addMixIn(FileIndex.class, JsonFileIndexMixin.class) + .addMixIn(FileMetadata.class, JsonFileMetadataMixin.class) + .addMixIn(FileBlob.class, JsonFileBlobMixin.class) + .addMixIn(CheckpointId.class, JsonCheckpointIdMixin.class); + + this.typeReference = new TypeReference() { }; + this.objectWriter = MAPPER.writerFor(typeReference); + } + + @Override + public SnapshotIndex fromBytes(byte[] bytes) { + try { + return MAPPER.readerFor(typeReference).readValue(bytes); + } catch (Exception exception) { + throw new SamzaException(String.format("Exception in deserializing SnapshotIndex bytes %s", + new String(bytes)), exception); + } + } + + @Override + public byte[] toBytes(SnapshotIndex snapshotIndex) { + try { + return objectWriter.writeValueAsBytes(snapshotIndex); + } catch (Exception exception) { + throw new SamzaException(String.format("Exception in serializing SnapshotIndex bytes %s", snapshotIndex), exception); + } + } +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreBackupManagerMetrics.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreBackupManagerMetrics.java new file mode 100644 index 0000000000..5e41073f27 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreBackupManagerMetrics.java @@ -0,0 +1,136 @@ +/* + * 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.samza.storage.blobstore.metrics; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.Gauge; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.metrics.Timer; + + +public class BlobStoreBackupManagerMetrics { + private static final String GROUP = BlobStoreBackupManagerMetrics.class.getName(); + private final MetricsRegistry metricsRegistry; + + // TODO LOW shesharm per-task throughput + public final Gauge initNs; + + public final Timer uploadNs; + // gauges of AtomicLong so that the value can be incremented/decremented atomically in a thread-safe way. + // don't set the gauge value directly. use gauge.getValue().incrementAndGet() etc instead. + public final Gauge filesToUpload; + public final Gauge bytesToUpload; + public final Gauge filesUploaded; + public final Gauge bytesUploaded; + public final Gauge filesRemaining; + public final Gauge bytesRemaining; + public final Gauge filesToRetain; + public final Gauge bytesToRetain; + public final Counter uploadRate; + + // per store breakdowns + public final Map storeDirDiffNs; + public final Map storeUploadNs; + + public final Map> storeFilesToUpload; + public final Map> storeFilesToRetain; + public final Map> storeFilesToRemove; + public final Map> storeSubDirsToUpload; + public final Map> storeSubDirsToRetain; + public final Map> storeSubDirsToRemove; + public final Map> storeBytesToUpload; + public final Map> storeBytesToRetain; + public final Map> storeBytesToRemove; + + public final Timer cleanupNs; + + // TODO shesharm LOW move to SamzaHistogram + public final Timer avgFileUploadNs; // avg time for each file uploaded + public final Timer avgFileSizeBytes; // avg size of each file uploaded + + public BlobStoreBackupManagerMetrics(MetricsRegistry metricsRegistry) { + this.metricsRegistry = metricsRegistry; + + this.initNs = metricsRegistry.newGauge(GROUP, "init-ns", 0L); + + this.uploadNs = metricsRegistry.newTimer(GROUP, "upload-ns"); + + this.filesToUpload = metricsRegistry.newGauge(GROUP, "files-to-upload", new AtomicLong(0L)); + this.bytesToUpload = metricsRegistry.newGauge(GROUP, "bytes-to-upload", new AtomicLong(0L)); + this.filesUploaded = metricsRegistry.newGauge(GROUP, "files-uploaded", new AtomicLong(0L)); + this.bytesUploaded = metricsRegistry.newGauge(GROUP, "bytes-uploaded", new AtomicLong(0L)); + this.filesRemaining = metricsRegistry.newGauge(GROUP, "files-remaining", new AtomicLong(0L)); + this.bytesRemaining = metricsRegistry.newGauge(GROUP, "bytes-remaining", new AtomicLong(0L)); + this.filesToRetain = metricsRegistry.newGauge(GROUP, "files-to-retain", new AtomicLong(0L)); + this.bytesToRetain = metricsRegistry.newGauge(GROUP, "bytes-to-retain", new AtomicLong(0L)); + + this.storeDirDiffNs = new ConcurrentHashMap<>(); + this.storeUploadNs = new ConcurrentHashMap<>(); + + this.storeFilesToUpload = new ConcurrentHashMap<>(); + this.storeFilesToRetain = new ConcurrentHashMap<>(); + this.storeFilesToRemove = new ConcurrentHashMap<>(); + this.storeSubDirsToUpload = new ConcurrentHashMap<>(); + this.storeSubDirsToRetain = new ConcurrentHashMap<>(); + this.storeSubDirsToRemove = new ConcurrentHashMap<>(); + this.storeBytesToUpload = new ConcurrentHashMap<>(); + this.storeBytesToRetain = new ConcurrentHashMap<>(); + this.storeBytesToRemove = new ConcurrentHashMap<>(); + + this.uploadRate = metricsRegistry.newCounter(GROUP, "upload-rate"); + + this.cleanupNs = metricsRegistry.newTimer(GROUP, "cleanup-ns"); + + this.avgFileUploadNs = metricsRegistry.newTimer(GROUP, "avg-file-upload-ns"); + this.avgFileSizeBytes = metricsRegistry.newTimer(GROUP, "avg-file-size-bytes"); + } + + public void initStoreMetrics(Collection storeNames) { + for (String storeName: storeNames) { + storeDirDiffNs.putIfAbsent(storeName, + metricsRegistry.newTimer(GROUP, String.format("%s-dir-diff-ns", storeName))); + storeUploadNs.putIfAbsent(storeName, + metricsRegistry.newTimer(GROUP, String.format("%s-upload-ns", storeName))); + + storeFilesToUpload.putIfAbsent(storeName, + metricsRegistry.newGauge(GROUP, String.format("%s-files-to-upload", storeName), 0L)); + storeFilesToRetain.putIfAbsent(storeName, + metricsRegistry.newGauge(GROUP, String.format("%s-files-to-retain", storeName), 0L)); + storeFilesToRemove.putIfAbsent(storeName, + metricsRegistry.newGauge(GROUP, String.format("%s-files-to-remove", storeName), 0L)); + storeSubDirsToUpload.putIfAbsent(storeName, + metricsRegistry.newGauge(GROUP, String.format("%s-sub-dirs-to-upload", storeName), 0L)); + storeSubDirsToRetain.putIfAbsent(storeName, + metricsRegistry.newGauge(GROUP, String.format("%s-sub-dirs-to-retain", storeName), 0L)); + storeSubDirsToRemove.putIfAbsent(storeName, + metricsRegistry.newGauge(GROUP, String.format("%s-sub-dirs-to-remove", storeName), 0L)); + storeBytesToUpload.putIfAbsent(storeName, + metricsRegistry.newGauge(GROUP, String.format("%s-bytes-to-upload", storeName), 0L)); + storeBytesToRetain.putIfAbsent(storeName, + metricsRegistry.newGauge(GROUP, String.format("%s-bytes-to-retain", storeName), 0L)); + storeBytesToRemove.putIfAbsent(storeName, + metricsRegistry.newGauge(GROUP, String.format("%s-bytes-to-remove", storeName), 0L)); + } + } +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreRestoreManagerMetrics.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreRestoreManagerMetrics.java new file mode 100644 index 0000000000..ac07f608a9 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreRestoreManagerMetrics.java @@ -0,0 +1,94 @@ +/* + * 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.samza.storage.blobstore.metrics; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.Gauge; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.metrics.Timer; + + +public class BlobStoreRestoreManagerMetrics { + private static final String GROUP = BlobStoreRestoreManagerMetrics.class.getName(); + private final MetricsRegistry metricsRegistry; + + // TODO LOW shesharma per-task throughput + public final Gauge initNs; + public final Gauge getSnapshotIndexNs; + + public final Gauge restoreNs; + // gauges of AtomicLong so that the value can be incremented/decremented atomically in a thread-safe way. + // don't set the gauge value directly. use gauge.getValue().incrementAndGet() etc instead. + public final Gauge filesToRestore; + public final Gauge bytesToRestore; + public final Gauge filesRestored; + public final Gauge bytesRestored; + public final Gauge filesRemaining; + public final Gauge bytesRemaining; + + public final Counter restoreRate; + + // per store breakdowns + public final Map> storePreRestoreNs; + public final Map> storeRestoreNs; + public final Map> storePostRestoreNs; + + // TODO LOW shesharma move to SamzaHistogram + public final Timer avgFileRestoreNs; // avg time for each file restored + + public BlobStoreRestoreManagerMetrics(MetricsRegistry metricsRegistry) { + this.metricsRegistry = metricsRegistry; + + this.initNs = metricsRegistry.newGauge(GROUP, "init-ns", 0L); + this.getSnapshotIndexNs = metricsRegistry.newGauge(GROUP, "get-snapshot-index-ns", 0L); + + this.restoreNs = metricsRegistry.newGauge(GROUP, "restore-ns", 0L); + this.filesToRestore = metricsRegistry.newGauge(GROUP, "files-to-restore", new AtomicLong()); + this.bytesToRestore = metricsRegistry.newGauge(GROUP, "bytes-to-restore", new AtomicLong()); + this.filesRestored = metricsRegistry.newGauge(GROUP, "files-restored", new AtomicLong()); + this.bytesRestored = metricsRegistry.newGauge(GROUP, "bytes-restored", new AtomicLong()); + this.filesRemaining = metricsRegistry.newGauge(GROUP, "files-remaining", new AtomicLong()); + this.bytesRemaining = metricsRegistry.newGauge(GROUP, "bytes-remaining", new AtomicLong()); + + + this.restoreRate = metricsRegistry.newCounter(GROUP, "restore-rate"); + + this.storePreRestoreNs = new ConcurrentHashMap<>(); + this.storeRestoreNs = new ConcurrentHashMap<>(); + this.storePostRestoreNs = new ConcurrentHashMap<>(); + + this.avgFileRestoreNs = metricsRegistry.newTimer(GROUP, "avg-file-restore-ns"); + } + + public void initStoreMetrics(Collection storeNames) { + for (String storeName: storeNames) { + storePreRestoreNs.putIfAbsent(storeName, + metricsRegistry.newGauge(GROUP, String.format("%s-pre-restore-ns", storeName), 0L)); + storeRestoreNs.putIfAbsent(storeName, + metricsRegistry.newGauge(GROUP, String.format("%s-restore-ns", storeName), 0L)); + storePostRestoreNs.putIfAbsent(storeName, + metricsRegistry.newGauge(GROUP, String.format("%s-post-restore-ns", storeName), 0L)); + } + } +} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java new file mode 100644 index 0000000000..9c8b61f522 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java @@ -0,0 +1,622 @@ +/* + * 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.samza.storage.blobstore.util; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.ExecutorService; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.zip.CRC32; +import java.util.zip.CheckedInputStream; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.samza.SamzaException; +import org.apache.samza.checkpoint.Checkpoint; +import org.apache.samza.checkpoint.CheckpointV2; +import org.apache.samza.storage.blobstore.BlobStoreManager; +import org.apache.samza.storage.blobstore.BlobStoreStateBackendFactory; +import org.apache.samza.storage.blobstore.Metadata; +import org.apache.samza.storage.blobstore.diff.DirDiff; +import org.apache.samza.storage.blobstore.exceptions.DeletedException; +import org.apache.samza.storage.blobstore.exceptions.RetriableException; +import org.apache.samza.storage.blobstore.index.DirIndex; +import org.apache.samza.storage.blobstore.index.FileBlob; +import org.apache.samza.storage.blobstore.index.FileIndex; +import org.apache.samza.storage.blobstore.index.FileMetadata; +import org.apache.samza.storage.blobstore.index.SnapshotIndex; +import org.apache.samza.storage.blobstore.index.SnapshotMetadata; +import org.apache.samza.storage.blobstore.index.serde.SnapshotIndexSerde; +import org.apache.samza.storage.blobstore.metrics.BlobStoreBackupManagerMetrics; +import org.apache.samza.storage.blobstore.metrics.BlobStoreRestoreManagerMetrics; +import org.apache.samza.util.FutureUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Helper methods to interact with remote blob store service and GET/PUT/DELETE a + * {@link SnapshotIndex} or {@link DirDiff}. + */ +public class BlobStoreUtil { + private static final Logger LOG = LoggerFactory.getLogger(BlobStoreUtil.class); + + private final BlobStoreManager blobStoreManager; + private final ExecutorService executor; + private final BlobStoreBackupManagerMetrics backupMetrics; + private final BlobStoreRestoreManagerMetrics restoreMetrics; + + public BlobStoreUtil(BlobStoreManager blobStoreManager, ExecutorService executor, + BlobStoreBackupManagerMetrics backupMetrics, BlobStoreRestoreManagerMetrics restoreMetrics) { + this.blobStoreManager = blobStoreManager; + this.executor = executor; + this.backupMetrics = backupMetrics; + this.restoreMetrics = restoreMetrics; + } + + /** + * Get the blob id of {@link SnapshotIndex} and {@link SnapshotIndex}es for the provided {@code task} + * in the provided {@code checkpoint}. + * @param jobName job name is used to build request metadata + * @param jobId job id is used to build request metadata + * @param taskName task name to get the store state checkpoint markers and snapshot indexes for + * @param checkpoint {@link Checkpoint} instance to get the store state checkpoint markers from. Only + * {@link CheckpointV2} and newer are supported for blob stores. + * @return Map of store name to its blob id of snapshot indices and their corresponding snapshot indices for the task. + */ + public Map> getStoreSnapshotIndexes( + String jobName, String jobId, String taskName, Checkpoint checkpoint) { + //TODO MED shesharma document error handling (checkpoint ver, blob not found, getBlob) + if (checkpoint == null) { + LOG.debug("No previous checkpoint found for taskName: {}", taskName); + return ImmutableMap.of(); + } + + if (checkpoint.getVersion() == 1) { + throw new SamzaException("Checkpoint version 1 is not supported for blob store backup and restore."); + } + + Map>> + storeSnapshotIndexFutures = new HashMap<>(); + + CheckpointV2 checkpointV2 = (CheckpointV2) checkpoint; + Map> factoryToStoreSCMs = checkpointV2.getStateCheckpointMarkers(); + Map storeSnapshotIndexBlobIds = factoryToStoreSCMs.get(BlobStoreStateBackendFactory.class.getName()); + + if (storeSnapshotIndexBlobIds != null) { + storeSnapshotIndexBlobIds.forEach((storeName, snapshotIndexBlobId) -> { + try { + LOG.debug("Getting snapshot index for taskName: {} store: {} blobId: {}", taskName, storeName, snapshotIndexBlobId); + Metadata requestMetadata = + new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), jobName, jobId, taskName, storeName); + CompletableFuture snapshotIndexFuture = + getSnapshotIndex(snapshotIndexBlobId, requestMetadata).toCompletableFuture(); + Pair, CompletableFuture> pairOfFutures = + Pair.of(CompletableFuture.completedFuture(snapshotIndexBlobId), snapshotIndexFuture); + + // save the future and block once in the end instead of blocking for each request. + storeSnapshotIndexFutures.put(storeName, FutureUtil.toFutureOfPair(pairOfFutures)); + } catch (Exception e) { + throw new SamzaException( + String.format("Error getting SnapshotIndex for blobId: %s for taskName: %s store: %s", + snapshotIndexBlobId, taskName, storeName), e); + } + }); + } else { + LOG.debug("No store SCMs found for blob store state backend in for taskName: {} in checkpoint {}", + taskName, checkpointV2.getCheckpointId()); + } + + try { + return FutureUtil.toFutureOfMap(t -> { + Throwable unwrappedException = FutureUtil.unwrapExceptions(CompletionException.class, t); + if (unwrappedException instanceof DeletedException) { + LOG.warn("Ignoring already deleted snapshot index for taskName: {}", taskName, t); + return true; + } else { + return false; + } + }, storeSnapshotIndexFutures).join(); + } catch (Exception e) { + throw new SamzaException( + String.format("Error while waiting to get store snapshot indexes for task %s", taskName), e); + } + } + + /** + * GETs the {@link SnapshotIndex} from the blob store. + * @param blobId blob ID of the {@link SnapshotIndex} to get + * @return a Future containing the {@link SnapshotIndex} + */ + public CompletableFuture getSnapshotIndex(String blobId, Metadata metadata) { + Preconditions.checkState(StringUtils.isNotBlank(blobId)); + String opName = "getSnapshotIndex: " + blobId; + return FutureUtil.executeAsyncWithRetries(opName, () -> { + ByteArrayOutputStream indexBlobStream = new ByteArrayOutputStream(); // no need to close ByteArrayOutputStream + return blobStoreManager.get(blobId, indexBlobStream, metadata).toCompletableFuture() + .thenApplyAsync(f -> new SnapshotIndexSerde().fromBytes(indexBlobStream.toByteArray()), executor); + }, isCauseNonRetriable(), executor); + } + + /** + * PUTs the {@link SnapshotIndex} to the blob store. + * @param snapshotIndex SnapshotIndex to put. + * @return a Future containing the blob ID of the {@link SnapshotIndex}. + */ + public CompletableFuture putSnapshotIndex(SnapshotIndex snapshotIndex) { + byte[] bytes = new SnapshotIndexSerde().toBytes(snapshotIndex); + String opName = "putSnapshotIndex for checkpointId: " + snapshotIndex.getSnapshotMetadata().getCheckpointId(); + return FutureUtil.executeAsyncWithRetries(opName, () -> { + InputStream inputStream = new ByteArrayInputStream(bytes); // no need to close ByteArrayInputStream + SnapshotMetadata snapshotMetadata = snapshotIndex.getSnapshotMetadata(); + Metadata metadata = new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.of((long) bytes.length), + snapshotMetadata.getJobName(), snapshotMetadata.getJobId(), snapshotMetadata.getTaskName(), + snapshotMetadata.getStoreName()); + return blobStoreManager.put(inputStream, metadata).toCompletableFuture(); + }, isCauseNonRetriable(), executor); + } + + /** + * WARNING: This method deletes the **SnapshotIndex blob** from the snapshot. This should only be called to clean + * up an older snapshot **AFTER** all the files and sub-dirs to be deleted from this snapshot are already deleted + * using {@link #cleanUpDir(DirIndex, Metadata)} + * + * @param snapshotIndexBlobId blob ID of SnapshotIndex blob to delete + * @return a future that completes when the index blob is deleted from remote store. + */ + public CompletionStage deleteSnapshotIndexBlob(String snapshotIndexBlobId, Metadata metadata) { + Preconditions.checkState(StringUtils.isNotBlank(snapshotIndexBlobId)); + LOG.debug("Deleting SnapshotIndex blob: {} from blob store", snapshotIndexBlobId); + String opName = "deleteSnapshotIndexBlob: " + snapshotIndexBlobId; + return FutureUtil.executeAsyncWithRetries(opName, () -> + blobStoreManager.delete(snapshotIndexBlobId, metadata).toCompletableFuture(), isCauseNonRetriable(), executor); + } + + /** + * Non-blocking restore of a {@link SnapshotIndex} to local store by downloading all the files and sub-dirs associated + * with this remote snapshot. + * @return A future that completes when all the async downloads completes + */ + public CompletableFuture restoreDir(File baseDir, DirIndex dirIndex, Metadata metadata) { + LOG.debug("Restoring contents of directory: {} from remote snapshot.", baseDir); + + List> downloadFutures = new ArrayList<>(); + + try { + // create parent directories if they don't exist + Files.createDirectories(baseDir.toPath()); + } catch (IOException exception) { + LOG.error("Error creating directory: {} for restore", baseDir.getAbsolutePath(), exception); + throw new SamzaException(String.format("Error creating directory: %s for restore", + baseDir.getAbsolutePath()), exception); + } + + // restore all files in the directory + for (FileIndex fileIndex : dirIndex.getFilesPresent()) { + File fileToRestore = Paths.get(baseDir.getAbsolutePath(), fileIndex.getFileName()).toFile(); + Metadata requestMetadata = + new Metadata(fileToRestore.getAbsolutePath(), Optional.of(fileIndex.getFileMetadata().getSize()), + metadata.getJobName(), metadata.getJobId(), metadata.getTaskName(), metadata.getStoreName()); + List fileBlobs = fileIndex.getBlobs(); + + String opName = "restoreFile: " + fileToRestore.getAbsolutePath(); + CompletableFuture fileRestoreFuture = + FutureUtil.executeAsyncWithRetries(opName, () -> getFile(fileBlobs, fileToRestore, requestMetadata), + isCauseNonRetriable(), executor); + downloadFutures.add(fileRestoreFuture); + } + + // restore any sub-directories + List subDirs = dirIndex.getSubDirsPresent(); + for (DirIndex subDir : subDirs) { + File subDirFile = Paths.get(baseDir.getAbsolutePath(), subDir.getDirName()).toFile(); + downloadFutures.add(restoreDir(subDirFile, subDir, metadata)); + } + + return FutureUtil.allOf(downloadFutures); + } + + /** + * Recursively upload all new files and upload or update contents of all subdirs in the {@link DirDiff} and return a + * Future containing the {@link DirIndex} associated with the directory. + * @param dirDiff diff for the contents of this directory + * @return A future with the {@link DirIndex} if the upload completed successfully. + */ + public CompletionStage putDir(DirDiff dirDiff, SnapshotMetadata snapshotMetadata) { + // Upload all new files in the dir + List filesToUpload = dirDiff.getFilesAdded(); + List> fileFutures = filesToUpload.stream() + .map(file -> putFile(file, snapshotMetadata)) + .collect(Collectors.toList()); + + CompletableFuture allFilesFuture = + CompletableFuture.allOf(fileFutures.toArray(new CompletableFuture[0])); + + List> subDirFutures = new ArrayList<>(); + // recursively upload all new subdirs of this dir + for (DirDiff subDirAdded: dirDiff.getSubDirsAdded()) { + subDirFutures.add(putDir(subDirAdded, snapshotMetadata)); + } + // recursively update contents of all subdirs that are retained but might have been modified + for (DirDiff subDirRetained: dirDiff.getSubDirsRetained()) { + subDirFutures.add(putDir(subDirRetained, snapshotMetadata)); + } + CompletableFuture allDirBlobsFuture = + CompletableFuture.allOf(subDirFutures.toArray(new CompletableFuture[0])); + + return CompletableFuture.allOf(allDirBlobsFuture, allFilesFuture) + .thenApplyAsync(f -> { + LOG.trace("All file and dir uploads complete for task: {} store: {}", + snapshotMetadata.getTaskName(), snapshotMetadata.getStoreName()); + List filesPresent = fileFutures.stream() + .map(blob -> blob.toCompletableFuture().join()) + .collect(Collectors.toList()); + + filesPresent.addAll(dirDiff.getFilesRetained()); + + List subDirsPresent = subDirFutures.stream() + .map(subDir -> subDir.toCompletableFuture().join()) + .collect(Collectors.toList()); + + LOG.debug("Uploaded diff for task: {} store: {} with statistics: {}", + snapshotMetadata.getTaskName(), snapshotMetadata.getStoreName(), DirDiff.getStats(dirDiff)); + + LOG.trace("Returning new DirIndex for task: {} store: {}", + snapshotMetadata.getTaskName(), snapshotMetadata.getStoreName()); + return new DirIndex(dirDiff.getDirName(), + filesPresent, + dirDiff.getFilesRemoved(), + subDirsPresent, + dirDiff.getSubDirsRemoved()); + }, executor); + } + + /** + * WARNING: Recursively delete **ALL** the associated files and subdirs within the provided {@link DirIndex}. + * @param dirIndex {@link DirIndex} whose entire contents are to be deleted. + * @param metadata {@link Metadata} related to the request + * @return a future that completes when ALL the files and subdirs associated with the dirIndex have been + * marked for deleted in the remote blob store. + */ + public CompletionStage deleteDir(DirIndex dirIndex, Metadata metadata) { + LOG.debug("Completely deleting dir: {} in blob store", dirIndex.getDirName()); + List> deleteFutures = new ArrayList<>(); + // Delete all files present in subDir + for (FileIndex file: dirIndex.getFilesPresent()) { + Metadata requestMetadata = + new Metadata(file.getFileName(), Optional.of(file.getFileMetadata().getSize()), + metadata.getJobName(), metadata.getJobId(), metadata.getTaskName(), metadata.getStoreName()); + deleteFutures.add(deleteFile(file, requestMetadata)); + } + + // Delete all subDirs present recursively + for (DirIndex subDir: dirIndex.getSubDirsPresent()) { + deleteFutures.add(deleteDir(subDir, metadata)); + } + + return CompletableFuture.allOf(deleteFutures.toArray(new CompletableFuture[0])); + } + + + /** + * Recursively issue delete requests for files and dirs marked to be removed in a previously created remote snapshot. + * Note: We do not immediately delete files/dirs to be removed when uploading a snapshot to the remote + * store. We just track them for deletion during the upload, and delete them AFTER the snapshot is uploaded, and the + * blob IDs have been persisted as part of the checkpoint. This is to prevent data loss if a failure happens + * part way through the commit. We issue delete these file/subdirs in cleanUp() phase of commit lifecycle. + * @param dirIndex the dir in the remote snapshot to clean up. + * @param metadata Metadata related to the request + * @return a future that completes when all the files and subdirs marked for deletion are cleaned up. + */ + public CompletionStage cleanUpDir(DirIndex dirIndex, Metadata metadata) { + String dirName = dirIndex.getDirName(); + if (DirIndex.ROOT_DIR_NAME.equals(dirName)) { + LOG.debug("Cleaning up root dir in blob store."); + } else { + LOG.debug("Cleaning up dir: {} in blob store.", dirIndex.getDirName()); + } + + List> cleanUpFuture = new ArrayList<>(); + List files = dirIndex.getFilesRemoved(); + for (FileIndex file: files) { + Metadata requestMetadata = + new Metadata(file.getFileName(), Optional.of(file.getFileMetadata().getSize()), metadata.getJobName(), + metadata.getJobId(), metadata.getTaskName(), metadata.getStoreName()); + cleanUpFuture.add(deleteFile(file, requestMetadata)); + } + + for (DirIndex subDirToDelete : dirIndex.getSubDirsRemoved()) { + // recursively delete ALL contents of the subDirToDelete. + cleanUpFuture.add(deleteDir(subDirToDelete, metadata)); + } + + for (DirIndex subDirToRetain : dirIndex.getSubDirsPresent()) { + // recursively clean up the subDir, only deleting files and subdirs marked for deletion. + cleanUpFuture.add(cleanUpDir(subDirToRetain, metadata)); + } + + return CompletableFuture.allOf(cleanUpFuture.toArray(new CompletableFuture[0])); + } + + /** + * Gets a file from the blob store. + * @param fileBlobs List of {@link FileBlob}s that constitute this file. + * @param fileToRestore File pointing to the local path where the file will be restored. + * @param requestMetadata {@link Metadata} associated with this request + * @return a future that completes when the file is downloaded and written or if an exception occurs. + */ + @VisibleForTesting + CompletableFuture getFile(List fileBlobs, File fileToRestore, Metadata requestMetadata) { + FileOutputStream outputStream = null; + try { + long restoreFileStartTime = System.nanoTime(); + if (fileToRestore.exists()) { + // delete the file if it already exists, e.g. from a previous retry. + Files.delete(fileToRestore.toPath()); + } + + outputStream = new FileOutputStream(fileToRestore); + final FileOutputStream finalOutputStream = outputStream; + // TODO HIGH shesharm add integration tests to ensure empty files and directories are handled correctly E2E. + fileToRestore.createNewFile(); // create file for 0 byte files (fileIndex entry but no fileBlobs). + // create a copy to ensure list being sorted is mutable. + List fileBlobsCopy = new ArrayList<>(fileBlobs); + fileBlobsCopy.sort(Comparator.comparingInt(FileBlob::getOffset)); // sort by offset. + + // chain the futures such that write to file for blobs is sequential. + // can be optimized to write concurrently to the file later. + CompletableFuture resultFuture = CompletableFuture.completedFuture(null); + for (FileBlob fileBlob : fileBlobsCopy) { + resultFuture = resultFuture.thenComposeAsync(v -> { + LOG.debug("Starting restore for file: {} with blob id: {} at offset: {}", fileToRestore, fileBlob.getBlobId(), + fileBlob.getOffset()); + return blobStoreManager.get(fileBlob.getBlobId(), finalOutputStream, requestMetadata); + }, executor); + } + + resultFuture = resultFuture.thenRunAsync(() -> { + LOG.debug("Finished restore for file: {}. Closing output stream.", fileToRestore); + try { + // flush the file contents to disk + finalOutputStream.getFD().sync(); + finalOutputStream.close(); + } catch (Exception e) { + throw new SamzaException(String.format("Error closing output stream for file: %s", fileToRestore.getAbsolutePath()), e); + } + }, executor); + + resultFuture.whenComplete((res, ex) -> { + if (restoreMetrics != null) { + restoreMetrics.avgFileRestoreNs.update(System.nanoTime() - restoreFileStartTime); + + long fileSize = requestMetadata.getPayloadSize(); + restoreMetrics.restoreRate.inc(fileSize); + restoreMetrics.filesRestored.getValue().addAndGet(1); + restoreMetrics.bytesRestored.getValue().addAndGet(fileSize); + restoreMetrics.filesRemaining.getValue().addAndGet(-1); + restoreMetrics.bytesRemaining.getValue().addAndGet(-1 * fileSize); + } + }); + return resultFuture; + } catch (Exception exception) { + try { + if (outputStream != null) { + outputStream.close(); + } + } catch (Exception err) { + LOG.error("Error closing output stream for file: {}", fileToRestore.getAbsolutePath(), err); + } + + throw new SamzaException(String.format("Error restoring file: %s in path: %s", + fileToRestore.getName(), requestMetadata.getPayloadPath()), exception); + } + } + + /** + * Upload a File to blob store. + * @param file File to upload to blob store. + * @return A future containing the {@link FileIndex} for the uploaded file. + */ + @VisibleForTesting + CompletableFuture putFile(File file, SnapshotMetadata snapshotMetadata) { + if (file == null || !file.isFile()) { + String message = file != null ? "Dir or Symbolic link" : "null"; + throw new SamzaException(String.format("Required a non-null parameter of type file, provided: %s", message)); + } + long putFileStartTime = System.nanoTime(); + + String opName = "putFile: " + file.getAbsolutePath(); + Supplier> fileUploadAction = () -> { + LOG.debug("Putting file: {} to blob store.", file.getPath()); + CompletableFuture fileBlobFuture; + CheckedInputStream inputStream = null; + try { + // TODO HIGH shesharm maybe use the more efficient CRC32C / PureJavaCRC32 impl + inputStream = new CheckedInputStream(new FileInputStream(file), new CRC32()); + CheckedInputStream finalInputStream = inputStream; + FileMetadata fileMetadata = FileMetadata.fromFile(file); + if (backupMetrics != null) { + backupMetrics.avgFileSizeBytes.update(fileMetadata.getSize()); + } + + Metadata metadata = + new Metadata(file.getAbsolutePath(), Optional.of(fileMetadata.getSize()), snapshotMetadata.getJobName(), + snapshotMetadata.getJobId(), snapshotMetadata.getTaskName(), snapshotMetadata.getStoreName()); + + fileBlobFuture = blobStoreManager.put(inputStream, metadata) + .thenApplyAsync(id -> { + LOG.trace("Put complete. Closing input stream for file: {}.", file.getPath()); + try { + finalInputStream.close(); + } catch (Exception e) { + throw new SamzaException(String.format("Error closing input stream for file: %s", + file.getAbsolutePath()), e); + } + + LOG.trace("Returning new FileIndex for file: {}.", file.getPath()); + return new FileIndex( + file.getName(), + Collections.singletonList(new FileBlob(id, 0)), + fileMetadata, + finalInputStream.getChecksum().getValue()); + }, executor).toCompletableFuture(); + } catch (Exception e) { + try { + if (inputStream != null) { + inputStream.close(); + } + } catch (Exception err) { + LOG.error("Error closing input stream for file: {}", file.getName(), err); + } + LOG.error("Error putting file: {}", file.getName(), e); + throw new SamzaException(String.format("Error putting file %s", file.getAbsolutePath()), e); + } + return fileBlobFuture; + }; + + return FutureUtil.executeAsyncWithRetries(opName, fileUploadAction, isCauseNonRetriable(), executor) + .whenComplete((res, ex) -> { + if (backupMetrics != null) { + backupMetrics.avgFileUploadNs.update(System.nanoTime() - putFileStartTime); + + long fileSize = file.length(); + backupMetrics.uploadRate.inc(fileSize); + backupMetrics.filesUploaded.getValue().addAndGet(1); + backupMetrics.bytesUploaded.getValue().addAndGet(fileSize); + backupMetrics.filesRemaining.getValue().addAndGet(-1); + backupMetrics.bytesRemaining.getValue().addAndGet(-1 * fileSize); + } + }); + } + + /** + * Delete a {@link FileIndex} from the remote store by deleting all {@link FileBlob}s associated with it. + * @param fileIndex FileIndex of the file to delete from the remote store. + * @param metadata + * @return a future that completes when the FileIndex has been marked for deletion in the remote blob store. + */ + private CompletionStage deleteFile(FileIndex fileIndex, Metadata metadata) { + List> deleteFutures = new ArrayList<>(); + List fileBlobs = fileIndex.getBlobs(); + for (FileBlob fileBlob : fileBlobs) { + LOG.debug("Deleting file: {} blobId: {} from blob store.", fileIndex.getFileName(), fileBlob.getBlobId()); + String opName = "deleteFile: " + fileIndex.getFileName() + " blobId: " + fileBlob.getBlobId(); + Supplier> fileDeletionAction = () -> + blobStoreManager.delete(fileBlob.getBlobId(), metadata).toCompletableFuture(); + CompletableFuture fileDeletionFuture = + FutureUtil.executeAsyncWithRetries(opName, fileDeletionAction, isCauseNonRetriable(), executor); + deleteFutures.add(fileDeletionFuture); + } + + return CompletableFuture.allOf(deleteFutures.toArray(new CompletableFuture[0])); + } + + /** + * Recursively mark all the blobs associated with the {@link DirIndex} to never expire (remove TTL). + * @param dirIndex the {@link DirIndex} whose contents' TTL needs to be removed + * @param metadata {@link Metadata} related to the request + * @return A future that completes when all the blobs associated with this dirIndex are marked to + * never expire. + */ + private CompletableFuture removeTTL(DirIndex dirIndex, Metadata metadata) { + String dirName = dirIndex.getDirName(); + if (DirIndex.ROOT_DIR_NAME.equals(dirName)) { + LOG.debug("Removing TTL for files and dirs present in DirIndex for root dir."); + } else { + LOG.debug("Removing TTL for files and dirs present in DirIndex for dir: {}", dirName); + } + + List> updateTTLsFuture = new ArrayList<>(); + for (DirIndex subDir: dirIndex.getSubDirsPresent()) { + updateTTLsFuture.add(removeTTL(subDir, metadata)); + } + + for (FileIndex file: dirIndex.getFilesPresent()) { + Metadata requestMetadata = + new Metadata(file.getFileName(), Optional.of(file.getFileMetadata().getSize()), + metadata.getJobName(), metadata.getJobId(), metadata.getTaskName(), metadata.getStoreName()); + List fileBlobs = file.getBlobs(); + for (FileBlob fileBlob : fileBlobs) { + String opname = "removeTTL for fileBlob: " + file.getFileName() + " with blobId: {}" + fileBlob.getBlobId(); + Supplier> ttlRemovalAction = () -> + blobStoreManager.removeTTL(fileBlob.getBlobId(), requestMetadata).toCompletableFuture(); + CompletableFuture ttlRemovalFuture = + FutureUtil.executeAsyncWithRetries(opname, ttlRemovalAction, isCauseNonRetriable(), executor); + updateTTLsFuture.add(ttlRemovalFuture); + } + } + + return CompletableFuture.allOf(updateTTLsFuture.toArray(new CompletableFuture[0])); + } + + + /** + * Marks all the blobs associated with an {@link SnapshotIndex} to never expire. + * @param snapshotIndex {@link SnapshotIndex} of the remote snapshot + * @param metadata {@link Metadata} related to the request + * @return A future that completes when all the files and subdirs associated with this remote snapshot are marked to + * never expire. + */ + public CompletionStage removeTTL(String indexBlobId, SnapshotIndex snapshotIndex, Metadata metadata) { + SnapshotMetadata snapshotMetadata = snapshotIndex.getSnapshotMetadata(); + LOG.debug("Marking contents of SnapshotIndex: {} to never expire", snapshotMetadata.toString()); + + String opName = "removeTTL for SnapshotIndex for checkpointId: " + snapshotMetadata.getCheckpointId(); + Supplier> removeDirIndexTTLAction = + () -> removeTTL(snapshotIndex.getDirIndex(), metadata).toCompletableFuture(); + CompletableFuture dirIndexTTLRemovalFuture = + FutureUtil.executeAsyncWithRetries(opName, removeDirIndexTTLAction, isCauseNonRetriable(), executor); + + return dirIndexTTLRemovalFuture.thenComposeAsync(aVoid -> { + String op2Name = "removeTTL for indexBlobId: " + indexBlobId; + Supplier> removeIndexBlobTTLAction = + () -> blobStoreManager.removeTTL(indexBlobId, metadata).toCompletableFuture(); + return FutureUtil.executeAsyncWithRetries(op2Name, removeIndexBlobTTLAction, isCauseNonRetriable(), executor); + }, executor); + } + + private static Predicate isCauseNonRetriable() { + return throwable -> { + Throwable unwrapped = FutureUtil.unwrapExceptions(CompletionException.class, throwable); + return unwrapped != null && !RetriableException.class.isAssignableFrom(unwrapped.getClass()); + }; + } +} \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/DirDiffUtil.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/DirDiffUtil.java new file mode 100644 index 0000000000..f09b56f8a2 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/DirDiffUtil.java @@ -0,0 +1,403 @@ +/* + * 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.samza.storage.blobstore.util; + +import com.google.common.base.Preconditions; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.nio.file.attribute.PosixFileAttributes; +import java.nio.file.attribute.PosixFilePermissions; +import java.util.ArrayList; +import java.util.Arrays; +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.Set; +import java.util.function.BiPredicate; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.zip.CRC32; +import java.util.zip.CheckedInputStream; +import org.apache.commons.lang3.StringUtils; +import org.apache.samza.SamzaException; +import org.apache.samza.storage.blobstore.diff.DirDiff; +import org.apache.samza.storage.blobstore.index.DirIndex; +import org.apache.samza.storage.blobstore.index.FileIndex; +import org.apache.samza.storage.blobstore.index.FileMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Provides helper methods to create a {@link DirDiff} between local and remote snapshots. + */ +public class DirDiffUtil { + private static final Logger LOG = LoggerFactory.getLogger(DirDiffUtil.class); + + /** + * Checks if a local directory and a remote directory are identical. Local and remote directories are identical iff: + * 1. The local directory has exactly the same set of files as the remote directory, and the files are themselves + * identical, as determined by {@link #areSameFile(boolean)}, except for those allowed to differ according to + * {@code filesToIgnore}. + * 2. The local directory has exactly the same set of sub-directories as the remote directory. + * + * @param filesToIgnore a set of file names to ignore during the directory comparisons + * (does not exclude directory names) + * @param compareLargeFileChecksums whether to compare checksums for large files (> 1 MB). + * @return boolean indicating whether the local and remote directory are identical. + */ + // TODO HIGH shesharm add unit tests + public BiPredicate areSameDir(Set filesToIgnore, boolean compareLargeFileChecksums) { + return (localDir, remoteDir) -> { + String remoteDirName = remoteDir.getDirName().equals(DirIndex.ROOT_DIR_NAME) ? "root" : remoteDir.getDirName(); + LOG.debug("Creating diff between local dir: {} and remote dir: {} for comparison.", + localDir.getAbsolutePath(), remoteDirName); + DirDiff dirDiff = DirDiffUtil.getDirDiff(localDir, remoteDir, DirDiffUtil.areSameFile(compareLargeFileChecksums)); + + boolean areSameDir = true; + List filesRemoved = dirDiff.getFilesRemoved().stream() + .map(FileIndex::getFileName) + .filter(name -> !filesToIgnore.contains(name)) + .collect(Collectors.toList()); + + if (!filesRemoved.isEmpty()) { + areSameDir = false; + LOG.error("Local directory: {} is missing files that are present in remote snapshot: {}", + localDir.getAbsolutePath(), StringUtils.join(filesRemoved, ", ")); + } + + List subDirsRemoved = dirDiff.getSubDirsRemoved(); + if (!subDirsRemoved.isEmpty()) { + areSameDir = false; + List missingSubDirs = subDirsRemoved.stream().map(DirIndex::getDirName).collect(Collectors.toList()); + LOG.error("Local directory: {} is missing sub-dirs that are present in remote snapshot: {}", + localDir.getAbsolutePath(), StringUtils.join(missingSubDirs, ", ")); + } + + List filesAdded = dirDiff.getFilesAdded().stream() + .map(File::getName) + .filter(name -> !filesToIgnore.contains(name)) + .collect(Collectors.toList()); + if (!filesAdded.isEmpty()) { + areSameDir = false; + LOG.error("Local directory: {} has additional files that are not present in remote snapshot: {}", + localDir.getAbsolutePath(), StringUtils.join(filesAdded, ", ")); + } + + List subDirsAdded = dirDiff.getSubDirsAdded(); + if (!subDirsAdded.isEmpty()) { + areSameDir = false; + List addedDirs = subDirsAdded.stream().map(DirDiff::getDirName).collect(Collectors.toList()); + LOG.error("Local directory: {} has additional sub-dirs that are not present in remote snapshot: {}", + localDir.getAbsolutePath(), StringUtils.join(addedDirs, ", ")); + } + + // dir diff calculation already ensures that all retained files are equal (by definition) + // recursively test that all retained sub-dirs are equal as well + Map remoteSubDirs = new HashMap<>(); + for (DirIndex subDir: remoteDir.getSubDirsPresent()) { + remoteSubDirs.put(subDir.getDirName(), subDir); + } + for (DirDiff subDirRetained: dirDiff.getSubDirsRetained()) { + String localSubDirName = subDirRetained.getDirName(); + File localSubDirFile = Paths.get(localDir.getAbsolutePath(), localSubDirName).toFile(); + DirIndex remoteSubDir = remoteSubDirs.get(localSubDirName); + boolean areSameSubDir = areSameDir(filesToIgnore, false).test(localSubDirFile, remoteSubDir); + if (!areSameSubDir) { + LOG.debug("Local sub-dir: {} and remote sub-dir: {} are not same.", + localSubDirFile.getAbsolutePath(), remoteSubDir.getDirName()); + areSameDir = false; + } + } + + LOG.debug("Local dir: {} and remote dir: {} are {}the same.", + localDir.getAbsolutePath(), remoteDirName, areSameDir ? "" : "not "); + return areSameDir; + }; + } + + /** + * Bipredicate to test a local file in the filesystem and a remote file {@link FileIndex} and find out if they represent + * the same file. Files with same attributes as well as content are same file. A SST file in a special case. They are + * immutable, so we only compare their attributes but not the content. + * @param compareLargeFileChecksums whether to compare checksums for large files (> 1 MB). + * @return BiPredicate to test similarity of local and remote files + */ + public static BiPredicate areSameFile(boolean compareLargeFileChecksums) { + return (localFile, remoteFile) -> { + if (localFile.getName().equals(remoteFile.getFileName())) { + FileMetadata remoteFileMetadata = remoteFile.getFileMetadata(); + + PosixFileAttributes localFileAttrs = null; + try { + localFileAttrs = Files.readAttributes(localFile.toPath(), PosixFileAttributes.class); + } catch (IOException e) { + LOG.error("Error reading attributes for file: {}", localFile.getAbsolutePath()); + throw new RuntimeException(String.format("Error reading attributes for file: %s", localFile.getAbsolutePath())); + } + + // Don't compare file timestamps. The ctime of a local file just restored will be different than the + // remote file, and will cause the file to be uploaded again during the first commit after restore. + + boolean areSameFiles = + localFileAttrs.size() == remoteFileMetadata.getSize() && + localFileAttrs.group().getName().equals(remoteFileMetadata.getGroup()) && + localFileAttrs.owner().getName().equals(remoteFileMetadata.getOwner()) && + PosixFilePermissions.toString(localFileAttrs.permissions()).equals(remoteFileMetadata.getPermissions()); + + if (!areSameFiles) { + LOG.debug("Local file: {} and remote file: {} are not same. " + + "Local file attributes: {}. Remote file attributes: {}.", + localFile.getAbsolutePath(), remoteFile.getFileName(), + fileAttributesToString(localFileAttrs), remoteFile.getFileMetadata().toString()); + return false; + } else { + LOG.trace("Local file: {}. Remote file: {}. " + + "Local file attributes: {}. Remote file attributes: {}.", + localFile.getAbsolutePath(), remoteFile.getFileName(), + fileAttributesToString(localFileAttrs), remoteFile.getFileMetadata().toString()); + } + + boolean isLargeFile = localFileAttrs.size() > 1024 * 1024; + if (!compareLargeFileChecksums && isLargeFile) { + // Since RocksDB SST files are immutable after creation, we can skip the expensive checksum computations + // which requires reading the entire file. + LOG.debug("Local file: {} and remote file: {} are same. " + + "Skipping checksum calculation for large file of size: {}.", + localFile.getAbsolutePath(), remoteFile.getFileName(), localFileAttrs.size()); + return true; + } else { + try { + FileInputStream fis = new FileInputStream(localFile); + CheckedInputStream cis = new CheckedInputStream(fis, new CRC32()); + byte[] buffer = new byte[8 * 1024]; // 8 KB + while (cis.read(buffer, 0, buffer.length) >= 0) { } + long localFileChecksum = cis.getChecksum().getValue(); + cis.close(); + + boolean areSameChecksum = localFileChecksum == remoteFile.getChecksum(); + if (!areSameChecksum) { + LOG.debug("Local file: {} and remote file: {} are not same. " + + "Local checksum: {}. Remote checksum: {}", + localFile.getAbsolutePath(), remoteFile.getFileName(), localFileChecksum, remoteFile.getChecksum()); + } else { + LOG.debug("Local file: {} and remote file: {} are same. Local checksum: {}. Remote checksum: {}", + localFile.getAbsolutePath(), remoteFile.getFileName(), localFileChecksum, remoteFile.getChecksum()); + } + return areSameChecksum; + } catch (IOException e) { + throw new SamzaException("Error calculating checksum for local file: " + localFile.getAbsolutePath(), e); + } + } + } + + return false; + }; + } + + /** + * Compare the local snapshot directory and the remote snapshot directory and return the recursive diff of the two as + * a {@link DirDiff}. + * @param localSnapshotDir File representing local snapshot root directory + * @param remoteSnapshotDir {@link DirIndex} representing the remote snapshot directory + * @param areSameFile A BiPredicate to test if a local and remote file are the same file + * @return {@link DirDiff} representing the recursive diff of local and remote snapshots directories + */ + public static DirDiff getDirDiff(File localSnapshotDir, DirIndex remoteSnapshotDir, + BiPredicate areSameFile) { + return getDirDiff(localSnapshotDir, remoteSnapshotDir, areSameFile, true); + } + + private static DirDiff getDirDiff(File localSnapshotDir, DirIndex remoteSnapshotDir, + BiPredicate areSameFile, boolean isRootDir) { + Preconditions.checkState(localSnapshotDir != null && localSnapshotDir.isDirectory()); + Preconditions.checkNotNull(remoteSnapshotDir); + + LOG.debug("Creating DirDiff between local dir: {} and remote dir: {}", + localSnapshotDir.getPath(), remoteSnapshotDir.getDirName()); + List subDirsAdded = new ArrayList<>(); + List subDirsRetained = new ArrayList<>(); + List subDirsRemoved = new ArrayList<>(); + + // list files returns empty list if local snapshot directory is empty + List localSnapshotFiles = Arrays.asList(Objects.requireNonNull(localSnapshotDir.listFiles(File::isFile))); + List remoteSnapshotFiles = remoteSnapshotDir.getFilesPresent(); + + // list files returns empty list if local snapshot directory is empty + List localSnapshotSubDirs = Arrays.asList(Objects.requireNonNull(localSnapshotDir.listFiles(File::isDirectory))); + Set localSnapshotSubDirNames = localSnapshotSubDirs.stream() + .map(File::getName) + .collect(Collectors.toCollection(HashSet::new)); + + List remoteSnapshotSubDirs = remoteSnapshotDir.getSubDirsPresent(); + Set remoteSnapshotSubDirNames = remoteSnapshotSubDirs.stream() + .map(DirIndex::getDirName) + .collect(Collectors.toCollection(HashSet::new)); + + // TODO MED shesharm: this compares each file in directory 3 times. Categorize files in one traversal instead. + List filesToUpload = getNewFilesToUpload(remoteSnapshotFiles, localSnapshotFiles, areSameFile); + List filesToRetain = getFilesToRetain(remoteSnapshotFiles, localSnapshotFiles, areSameFile); + List filesToRemove = getFilesToRemove(remoteSnapshotFiles, localSnapshotFiles, areSameFile); + + for (File localSnapshotSubDir: localSnapshotSubDirs) { + if (!remoteSnapshotSubDirNames.contains(localSnapshotSubDir.getName())) { + LOG.debug("Subdir {} present in local snapshot but not in remote snapshot. " + + "Recursively adding subdir contents.", localSnapshotSubDir.getPath()); + subDirsAdded.add(getDiffForNewDir(localSnapshotSubDir)); + } else { + LOG.debug("Subdir {} present in local snapshot and in remote snapshot. " + + "Recursively comparing local and remote subdirs.", localSnapshotSubDir.getPath()); + DirIndex remoteSubDirIndex = + remoteSnapshotSubDirs.stream() + .filter(indexBlob -> indexBlob.getDirName().equals(localSnapshotSubDir.getName())) + .findFirst().get(); + subDirsRetained.add(getDirDiff(localSnapshotSubDir, remoteSubDirIndex, areSameFile, false)); + } + } + + // 3. Subdir in remote snapshot but not in local snapshot + for (DirIndex remoteSnapshotSubDir: remoteSnapshotSubDirs) { + if (!localSnapshotSubDirNames.contains(remoteSnapshotSubDir.getDirName())) { + LOG.debug("Subdir {} present in remote snapshot but not in local snapshot. " + + "Marking for removal from remote snapshot. ", remoteSnapshotDir.getDirName()); + subDirsRemoved.add(remoteSnapshotSubDir); + } + } + + String dirName = isRootDir ? DirIndex.ROOT_DIR_NAME : localSnapshotDir.getName(); + return new DirDiff(dirName, + filesToUpload, filesToRetain, filesToRemove, + subDirsAdded, subDirsRetained, subDirsRemoved); + } + + /** + * Builds a {@link DirDiff} from a new local directory that is not already present in the remote snapshot. + * @param localSubDir File representing the local directory to create the new {@link DirDiff} for. + */ + private static DirDiff getDiffForNewDir(File localSubDir) { + List filesAdded = new ArrayList<>(); + List subDirsAdded = new ArrayList<>(); + + File[] files = localSubDir.listFiles(); + if (files != null) { + for (File file: files) { + if (file.isFile()) { + LOG.debug("Adding file {} to local sub dir {}", file.getName(), localSubDir.getPath()); + filesAdded.add(file); + } else { + LOG.debug("Adding sub dir {} to sub dir {}", file.getName(), localSubDir.getPath()); + subDirsAdded.add(getDiffForNewDir(file)); + } + } + } + + return new DirDiff(localSubDir.getName(), filesAdded, Collections.emptyList(), Collections.emptyList(), + subDirsAdded, Collections.emptyList(), Collections.emptyList()); + } + + /** + * Returns a list of files uploaded in remote checkpoint that are not present in new local snapshot and needs to be + * deleted/reclaimed from remote store. + */ + private static List getFilesToRemove( + List remoteSnapshotFiles, List localSnapshotFiles, + BiPredicate areSameFile) { + List filesToRemove = new ArrayList<>(); + + Map localFiles = localSnapshotFiles.stream() + .collect(Collectors.toMap(File::getName, Function.identity())); + + for (FileIndex remoteFile : remoteSnapshotFiles) { + String remoteFileName = remoteFile.getFileName(); + if (!localFiles.containsKey(remoteFileName) || + !areSameFile.test(localFiles.get(remoteFileName), remoteFile)) { + LOG.debug("File {} only present in remote snapshot or is not the same as local file.", remoteFile.getFileName()); + filesToRemove.add(remoteFile); + } + } + + return filesToRemove; + } + + /** + * Returns a list of files to be uploaded to remote store that are part of new snapshot created locally. + */ + private static List getNewFilesToUpload( + List remoteSnapshotFiles, List localSnapshotFiles, + BiPredicate areSameFile) { + List filesToUpload = new ArrayList<>(); + + Map remoteFiles = remoteSnapshotFiles.stream() + .collect(Collectors.toMap(FileIndex::getFileName, Function.identity())); + + for (File localFile: localSnapshotFiles) { + String localFileName = localFile.getName(); + if (!remoteFiles.containsKey(localFileName) || + !areSameFile.test(localFile, remoteFiles.get(localFileName))) { + LOG.debug("File {} only present in local snapshot or is not the same as remote file.", localFile.getPath()); + filesToUpload.add(localFile); + } + } + + return filesToUpload; + } + + /** + * Returns a list of common files between local and remote snapshot. These files are reused from prev remote snapshot + * and do not need to be uploaded again. + */ + private static List getFilesToRetain( + List remoteSnapshotFiles, List localSnapshotFiles, + BiPredicate areSameFile) { + List filesToRetain = new ArrayList<>(); + + Map localFiles = localSnapshotFiles.stream() + .collect(Collectors.toMap(File::getName, Function.identity())); + + for (FileIndex remoteFile : remoteSnapshotFiles) { + String remoteFileName = remoteFile.getFileName(); + if (localFiles.containsKey(remoteFileName) && + areSameFile.test(localFiles.get(remoteFileName), remoteFile)) { + String localFilePath = localFiles.get(remoteFileName).getPath(); + LOG.debug("File {} present in both local and remote snapshot and is the same.", localFilePath); + filesToRetain.add(remoteFile); + } + } + + return filesToRetain; + } + + private static String fileAttributesToString(PosixFileAttributes fileAttributes) { + return "PosixFileAttributes{" + + "creationTimeMillis=" + fileAttributes.creationTime().toMillis() + + ", lastModifiedTimeMillis=" + fileAttributes.lastModifiedTime().toMillis() + + ", size=" + fileAttributes.size() + + ", owner='" + fileAttributes.owner() + '\'' + + ", group='" + fileAttributes.group() + '\'' + + ", permissions=" + PosixFilePermissions.toString(fileAttributes.permissions()) + + '}'; + } +} \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/util/FutureUtil.java b/samza-core/src/main/java/org/apache/samza/util/FutureUtil.java index dc527e984c..16e38bcf21 100644 --- a/samza-core/src/main/java/org/apache/samza/util/FutureUtil.java +++ b/samza-core/src/main/java/org/apache/samza/util/FutureUtil.java @@ -146,14 +146,14 @@ public static CompletableFuture executeAsyncWithRetries(String opName, Supplier> action, Predicate abortRetries, ExecutorService executor) { - Duration maxDuration = Duration.ofMinutes(1); + Duration maxDuration = Duration.ofMinutes(10); RetryPolicy retryPolicy = new RetryPolicy<>() - .withBackoff(100, 10000, ChronoUnit.MILLIS) + .withBackoff(100, 312500, ChronoUnit.MILLIS, 5) // 100 ms, 500 ms, 2500 ms, 12.5 s, 1.05 min, 5.20 min, 5.20 min .withMaxDuration(maxDuration) .abortOn(abortRetries) // stop retrying if predicate returns true - .onRetry(e -> LOG.warn("Action: {} attempt: {} completed with error {} after start. Retrying up to {}.", - opName, e.getAttemptCount(), e.getElapsedTime(), maxDuration, e.getLastFailure())); + .onRetry(e -> LOG.warn("Action: {} attempt: {} completed with error {} ms after start. Retrying up to {} ms.", + opName, e.getAttemptCount(), e.getElapsedTime().toMillis(), maxDuration.toMillis(), e.getLastFailure())); return Failsafe.with(retryPolicy).with(executor).getStageAsync(action::get); } diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index dceb27b1a3..1e169285db 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -25,12 +25,15 @@ import java.net.{URL, UnknownHostException} import java.nio.file.Path import java.time.Duration import java.util -import java.util.{Base64, Optional} -import java.util.concurrent.{CountDownLatch, ExecutorService, Executors, ScheduledExecutorService, ThreadPoolExecutor, TimeUnit} +import java.util.concurrent._ import java.util.function.Consumer +import java.util.{Base64, Optional} + import com.google.common.annotations.VisibleForTesting import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.samza.SamzaException import org.apache.samza.checkpoint.{CheckpointListener, OffsetManager, OffsetManagerMetrics} +import org.apache.samza.clustermanager.StandbyTaskUtil import org.apache.samza.config.{StreamConfig, _} import org.apache.samza.container.disk.DiskSpaceMonitor.Listener import org.apache.samza.container.disk.{DiskQuotaPolicyFactory, DiskSpaceMonitor, NoThrottlingDiskQuotaPolicyFactory, PollingScanDiskSpaceMonitor} @@ -38,7 +41,7 @@ import org.apache.samza.container.host.{StatisticsMonitorImpl, SystemMemoryStati import org.apache.samza.context._ import org.apache.samza.diagnostics.DiagnosticsManager import org.apache.samza.job.model.{ContainerModel, JobModel, TaskMode} -import org.apache.samza.metrics.{JmxServer, JvmMetrics, MetricsRegistry, MetricsRegistryMap, MetricsReporter} +import org.apache.samza.metrics.{JmxServer, JvmMetrics, MetricsRegistryMap, MetricsReporter} import org.apache.samza.serializers._ import org.apache.samza.serializers.model.SamzaObjectMapper import org.apache.samza.startpoint.StartpointManager @@ -49,8 +52,6 @@ import org.apache.samza.table.TableManager import org.apache.samza.task._ import org.apache.samza.util.ScalaJavaUtil.JavaOptionals import org.apache.samza.util.{Util, _} -import org.apache.samza.SamzaException -import org.apache.samza.clustermanager.StandbyTaskUtil import scala.collection.JavaConverters._ @@ -345,6 +346,23 @@ object SamzaContainer extends Logging { info("Got change log system streams: %s" format storeChangelogs) + /* + * This keeps track of the changelog SSPs that are associated with the whole container. This is used so that we can + * prefetch the metadata about the all of the changelog SSPs associated with the container whenever we need the + * metadata about some of the changelog SSPs. + * An example use case is when Samza writes offset files for stores ({@link TaskStorageManager}). Each task is + * responsible for its own offset file, but if we can do prefetching, then most tasks will already have cached + * metadata by the time they need the offset metadata. + * Note: By using all changelog streams to build the sspsToPrefetch, any fetches done for persisted stores will + * include the ssps for non-persisted stores, so this is slightly suboptimal. However, this does not increase the + * actual number of calls to the {@link SystemAdmin}, and we can decouple this logic from the per-task objects (e.g. + * {@link TaskStorageManager}). + */ + val changelogSSPMetadataCache = new SSPMetadataCache(systemAdmins, + Duration.ofSeconds(5), + SystemClock.instance, + getChangelogSSPsForContainer(containerModel, storeChangelogs).asJava) + val intermediateStreams = streamConfig .getStreamIds() .asScala @@ -453,6 +471,7 @@ object SamzaContainer extends Logging { val threadPoolSize = jobConfig.getThreadPoolSize info("Got thread pool size: " + threadPoolSize) + samzaContainerMetrics.containerThreadPoolSize.set(threadPoolSize) val taskThreadPool = if (threadPoolSize > 0) { Executors.newFixedThreadPool(threadPoolSize, @@ -500,8 +519,9 @@ object SamzaContainer extends Logging { val loggedStorageBaseDir = getLoggedStorageBaseDir(jobConfig, defaultStoreBaseDir) info("Got base directory for logged data stores: %s" format loggedStorageBaseDir) + // TODO dchen should we enforce restore factories to be subset of backup factories? val stateStorageBackendRestoreFactory = ReflectionUtil - .getObj(storageConfig.getStateBackendRestoreFactory(), classOf[StateBackendFactory]) + .getObj(storageConfig.getRestoreFactory(), classOf[StateBackendFactory]) val containerStorageManager = new ContainerStorageManager( checkpointManager, @@ -527,7 +547,7 @@ object SamzaContainer extends Logging { storeWatchPaths.addAll(containerStorageManager.getStoreDirectoryPaths) - val stateStorageBackendBackupFactories = storageConfig.getStateBackendBackupFactories.asScala.map( + val stateStorageBackendBackupFactories = storageConfig.getBackupFactories().asScala.map( ReflectionUtil.getObj(_, classOf[StateBackendFactory]) ) @@ -559,7 +579,7 @@ object SamzaContainer extends Logging { if (taskInstanceMetrics.contains(taskName) && taskInstanceMetrics.get(taskName).isDefined) taskInstanceMetrics.get(taskName).get.registry else new MetricsRegistryMap - val taskBackupManager = factory.getBackupManager(jobContext, containerContext, + val taskBackupManager = factory.getBackupManager(jobContext, containerModel, taskModel, commitThreadPool, taskMetricsRegistry, config, new SystemClock, loggedStorageBaseDir, nonLoggedStorageBaseDir) taskBackupManagerMap.put(factory.getClass.getName, taskBackupManager) @@ -616,8 +636,8 @@ object SamzaContainer extends Logging { val containerMemoryMb : Int = new ClusterManagerConfig(config).getContainerMemoryMb - val hostStatisticsMonitor : SystemStatisticsMonitor = new StatisticsMonitorImpl() - hostStatisticsMonitor.registerListener(new SystemStatisticsMonitor.Listener { + val memoryStatisticsMonitor : SystemStatisticsMonitor = new StatisticsMonitorImpl() + memoryStatisticsMonitor.registerListener(new SystemStatisticsMonitor.Listener { override def onUpdate(sample: SystemMemoryStatistics): Unit = { val physicalMemoryBytes : Long = sample.getPhysicalMemoryBytes val physicalMemoryMb : Float = physicalMemoryBytes / (1024.0F * 1024.0F) @@ -625,16 +645,7 @@ object SamzaContainer extends Logging { logger.debug("Container physical memory utilization (mb): " + physicalMemoryMb) logger.debug("Container physical memory utilization: " + memoryUtilization) samzaContainerMetrics.physicalMemoryMb.set(physicalMemoryMb) - samzaContainerMetrics.physicalMemoryUtilization.set(memoryUtilization) - - var containerThreadPoolSize : Long = 0 - var containerActiveThreads : Long = 0 - if (taskThreadPool != null) { - containerThreadPoolSize = taskThreadPool.asInstanceOf[ThreadPoolExecutor].getPoolSize - containerActiveThreads = taskThreadPool.asInstanceOf[ThreadPoolExecutor].getActiveCount - } - samzaContainerMetrics.containerThreadPoolSize.set(containerThreadPoolSize) - samzaContainerMetrics.containerActiveThreads.set(containerActiveThreads) + samzaContainerMetrics.physicalMemoryUtilization.set(memoryUtilization); } }) @@ -680,7 +691,7 @@ object SamzaContainer extends Logging { reporters = reporters, jvm = jvm, diskSpaceMonitor = diskSpaceMonitor, - hostStatisticsMonitor = hostStatisticsMonitor, + hostStatisticsMonitor = memoryStatisticsMonitor, taskThreadPool = taskThreadPool, commitThreadPool = commitThreadPool, timerExecutor = timerExecutor, @@ -690,6 +701,19 @@ object SamzaContainer extends Logging { containerStorageManager = containerStorageManager, diagnosticsManager = diagnosticsManager) } + + /** + * Builds the set of SSPs for all changelogs on this container. + */ + @VisibleForTesting + private[container] def getChangelogSSPsForContainer(containerModel: ContainerModel, + changeLogSystemStreams: util.Map[String, SystemStream]): Set[SystemStreamPartition] = { + containerModel.getTasks.values().asScala + .map(taskModel => taskModel.getChangelogPartition) + .flatMap(changelogPartition => changeLogSystemStreams.asScala.map { case (_, systemStream) => + new SystemStreamPartition(systemStream, changelogPartition) }) + .toSet + } } class SamzaContainer( @@ -719,7 +743,10 @@ class SamzaContainer( private val jobConfig = new JobConfig(config) private val taskConfig = new TaskConfig(config) - val shutdownMs: Long = taskConfig.getShutdownMs + + val shutdownMs: Long = taskConfig.getLong(TaskConfig.TASK_SHUTDOWN_MS, 5000) + + var shutdownHookThread: Thread = null var jmxServer: JmxServer = null @volatile private var status = SamzaContainerStatus.NOT_STARTED @@ -1132,4 +1159,4 @@ class SamzaContainer( hostStatisticsMonitor.stop() } } -} \ No newline at end of file +} diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ProcessJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ProcessJobFactory.scala index 32649329b2..64e24086f8 100644 --- a/samza-core/src/main/scala/org/apache/samza/job/local/ProcessJobFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/job/local/ProcessJobFactory.scala @@ -24,7 +24,7 @@ import java.util import org.apache.samza.SamzaException import org.apache.samza.application.ApplicationUtil import org.apache.samza.application.descriptors.ApplicationDescriptorUtil -import org.apache.samza.config.{Config, JobConfig, TaskConfig} +import org.apache.samza.config.{BlobStoreConfig, Config, JobConfig, StorageConfig, TaskConfig} import org.apache.samza.container.TaskName import org.apache.samza.coordinator.metadatastore.{CoordinatorStreamStore, NamespaceAwareCoordinatorStreamStore} import org.apache.samza.coordinator.stream.messages.SetChangelogMapping @@ -34,7 +34,7 @@ import org.apache.samza.job.model.JobModelUtil import org.apache.samza.job.{CommandBuilder, ShellCommandBuilder, StreamJob, StreamJobFactory} import org.apache.samza.metrics.MetricsRegistryMap import org.apache.samza.startpoint.StartpointManager -import org.apache.samza.storage.ChangelogStreamManager +import org.apache.samza.storage.{ChangelogStreamManager, StateBackendFactory} import org.apache.samza.util.{ConfigUtil, CoordinatorStreamUtil, DiagnosticsUtil, Logging, ReflectionUtil} import scala.collection.JavaConversions._ @@ -91,6 +91,17 @@ class ProcessJobFactory extends StreamJobFactory with Logging { val metadataResourceUtil = new MetadataResourceUtil(jobModel, metricsRegistry, config) metadataResourceUtil.createResources() + val storageConfig = new StorageConfig(config) + storageConfig.getBackupFactories.foreach(stateStorageBackendBackupFactory => { + val stateBackendFactory : StateBackendFactory = + ReflectionUtil.getObj(stateStorageBackendBackupFactory, classOf[StateBackendFactory]) + val stateBackendAdmin = stateBackendFactory.getAdmin(jobModel, config) + // Create resources required for state backend admin + stateBackendAdmin.createResources() + // Validate resources required for state backend admin + stateBackendAdmin.validateResources() + }) + if (new JobConfig(config).getStartpointEnabled()) { // fan out the startpoints val startpointManager = new StartpointManager(coordinatorStreamStore) diff --git a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java index 6838ec2f78..5cc79b3c7b 100644 --- a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java +++ b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java @@ -443,9 +443,9 @@ private Map> createTaskStores(Set s } for (String storeName : storesToCreate) { + List storeBackupManager = storageConfig.getStoreBackupFactory(storeName); // A store is considered durable if it is backed by a changelog or another backupManager factory - boolean isDurable = changelogSystemStreams.containsKey(storeName) || - !storageConfig.getStoreBackupManagerClassName(storeName).isEmpty(); + boolean isDurable = changelogSystemStreams.containsKey(storeName) || !storeBackupManager.isEmpty(); boolean isSideInput = this.sideInputStoreNames.contains(storeName); // Use the logged-store-base-directory for change logged stores and sideInput stores, and non-logged-store-base-dir // for non logged stores diff --git a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java index e634940999..ca7196a6b0 100644 --- a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java +++ b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java @@ -146,9 +146,9 @@ public void testGetBackupManagerFactories() { String factory3 = "factory3"; StorageConfig storageConfig = new StorageConfig(new MapConfig( ImmutableMap.of( - String.format(STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME0), factory1 + "," + factory2, - String.format(STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME1), factory1, - String.format(STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME2), factory3, + String.format(STORE_BACKUP_FACTORIES, STORE_NAME0), factory1 + "," + factory2, + String.format(STORE_BACKUP_FACTORIES, STORE_NAME1), factory1, + String.format(STORE_BACKUP_FACTORIES, STORE_NAME2), factory3, // store_name3 should use DEFAULT_STATE_BACKEND_FACTORY due to changelog presence String.format(CHANGELOG_STREAM, STORE_NAME3), "nondefault-changelog-system.streamName"), ImmutableMap.of( @@ -158,20 +158,20 @@ public void testGetBackupManagerFactories() { String.format(FACTORY, STORE_NAME3), "store3.factory.class", // this store should have no backend factory configured String.format(FACTORY, "noFactoryStore"), "noFactory.factory.class" - ) - )); - Set factories = storageConfig.getStateBackendBackupFactories(); + ) + )); + Set factories = storageConfig.getBackupFactories(); assertTrue(factories.contains(factory1)); assertTrue(factories.contains(factory2)); assertTrue(factories.contains(factory3)); - assertTrue(factories.contains(DEFAULT_STATE_BACKEND_FACTORY)); + assertTrue(factories.contains(KAFKA_STATE_BACKEND_FACTORY)); assertEquals(4, factories.size()); - assertEquals(ImmutableList.of(factory1, factory2), storageConfig.getStoreBackupManagerClassName(STORE_NAME0)); - assertEquals(ImmutableList.of(factory1), storageConfig.getStoreBackupManagerClassName(STORE_NAME1)); - assertEquals(ImmutableList.of(factory3), storageConfig.getStoreBackupManagerClassName(STORE_NAME2)); - assertEquals(DEFAULT_STATE_BACKEND_BACKUP_FACTORIES, storageConfig.getStoreBackupManagerClassName(STORE_NAME3)); - assertTrue(storageConfig.getStoreBackupManagerClassName("emptyStore").isEmpty()); - assertTrue(storageConfig.getStoreBackupManagerClassName("noFactoryStore").isEmpty()); + assertEquals(ImmutableList.of(factory1, factory2), storageConfig.getStoreBackupFactory(STORE_NAME0)); + assertEquals(ImmutableList.of(factory1), storageConfig.getStoreBackupFactory(STORE_NAME1)); + assertEquals(ImmutableList.of(factory3), storageConfig.getStoreBackupFactory(STORE_NAME2)); + assertEquals(DEFAULT_BACKUP_FACTORIES, storageConfig.getStoreBackupFactory(STORE_NAME3)); + assertTrue(storageConfig.getStoreBackupFactory("emptyStore").isEmpty()); + assertTrue(storageConfig.getStoreBackupFactory("noFactoryStore").isEmpty()); } @Test @@ -179,10 +179,10 @@ public void testGetStoreToBackup() { String targetFactory = "target.class"; StorageConfig config = new StorageConfig(new MapConfig( ImmutableMap.of( - String.format(StorageConfig.STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME0), targetFactory, - String.format(StorageConfig.STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME1), targetFactory + "," + - DEFAULT_STATE_BACKEND_FACTORY, - String.format(StorageConfig.STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME2), DEFAULT_STATE_BACKEND_FACTORY), + String.format(StorageConfig.STORE_BACKUP_FACTORIES, STORE_NAME0), targetFactory, + String.format(StorageConfig.STORE_BACKUP_FACTORIES, STORE_NAME1), targetFactory + "," + + KAFKA_STATE_BACKEND_FACTORY, + String.format(StorageConfig.STORE_BACKUP_FACTORIES, STORE_NAME2), KAFKA_STATE_BACKEND_FACTORY), ImmutableMap.of( String.format(FACTORY, STORE_NAME0), "store0.factory.class", String.format(FACTORY, STORE_NAME1), "store1.factory.class", @@ -192,9 +192,9 @@ public void testGetStoreToBackup() { ) )); - List targetStoreNames = config.getBackupStoreNamesForStateBackupFactory(targetFactory); - List defaultStoreNames = config.getBackupStoreNamesForStateBackupFactory( - DEFAULT_STATE_BACKEND_FACTORY); + List targetStoreNames = config.getStoresWithBackupFactory(targetFactory); + List defaultStoreNames = config.getStoresWithBackupFactory( + KAFKA_STATE_BACKEND_FACTORY); assertTrue(targetStoreNames.containsAll(ImmutableList.of(STORE_NAME0, STORE_NAME1))); assertEquals(2, targetStoreNames.size()); assertTrue(defaultStoreNames.containsAll(ImmutableList.of(STORE_NAME2, STORE_NAME1, STORE_NAME3))); diff --git a/samza-core/src/test/java/org/apache/samza/storage/TestTaskStorageCommitManager.java b/samza-core/src/test/java/org/apache/samza/storage/TestTaskStorageCommitManager.java index e682aef332..ea00433c35 100644 --- a/samza-core/src/test/java/org/apache/samza/storage/TestTaskStorageCommitManager.java +++ b/samza-core/src/test/java/org/apache/samza/storage/TestTaskStorageCommitManager.java @@ -20,6 +20,7 @@ package org.apache.samza.storage; import com.google.common.collect.ImmutableMap; + import java.io.File; import java.io.FileFilter; import java.io.IOException; @@ -47,10 +48,23 @@ import org.apache.samza.system.SystemStream; import org.apache.samza.system.SystemStreamPartition; import org.junit.Test; +import org.mockito.stubbing.Answer; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Matchers.any; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class TestTaskStorageCommitManager { @@ -400,7 +414,7 @@ public void testPersistToFileSystemCheckpointV1AndV2Checkpoint() throws IOExcept when(iStoreProps.isPersistedToDisk()).thenReturn(false); when(iStoreProps.isDurableStore()).thenReturn(false); - java.util.Map taskStores = ImmutableMap.of( + Map taskStores = ImmutableMap.of( "loggedPersistentStore", mockLPStore, "persistentStore", mockPStore, "loggedInMemStore", mockLIStore, @@ -410,7 +424,7 @@ public void testPersistToFileSystemCheckpointV1AndV2Checkpoint() throws IOExcept Partition changelogPartition = new Partition(0); SystemStream changelogSystemStream = new SystemStream("changelogSystem", "changelogStream"); SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition); - java.util.Map storeChangelogsStreams = ImmutableMap.of( + Map storeChangelogsStreams = ImmutableMap.of( "loggedPersistentStore", changelogSystemStream, "loggedInMemStore", new SystemStream("system", "stream") ); @@ -429,12 +443,17 @@ public void testPersistToFileSystemCheckpointV1AndV2Checkpoint() throws IOExcept Collections.emptyMap(), containerStorageManager, storeChangelogsStreams, changelogPartition, null, null, ForkJoinPool.commonPool(), storageManagerUtil, durableStoreDir, metrics)); doNothing().when(commitManager).writeChangelogOffsetFile(any(), any(), any(), any()); - + when(storageManagerUtil.getStoreCheckpointDir(any(File.class), any(CheckpointId.class))) + .thenAnswer((Answer) invocation -> { + File file = invocation.getArgumentAt(0, File.class); + CheckpointId checkpointId = invocation.getArgumentAt(1, CheckpointId.class); + return file + "-" + checkpointId; + }); CheckpointId newCheckpointId = CheckpointId.create(); String newestOffset = "1"; KafkaChangelogSSPOffset kafkaChangelogSSPOffset = new KafkaChangelogSSPOffset(newCheckpointId, newestOffset); - java.util.Map offsetsJava = ImmutableMap.of( + Map offsetsJava = ImmutableMap.of( changelogSSP, kafkaChangelogSSPOffset.toString() ); @@ -446,12 +465,12 @@ public void testPersistToFileSystemCheckpointV1AndV2Checkpoint() throws IOExcept // evoked twice, for OFFSET-V1 and OFFSET-V2 verify(commitManager).writeChangelogOffsetFile( eq("loggedPersistentStore"), eq(changelogSSP), eq(newestOffset), eq(durableStoreDir)); - File checkpointFile = Paths.get(StorageManagerUtil - .getCheckpointDirPath(durableStoreDir, kafkaChangelogSSPOffset.getCheckpointId())).toFile(); + File checkpointFile = Paths.get(storageManagerUtil.getStoreCheckpointDir( + durableStoreDir, kafkaChangelogSSPOffset.getCheckpointId())).toFile(); verify(commitManager).writeChangelogOffsetFile( eq("loggedPersistentStore"), eq(changelogSSP), eq(newestOffset), eq(checkpointFile)); - java.util.Map storeSCM = ImmutableMap.of( + Map storeSCM = ImmutableMap.of( "loggedPersistentStore", "system;loggedPersistentStoreStream;1", "persistentStore", "system;persistentStoreStream;1", "loggedInMemStore", "system;loggedInMemStoreStream;1", @@ -464,7 +483,7 @@ public void testPersistToFileSystemCheckpointV1AndV2Checkpoint() throws IOExcept // Validate only durable and persisted stores are persisted // This should be evoked twice, for checkpointV1 and checkpointV2 verify(storageManagerUtil, times(2)).getTaskStoreDir(eq(durableStoreDir), eq("loggedPersistentStore"), eq(taskName), any()); - File checkpointPath = Paths.get(StorageManagerUtil.getCheckpointDirPath(durableStoreDir, newCheckpointId)).toFile(); + File checkpointPath = Paths.get(storageManagerUtil.getStoreCheckpointDir(durableStoreDir, newCheckpointId)).toFile(); verify(storageManagerUtil).writeCheckpointV2File(eq(checkpointPath), eq(checkpoint)); } @@ -528,6 +547,13 @@ public void testPersistToFileSystemCheckpointV2Only() throws IOException { null, null, ForkJoinPool.commonPool(), storageManagerUtil, durableStoreDir, metrics)); doNothing().when(commitManager).writeChangelogOffsetFile(any(), any(), any(), any()); + when(storageManagerUtil.getStoreCheckpointDir(any(File.class), any(CheckpointId.class))) + .thenAnswer((Answer) invocation -> { + File file = invocation.getArgumentAt(0, File.class); + CheckpointId checkpointId = invocation.getArgumentAt(1, CheckpointId.class); + return file + "-" + checkpointId; + }); + CheckpointId newCheckpointId = CheckpointId.create(); java.util.Map storeSCM = ImmutableMap.of( @@ -543,7 +569,7 @@ public void testPersistToFileSystemCheckpointV2Only() throws IOException { commitManager.writeCheckpointToStoreDirectories(checkpoint); // Validate only durable and persisted stores are persisted verify(storageManagerUtil).getTaskStoreDir(eq(durableStoreDir), eq("loggedPersistentStore"), eq(taskName), any()); - File checkpointPath = Paths.get(StorageManagerUtil.getCheckpointDirPath(durableStoreDir, newCheckpointId)).toFile(); + File checkpointPath = Paths.get(storageManagerUtil.getStoreCheckpointDir(durableStoreDir, newCheckpointId)).toFile(); verify(storageManagerUtil).writeCheckpointV2File(eq(checkpointPath), eq(checkpoint)); } @@ -579,6 +605,13 @@ public void testWriteChangelogOffsetFilesV1() throws IOException { Collections.emptyMap(), containerStorageManager, storeChangelogsStreams, changelogPartition, null, null, ForkJoinPool.commonPool(), storageManagerUtil, tmpTestPath, metrics)); + when(storageManagerUtil.getStoreCheckpointDir(any(File.class), any(CheckpointId.class))) + .thenAnswer((Answer) invocation -> { + File file = invocation.getArgumentAt(0, File.class); + CheckpointId checkpointId = invocation.getArgumentAt(1, CheckpointId.class); + return file + "-" + checkpointId; + }); + doAnswer(invocation -> { String fileDir = invocation.getArgumentAt(3, File.class).getName(); SystemStreamPartition ssp = invocation.getArgumentAt(1, SystemStreamPartition.class); @@ -607,8 +640,7 @@ public void testWriteChangelogOffsetFilesV1() throws IOException { assertEquals(2, mockFileSystem.size()); // check if v2 offsets are written correctly - String v2FilePath = StorageManagerUtil - .getCheckpointDirPath(tmpTestPath, newCheckpointId); + String v2FilePath = storageManagerUtil.getStoreCheckpointDir(tmpTestPath, newCheckpointId); assertTrue(mockFileSystem.containsKey(v2FilePath)); assertTrue(mockFileSystem.get(v2FilePath).containsKey(changelogSSP)); assertEquals(1, mockFileSystem.get(v2FilePath).size()); @@ -675,6 +707,13 @@ public void testWriteChangelogOffsetFilesV2andV1() throws IOException { return null; }).when(storageManagerUtil).writeCheckpointV2File(any(), any()); + when(storageManagerUtil.getStoreCheckpointDir(any(File.class), any(CheckpointId.class))) + .thenAnswer((Answer) invocation -> { + File file = invocation.getArgumentAt(0, File.class); + CheckpointId checkpointId = invocation.getArgumentAt(1, CheckpointId.class); + return file + "-" + checkpointId; + }); + CheckpointId newCheckpointId = CheckpointId.create(); String newestOffset = "1"; @@ -689,8 +728,7 @@ public void testWriteChangelogOffsetFilesV2andV1() throws IOException { assertEquals(2, mockFileSystem.size()); // check if v2 offsets are written correctly - String v2FilePath = StorageManagerUtil - .getCheckpointDirPath(tmpTestPath, newCheckpointId); + String v2FilePath = storageManagerUtil.getStoreCheckpointDir(tmpTestPath, newCheckpointId); assertTrue(mockFileSystem.containsKey(v2FilePath)); assertTrue(mockFileSystem.get(v2FilePath).containsKey(changelogSSP)); assertEquals(1, mockFileSystem.get(v2FilePath).size()); diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreBackupManager.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreBackupManager.java new file mode 100644 index 0000000000..ca9e211bfd --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreBackupManager.java @@ -0,0 +1,542 @@ +/* + * 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.samza.storage.blobstore; + +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.MoreExecutors; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.samza.SamzaException; +import org.apache.samza.checkpoint.Checkpoint; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.checkpoint.CheckpointV1; +import org.apache.samza.checkpoint.CheckpointV2; +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; +import org.apache.samza.container.TaskName; +import org.apache.samza.job.model.ContainerModel; +import org.apache.samza.job.model.JobModel; +import org.apache.samza.job.model.TaskMode; +import org.apache.samza.job.model.TaskModel; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.Gauge; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.metrics.Timer; +import org.apache.samza.storage.StorageEngine; +import org.apache.samza.storage.StorageManagerUtil; +import org.apache.samza.storage.blobstore.diff.DirDiff; +import org.apache.samza.storage.blobstore.index.DirIndex; +import org.apache.samza.storage.blobstore.index.SnapshotIndex; +import org.apache.samza.storage.blobstore.index.SnapshotMetadata; +import org.apache.samza.storage.blobstore.metrics.BlobStoreBackupManagerMetrics; +import org.apache.samza.storage.blobstore.util.BlobStoreTestUtil; +import org.apache.samza.storage.blobstore.util.BlobStoreUtil; +import org.apache.samza.storage.blobstore.util.DirDiffUtil; +import org.apache.samza.util.Clock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import static org.mockito.Matchers.*; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.*; + + +public class TestBlobStoreBackupManager { + private final ExecutorService mockExecutor = MoreExecutors.newDirectExecutorService(); + // mock container - task - job models + private final JobModel jobModel = mock(JobModel.class); + private final ContainerModel containerModel = mock(ContainerModel.class); + private final TaskModel taskModel = mock(TaskModel.class, RETURNS_DEEP_STUBS); + private final Clock clock = mock(Clock.class); + private final BlobStoreUtil blobStoreUtil = mock(BlobStoreUtil.class); + private final BlobStoreManager blobStoreManager = mock(BlobStoreManager.class); + private final StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class); + + //job and store definition + private final CheckpointId checkpointId = CheckpointId.deserialize("1234-567"); + private final String jobName = "testJobName"; + private final String jobId = "testJobID"; + private final String taskName = "testTaskName"; + private final String prevSnapshotIndexBlobId = "testPrevBlobId"; + private Map storeStorageEngineMap = new HashMap<>(); + private Map mapConfig = new HashMap<>(); + + private final MetricsRegistry metricsRegistry = mock(MetricsRegistry.class); + private final Counter counter = mock(Counter.class); + private final Timer timer = mock(Timer.class); + private final Gauge longGauge = mock(Gauge.class); + private final Gauge atomicLongGauge = mock(Gauge.class); + + private BlobStoreBackupManager blobStoreBackupManager; + private BlobStoreBackupManagerMetrics blobStoreTaskBackupMetrics; + + // Remote and local snapshot definitions + private Map testBlobStore = new HashMap<>(); + private Map> indexBlobIdAndLocalRemoteSnapshotsPair; + private Map testStoreNameAndSCMMap; + + @Before + public void setup() throws Exception { + when(clock.currentTimeMillis()).thenReturn(1234567L); + // setup test local and remote snapshots + indexBlobIdAndLocalRemoteSnapshotsPair = setupRemoteAndLocalSnapshots(true); + // setup test store name and SCMs map + testStoreNameAndSCMMap = setupTestStoreSCMMapAndStoreBackedFactoryConfig(indexBlobIdAndLocalRemoteSnapshotsPair); + // setup: setup task backup manager with expected storeName->storageEngine map + testStoreNameAndSCMMap.forEach((storeName, scm) -> storeStorageEngineMap.put(storeName, null)); + + mapConfig.putAll(new MapConfig(ImmutableMap.of("job.name", jobName, "job.id", jobId))); + + Config config = new MapConfig(mapConfig); + + // Mock - return snapshot index for blob id from test blob store map + ArgumentCaptor captor = ArgumentCaptor.forClass(String.class); + when(blobStoreUtil.getSnapshotIndex(captor.capture(), any(Metadata.class))) + .then((Answer>) invocation -> { + String blobId = invocation.getArgumentAt(0, String.class); + return CompletableFuture.completedFuture(testBlobStore.get(blobId)); + }); + +// doNothing().when(blobStoreManager).init(); + when(taskModel.getTaskName().getTaskName()).thenReturn(taskName); + when(taskModel.getTaskMode()).thenReturn(TaskMode.Active); + + when(metricsRegistry.newCounter(anyString(), anyString())).thenReturn(counter); + when(metricsRegistry.newGauge(anyString(), anyString(), anyLong())).thenReturn(longGauge); + when(metricsRegistry.newGauge(anyString(), anyString(), any(AtomicLong.class))).thenReturn(atomicLongGauge); + when(atomicLongGauge.getValue()).thenReturn(new AtomicLong()); + when(metricsRegistry.newTimer(anyString(), anyString())).thenReturn(timer); + blobStoreTaskBackupMetrics = new BlobStoreBackupManagerMetrics(metricsRegistry); + + blobStoreBackupManager = + new MockBlobStoreBackupManager(jobModel, containerModel, taskModel, mockExecutor, + blobStoreTaskBackupMetrics, config, + Files.createTempDirectory("logged-store-").toFile(), storageManagerUtil, blobStoreManager); + } + + @Test + public void testInitWithInvalidCheckpointFails() { + // init called with null checkpoint storeStorageEngineMap + blobStoreBackupManager.init(null); + // verify delete snapshot index blob called from init 0 times because prevSnapshotMap returned from init is empty + // in case of null checkpoint. + verify(blobStoreUtil, times(0)).deleteSnapshotIndexBlob(anyString(), any(Metadata.class)); + when(blobStoreUtil.getStoreSnapshotIndexes(anyString(), anyString(), anyString(), any(Checkpoint.class))).thenCallRealMethod(); + + // init called with Checkpoint V1 -> unsupported + Checkpoint checkpoint = new CheckpointV1(new HashMap<>()); + String expectedException = "Checkpoint version 1 is not supported for blob store backup and restore."; + try { + blobStoreBackupManager.init(checkpoint); + Assert.fail("Checkpoint V1 is exepcted to fail."); + } catch (SamzaException exception) { + Assert.assertEquals(exception.getMessage(), expectedException); + } + } + + @Test + public void testUploadWithNoPreviousCheckpoints() throws IOException { + // Track directory for post cleanup + List checkpointDirsToClean = new ArrayList<>(); + + // Setup: init local/remote snapshots and back manager with no previous checkpoints + indexBlobIdAndLocalRemoteSnapshotsPair = setupRemoteAndLocalSnapshots(false); + Checkpoint checkpoint = + new CheckpointV2(checkpointId, new HashMap<>(), + ImmutableMap.of(BlobStoreStateBackendFactory.class.getName(), new HashMap<>())); + blobStoreBackupManager.init(checkpoint); + + // mock: set task store dir to return corresponding test local store and create checkpoint dir + ArgumentCaptor stringCaptor = ArgumentCaptor.forClass(String.class); + when(storageManagerUtil.getTaskStoreDir(any(File.class), stringCaptor.capture(), any(TaskName.class), any(TaskMode.class))) + .then((Answer) invocation -> { + String storeName = invocation.getArgumentAt(1, String.class); + String snapshotIndexBlobId = testStoreNameAndSCMMap.get(storeName); + String storeDir = indexBlobIdAndLocalRemoteSnapshotsPair.get(snapshotIndexBlobId).getLeft(); + try { + BlobStoreTestUtil.createTestCheckpointDirectory(storeDir, checkpointId.serialize()); // create test checkpoint dir + checkpointDirsToClean.add(storeDir + "-" + checkpointId.serialize()); // track checkpoint dir to cleanup later + } catch (IOException e) { + Assert.fail("Couldn't create checkpoint directory. Test failed."); + } + return new File(storeDir); + }); + + + ArgumentCaptor storeDirCaptor = ArgumentCaptor.forClass(File.class); + when(storageManagerUtil.getStoreCheckpointDir(storeDirCaptor.capture(), eq(checkpointId))) + .thenAnswer(new Answer() { + @Override + public String answer(InvocationOnMock invocation) throws Throwable { + File storeDir = invocation.getArgumentAt(0, File.class); + return storeDir.getAbsolutePath() + "-" + checkpointId.serialize(); + } + }); + + SortedSet actualDirDiffs = new TreeSet<>(Comparator.comparing(DirDiff::getDirName)); + // mock: mock putDir and capture DirDiff + ArgumentCaptor dirDiffCaptor = ArgumentCaptor.forClass(DirDiff.class); + ArgumentCaptor snapshotMetadataCaptor = ArgumentCaptor.forClass(SnapshotMetadata.class); + when(blobStoreUtil.putDir(dirDiffCaptor.capture(), snapshotMetadataCaptor.capture())) + .then((Answer>) invocation -> { + DirDiff dirDiff = invocation.getArgumentAt(0, DirDiff.class); + SnapshotMetadata snapshotMetadata = invocation.getArgumentAt(1, SnapshotMetadata.class); + actualDirDiffs.add(dirDiff); + SnapshotIndex snapshotIndex = testBlobStore.get(testStoreNameAndSCMMap.get(snapshotMetadata.getStoreName())); + return CompletableFuture.completedFuture(snapshotIndex.getDirIndex()); + }); + + SortedSet expectedSnapshotIndexesUploaded = indexBlobIdAndLocalRemoteSnapshotsPair.values().stream() + .map(Pair::getRight) + .collect(Collectors.toCollection(() -> new TreeSet<>(Comparator.comparing(SnapshotIndex::getCreationTimeMillis)))); + String expectedPreviousSnapshotIndexBlobId = "empty"; + // mock: mock putSnapshotIndex and capture previous snapshot index + SortedSet actualSnapshotIndexesUploaded = + new TreeSet<>(Comparator.comparing(SnapshotIndex::getCreationTimeMillis)); + final String[] actualPreviousSnapshotIndexBlobId = {"empty"}; + ArgumentCaptor snapshotIndexCaptor = ArgumentCaptor.forClass(SnapshotIndex.class); + when(blobStoreUtil.putSnapshotIndex(snapshotIndexCaptor.capture())) + .then((Answer>) invocation -> { + SnapshotIndex snapshotIndex = invocation.getArgumentAt(0, SnapshotIndex.class); + actualSnapshotIndexesUploaded.add(snapshotIndex); + if (!snapshotIndex.getPrevSnapshotIndexBlobId().equals(Optional.empty())) { + actualPreviousSnapshotIndexBlobId[0] = "not-empty"; + } + return CompletableFuture.completedFuture("random-blob-id"); + }); + + // execute + blobStoreBackupManager.upload(checkpointId, testStoreNameAndSCMMap); + + // setup expected dir diffs after execute: needs checkpoint dirs created in upload() + TreeSet expectedDirDiffs = indexBlobIdAndLocalRemoteSnapshotsPair.values().stream() + .map(localRemoteSnapshotPair -> { + File localCheckpointDir = new File(localRemoteSnapshotPair.getLeft() + "-" + checkpointId.serialize()); + DirIndex dirIndex = new DirIndex(localCheckpointDir.getName(), Collections.emptyList(), Collections.emptyList(), + Collections.emptyList(), Collections.emptyList()); + return DirDiffUtil.getDirDiff(localCheckpointDir, dirIndex, DirDiffUtil.areSameFile(false)); + }).collect(Collectors.toCollection(() -> new TreeSet<>(Comparator.comparing(DirDiff::getDirName)))); + + // assert - asset all DirDiff are put to blob store + Assert.assertEquals(actualDirDiffs, expectedDirDiffs); + // assert - assert no previous snapshot indexes were found + Assert.assertEquals(actualPreviousSnapshotIndexBlobId[0], expectedPreviousSnapshotIndexBlobId); + // assert - assert all snapshot indexes are uploaded + Assert.assertEquals(actualSnapshotIndexesUploaded, expectedSnapshotIndexesUploaded); + + // cleanup + checkpointDirsToClean.forEach(path -> { + try { + FileUtils.deleteDirectory(new File(path)); + } catch (IOException exception) { + Assert.fail("Failed to cleanup temporary checkpoint dirs."); + } + }); + } + + @Test + public void testUploadWithPreviousCheckpoints() throws IOException { + // Track directory for post cleanup + List checkpointDirsToClean = new ArrayList<>(); + + // Setup: init back manager with previous checkpoints + //indexBlobIdAndLocalRemoteSnapshotsPair = setupRemoteAndLocalSnapshots(true); + Map previousCheckpoints = + // map store name, previous snapshot index blob id + indexBlobIdAndLocalRemoteSnapshotsPair.entrySet().stream() + .collect(Collectors.toMap(e -> e.getValue().getLeft(), + e -> e.getValue().getRight().getPrevSnapshotIndexBlobId().get())); + + Checkpoint checkpoint = + new CheckpointV2(checkpointId, new HashMap<>(), + ImmutableMap.of(BlobStoreStateBackendFactory.class.getName(), previousCheckpoints)); + when(blobStoreUtil.getStoreSnapshotIndexes(anyString(), anyString(), anyString(), any(Checkpoint.class))).thenCallRealMethod(); + blobStoreBackupManager.init(checkpoint); + + // mock: set task store dir to return corresponding test local store and create checkpoint dir + ArgumentCaptor stringCaptor = ArgumentCaptor.forClass(String.class); + when(storageManagerUtil.getTaskStoreDir(any(File.class), stringCaptor.capture(), any(TaskName.class), any(TaskMode.class))) + .then((Answer) invocation -> { + String storeName = invocation.getArgumentAt(1, String.class); + String snapshotIndexBlobId = testStoreNameAndSCMMap.get(storeName); + String storeDir = indexBlobIdAndLocalRemoteSnapshotsPair.get(snapshotIndexBlobId).getLeft(); + try { // create test checkpoint dir + BlobStoreTestUtil.createTestCheckpointDirectory(storeDir, checkpointId.serialize()); + checkpointDirsToClean.add(storeDir + "-" + checkpointId.serialize()); + } catch (IOException e) { + Assert.fail("Couldn't create checkpoint directory. Test failed."); + } + return new File(storeDir); + }); + + ArgumentCaptor storeDirCaptor = ArgumentCaptor.forClass(File.class); + when(storageManagerUtil.getStoreCheckpointDir(storeDirCaptor.capture(), eq(checkpointId))) + .thenAnswer(new Answer() { + @Override + public String answer(InvocationOnMock invocation) throws Throwable { + File storeDir = invocation.getArgumentAt(0, File.class); + return storeDir.getAbsolutePath() + "-" + checkpointId.serialize(); + } + }); + + // mock: mock putDir and capture DirDiff + SortedSet actualDirDiffs = new TreeSet<>(Comparator.comparing(DirDiff::getDirName)); + ArgumentCaptor dirDiffCaptor = ArgumentCaptor.forClass(DirDiff.class); + ArgumentCaptor snapshotMetadataCaptor = ArgumentCaptor.forClass(SnapshotMetadata.class); + when(blobStoreUtil.putDir(dirDiffCaptor.capture(), snapshotMetadataCaptor.capture())) + .then((Answer>) invocation -> { + DirDiff dirDiff = invocation.getArgumentAt(0, DirDiff.class); + SnapshotMetadata snapshotMetadata = invocation.getArgumentAt(1, SnapshotMetadata.class); + actualDirDiffs.add(dirDiff); + SnapshotIndex snapshotIndex = testBlobStore.get(testStoreNameAndSCMMap.get(snapshotMetadata.getStoreName())); + return CompletableFuture.completedFuture(snapshotIndex.getDirIndex()); + }); + + // mock: mock putSnapshotIndex and capture previous snapshot index + SortedSet expectedSnapshotIndexesUploaded = indexBlobIdAndLocalRemoteSnapshotsPair.values().stream() + .map(Pair::getRight) + .collect(Collectors.toCollection(() -> new TreeSet<>(Comparator.comparing(SnapshotIndex::getCreationTimeMillis)))); + SortedSet actualSnapshotIndexesUploaded = new TreeSet<>(Comparator.comparing(SnapshotIndex::getCreationTimeMillis)); + SortedSet actualPreviousSnapshotIndexBlobIds = new TreeSet<>(); + SortedSet expectedPreviousSnapshotIndexBlobIds = new TreeSet<>(previousCheckpoints.values()); + ArgumentCaptor snapshotIndexCaptor = ArgumentCaptor.forClass(SnapshotIndex.class); + when(blobStoreUtil.putSnapshotIndex(snapshotIndexCaptor.capture())) + .then((Answer>) invocation -> { + SnapshotIndex snapshotIndex = invocation.getArgumentAt(0, SnapshotIndex.class); + actualSnapshotIndexesUploaded.add(snapshotIndex); + if (snapshotIndex.getPrevSnapshotIndexBlobId().isPresent()) { + actualPreviousSnapshotIndexBlobIds.add(snapshotIndex.getPrevSnapshotIndexBlobId().get()); + } + return CompletableFuture.completedFuture("random-blob-id"); + }); + + // execute + blobStoreBackupManager.upload(checkpointId, ImmutableMap.of()); + + TreeSet expectedDirDiffs = indexBlobIdAndLocalRemoteSnapshotsPair.values() + .stream() + .map(localRemoteSnapshotPair -> + DirDiffUtil.getDirDiff(new File(localRemoteSnapshotPair.getLeft() + "-" + checkpointId.serialize()), + localRemoteSnapshotPair.getRight().getDirIndex(), DirDiffUtil.areSameFile(false))) + .collect(Collectors.toCollection(() -> new TreeSet<>(Comparator.comparing(DirDiff::getDirName)))); + + // assert - asset all DirDiff are put to blob store + Assert.assertEquals(actualDirDiffs, expectedDirDiffs); + // assert - assert no previous snapshot indexes were found + Assert.assertEquals(actualPreviousSnapshotIndexBlobIds, expectedPreviousSnapshotIndexBlobIds); + // assert - assert all snapshot indexes are uploaded + Assert.assertEquals(actualSnapshotIndexesUploaded, expectedSnapshotIndexesUploaded); + + // cleanup + checkpointDirsToClean.forEach(path -> { + try { + FileUtils.deleteDirectory(new File(path)); + } catch (IOException exception) { + Assert.fail("Failed to cleanup temporary checkpoint dirs."); + } + }); + } + + + + @Test + public void testCleanupRemovesTTLForAllIndexBlobs() { + SortedSet actualRemoveTTLsResult = new TreeSet<>(testStoreNameAndSCMMap.values()); + + SortedSet expectedRemoveTTLsResult = new TreeSet<>(); + + // mock + ArgumentCaptor captor = ArgumentCaptor.forClass(String.class); + when(blobStoreUtil.removeTTL(captor.capture(), any(SnapshotIndex.class), any(Metadata.class))) + .then((Answer>) invocation -> { + String blobId = invocation.getArgumentAt(0, String.class); + expectedRemoveTTLsResult.add(blobId); + return CompletableFuture.completedFuture(null); + }); + + // stub out non-tested methods + when(blobStoreUtil.cleanUpDir(any(DirIndex.class), any(Metadata.class))).thenReturn(CompletableFuture.completedFuture(null)); + when(blobStoreUtil.deleteSnapshotIndexBlob(any(String.class), any(Metadata.class))).thenReturn(CompletableFuture.completedFuture(null)); + + // execute + blobStoreBackupManager.cleanUp(checkpointId, testStoreNameAndSCMMap); + + // Assert + Assert.assertEquals(actualRemoveTTLsResult, expectedRemoveTTLsResult); + } + + @Test + public void testCleanupCleansUpRemoteSnapshot() throws Exception { + SortedSet actualCleanedupDirs = indexBlobIdAndLocalRemoteSnapshotsPair.values().stream() + .map(remoteLocalPair -> remoteLocalPair.getRight().getDirIndex()) + .collect(Collectors.toCollection(() -> new TreeSet<>(Comparator.comparing(DirIndex::getDirName)))); + + SortedSet expectedCleanupDirs = new TreeSet<>(Comparator.comparing(DirIndex::getDirName)); + + // mock + ArgumentCaptor captor = ArgumentCaptor.forClass(DirIndex.class); + when(blobStoreUtil.cleanUpDir(captor.capture(), any(Metadata.class))) + .then((Answer>) invocation -> { + DirIndex dirIndex = invocation.getArgumentAt(0, DirIndex.class); + expectedCleanupDirs.add(dirIndex); + return CompletableFuture.completedFuture(null); + }); + + // stub out non-tested methods + when(blobStoreUtil.removeTTL(anyString(), any(SnapshotIndex.class), any(Metadata.class))) + .thenReturn(CompletableFuture.completedFuture(null)); + when(blobStoreUtil.deleteSnapshotIndexBlob(any(String.class), any(Metadata.class))) + .thenReturn(CompletableFuture.completedFuture(null)); + + // execute + blobStoreBackupManager.cleanUp(checkpointId, testStoreNameAndSCMMap); + + // Assert + Assert.assertEquals(actualCleanedupDirs, expectedCleanupDirs); + } + + @Test + public void testCleanupRemovesOldSnapshots() throws Exception { + TreeSet expectedOldSnapshotsRemoved = indexBlobIdAndLocalRemoteSnapshotsPair.values().stream() + .map(remoteLocalPair -> { + Optional prevSnapshotIndexBlobId = remoteLocalPair.getRight().getPrevSnapshotIndexBlobId(); + return prevSnapshotIndexBlobId.orElse(null); + }) + .collect(Collectors.toCollection(TreeSet::new)); + + SortedSet actualOldSnapshotsRemoved = new TreeSet<>(); + + // mock + ArgumentCaptor captor = ArgumentCaptor.forClass(String.class); + when(blobStoreUtil.deleteSnapshotIndexBlob(captor.capture(), any(Metadata.class))) + .then((Answer>) invocation -> { + String prevIndexBlobId = invocation.getArgumentAt(0, String.class); + actualOldSnapshotsRemoved.add(prevIndexBlobId); + return CompletableFuture.completedFuture(null); + }); + + // stub out non-tested methods + when(blobStoreUtil.removeTTL(anyString(), any(SnapshotIndex.class), any(Metadata.class))) + .thenReturn(CompletableFuture.completedFuture(null)); + when(blobStoreUtil.cleanUpDir(any(DirIndex.class), any(Metadata.class))) + .thenReturn(CompletableFuture.completedFuture(null)); + + // execute + blobStoreBackupManager.cleanUp(checkpointId, testStoreNameAndSCMMap); + + // Assert + Assert.assertEquals(actualOldSnapshotsRemoved, expectedOldSnapshotsRemoved); + } + + @Test + public void testCleanupIgnoresStoresNotConfiguredWithBlobStoreStateBackend() throws Exception { + // TODO HIGH shesharm Complete test + } + + private Map setupTestStoreSCMMapAndStoreBackedFactoryConfig(Map> indexBlobIdAndRemoteAndLocalSnapshotMap) { + Map storeNameSCMMap = new HashMap<>(); + indexBlobIdAndRemoteAndLocalSnapshotMap + .forEach((blobId, localRemoteSnapshots) -> { + mapConfig.put("stores." + localRemoteSnapshots.getLeft() + ".factory", + BlobStoreStateBackendFactory.class.getName()); + mapConfig.put("stores." + localRemoteSnapshots.getLeft() + ".backup.factories", + BlobStoreStateBackendFactory.class.getName()); + storeNameSCMMap.put(localRemoteSnapshots.getLeft(), blobId); + }); + return storeNameSCMMap; + } + + private Map> setupRemoteAndLocalSnapshots(boolean addPrevCheckpoints) throws IOException { + testBlobStore = new HashMap<>(); // reset blob store + Map> indexBlobIdAndRemoteAndLocalSnapshotMap = new HashMap<>(); + List localSnapshots = new ArrayList<>(); + List previousRemoteSnapshots = new ArrayList<>(); + + localSnapshots.add("[a, c, z/1, y/2, p/m/3, q/n/4]"); + previousRemoteSnapshots.add("[a, b, z/1, x/5, p/m/3, r/o/6]"); + + localSnapshots.add("[a, c, z/1, y/1, p/m/1, q/n/1]"); + previousRemoteSnapshots.add("[a, z/1, p/m/1]"); + + localSnapshots.add("[z/i/1, y/j/1]"); + previousRemoteSnapshots.add("[z/i/1, x/k/1]"); + + // setup local and corresponding remote snapshots + for (int i = 0; i < localSnapshots.size(); i++) { + Path localSnapshot = BlobStoreTestUtil.createLocalDir(localSnapshots.get(i)); + String testLocalSnapshot = localSnapshot.toAbsolutePath().toString(); + DirIndex dirIndex = BlobStoreTestUtil.createDirIndex(localSnapshots.get(i)); + SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, testLocalSnapshot); + Optional prevCheckpointId = Optional.empty(); + if (addPrevCheckpoints) { + prevCheckpointId = Optional.of(prevSnapshotIndexBlobId + "-" + i); + DirIndex prevDirIndex = BlobStoreTestUtil.createDirIndex(previousRemoteSnapshots.get(i)); + testBlobStore.put(prevCheckpointId.get(), + new SnapshotIndex(clock.currentTimeMillis(), snapshotMetadata, prevDirIndex, Optional.empty())); + } + SnapshotIndex testRemoteSnapshot = + new SnapshotIndex(clock.currentTimeMillis(), snapshotMetadata, dirIndex, prevCheckpointId); + indexBlobIdAndRemoteAndLocalSnapshotMap.put("blobId-" + i, Pair.of(testLocalSnapshot, testRemoteSnapshot)); + testBlobStore.put("blobId-" + i, testRemoteSnapshot); + } + return indexBlobIdAndRemoteAndLocalSnapshotMap; + } + + private class MockBlobStoreBackupManager extends BlobStoreBackupManager { + + public MockBlobStoreBackupManager(JobModel jobModel, ContainerModel containerModel, TaskModel taskModel, + ExecutorService backupExecutor, BlobStoreBackupManagerMetrics blobStoreTaskBackupMetrics, Config config, + File loggedStoreBaseDir, StorageManagerUtil storageManagerUtil, + BlobStoreManager blobStoreManager) { + super(jobModel, containerModel, taskModel, backupExecutor, blobStoreTaskBackupMetrics, config, clock, + loggedStoreBaseDir, storageManagerUtil, blobStoreManager); + } + + @Override + protected BlobStoreUtil createBlobStoreUtil(BlobStoreManager blobStoreManager, ExecutorService executor, + BlobStoreBackupManagerMetrics metrics) { + return blobStoreUtil; + } + } +} diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java new file mode 100644 index 0000000000..7e0da6d839 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java @@ -0,0 +1,361 @@ +/* + * 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.samza.storage.blobstore; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.MoreExecutors; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.config.BlobStoreConfig; +import org.apache.samza.config.StorageConfig; +import org.apache.samza.container.TaskName; +import org.apache.samza.job.model.TaskMode; +import org.apache.samza.metrics.MetricsRegistryMap; +import org.apache.samza.storage.StorageManagerUtil; +import org.apache.samza.storage.blobstore.index.DirIndex; +import org.apache.samza.storage.blobstore.index.SnapshotIndex; +import org.apache.samza.storage.blobstore.index.SnapshotMetadata; +import org.apache.samza.storage.blobstore.metrics.BlobStoreRestoreManagerMetrics; +import org.apache.samza.storage.blobstore.util.BlobStoreTestUtil; +import org.apache.samza.storage.blobstore.util.BlobStoreUtil; +import org.apache.samza.storage.blobstore.util.DirDiffUtil; +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.anySet; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; +import static org.mockito.Mockito.when; + + +public class TestBlobStoreRestoreManager { + private static final ExecutorService EXECUTOR = MoreExecutors.newDirectExecutorService(); + + @Test + public void testDeleteUnusedStoresRemovesStoresDeletedFromConfig() { + String jobName = "testJobName"; + String jobId = "testJobId"; + String taskName = "taskName"; + StorageConfig storageConfig = mock(StorageConfig.class); + BlobStoreConfig blobStoreConfig = mock(BlobStoreConfig.class); + SnapshotIndex mockSnapshotIndex = mock(SnapshotIndex.class); + String blobId = "blobId"; + Map> initialStoreSnapshotIndexes = + ImmutableMap.of("oldStoreName", Pair.of(blobId, mockSnapshotIndex)); + + when(storageConfig.getStoresWithBackupFactory(eq(BlobStoreStateBackendFactory.class.getName()))) + .thenReturn(ImmutableList.of("newStoreName")); + when(storageConfig.getStoresWithRestoreFactory(eq(BlobStoreStateBackendFactory.class.getName()))) + .thenReturn(ImmutableList.of("newStoreName")); + + DirIndex dirIndex = mock(DirIndex.class); + when(mockSnapshotIndex.getDirIndex()).thenReturn(dirIndex); + + BlobStoreUtil blobStoreUtil = mock(BlobStoreUtil.class); + when(blobStoreUtil.cleanUpDir(any(DirIndex.class), any(Metadata.class))).thenReturn(CompletableFuture.completedFuture(null)); + when(blobStoreUtil.deleteDir(any(DirIndex.class), any(Metadata.class))).thenReturn(CompletableFuture.completedFuture(null)); + when(blobStoreUtil.deleteSnapshotIndexBlob(anyString(), any(Metadata.class))).thenReturn(CompletableFuture.completedFuture(null)); + + BlobStoreRestoreManager.deleteUnusedStoresFromBlobStore( + jobName, jobId, taskName, storageConfig, blobStoreConfig, initialStoreSnapshotIndexes, blobStoreUtil, EXECUTOR); + + verify(blobStoreUtil, times(1)).cleanUpDir(eq(dirIndex), any(Metadata.class)); + verify(blobStoreUtil, times(1)).deleteDir(eq(dirIndex), any(Metadata.class)); + verify(blobStoreUtil, times(1)).deleteSnapshotIndexBlob(eq(blobId), any(Metadata.class)); + + } + + @Test + public void testShouldRestoreIfNoCheckpointDir() throws IOException { + String taskName = "taskName"; + String storeName = "storeName"; + DirIndex dirIndex = mock(DirIndex.class); + Path storeCheckpointDir = Paths.get("/tmp/non-existent-checkpoint-dir"); + StorageConfig storageConfig = mock(StorageConfig.class); + when(storageConfig.cleanLoggedStoreDirsOnStart(anyString())).thenReturn(false); + DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class); + + boolean shouldRestore = BlobStoreRestoreManager.shouldRestore( + taskName, storeName, dirIndex, storeCheckpointDir, storageConfig, dirDiffUtil); + + verifyZeroInteractions(dirDiffUtil); + assertTrue(shouldRestore); + } + + @Test + public void testShouldRestoreIfCleanStateOnRestartEnabled() throws IOException { + String taskName = "taskName"; + String storeName = "storeName"; + DirIndex dirIndex = mock(DirIndex.class); + Path storeCheckpointDir = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX); // must exist + StorageConfig storageConfig = mock(StorageConfig.class); + when(storageConfig.cleanLoggedStoreDirsOnStart(anyString())).thenReturn(true); // clean on restart + DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class); + + boolean shouldRestore = BlobStoreRestoreManager.shouldRestore( + taskName, storeName, dirIndex, storeCheckpointDir, storageConfig, dirDiffUtil); + + verifyZeroInteractions(dirDiffUtil); + assertTrue(shouldRestore); // should not restore, should retain checkpoint dir instead + } + + @Test + public void testShouldRestoreIfCheckpointDirNotIdenticalToRemoteSnapshot() throws IOException { + String taskName = "taskName"; + String storeName = "storeName"; + DirIndex dirIndex = mock(DirIndex.class); + Path storeCheckpointDir = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX); // must exist + StorageConfig storageConfig = mock(StorageConfig.class); + when(storageConfig.cleanLoggedStoreDirsOnStart(anyString())).thenReturn(false); + DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class); + when(dirDiffUtil.areSameDir(anySet(), anyBoolean())).thenReturn((arg1, arg2) -> false); + + boolean shouldRestore = BlobStoreRestoreManager.shouldRestore( + taskName, storeName, dirIndex, storeCheckpointDir, storageConfig, dirDiffUtil); + + assertTrue(shouldRestore); + } + + @Test + public void testShouldNotRestoreIfPreviousCheckpointDirIdenticalToRemoteSnapshot() throws IOException { + String taskName = "taskName"; + String storeName = "storeName"; + DirIndex dirIndex = mock(DirIndex.class); + Path storeCheckpointDir = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX); // must exist + StorageConfig storageConfig = mock(StorageConfig.class); + when(storageConfig.cleanLoggedStoreDirsOnStart(anyString())).thenReturn(false); + DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class); + when(dirDiffUtil.areSameDir(anySet(), anyBoolean())).thenReturn((arg1, arg2) -> true); // are same dir + + boolean shouldRestore = BlobStoreRestoreManager.shouldRestore( + taskName, storeName, dirIndex, storeCheckpointDir, storageConfig, dirDiffUtil); + + verify(dirDiffUtil, times(1)).areSameDir(anySet(), anyBoolean()); + assertFalse(shouldRestore); // should not restore, should retain checkpoint dir instead + } + + @Test + public void testRestoreDeletesStoreDir() throws IOException { + String jobName = "testJobName"; + String jobId = "testJobId"; + TaskName taskName = mock(TaskName.class); + BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(new MetricsRegistryMap()); + metrics.initStoreMetrics(ImmutableList.of("storeName")); + List storesToRestore = ImmutableList.of("storeName"); + SnapshotIndex snapshotIndex = mock(SnapshotIndex.class); + Map> prevStoreSnapshotIndexes = + ImmutableMap.of("storeName", Pair.of("blobId", snapshotIndex)); + DirIndex dirIndex = BlobStoreTestUtil.createDirIndex("[a]"); + when(snapshotIndex.getDirIndex()).thenReturn(dirIndex); + when(snapshotIndex.getSnapshotMetadata()) + .thenReturn(new SnapshotMetadata(CheckpointId.create(), "jobName", "jobId", "taskName", "storeName")); + + Path loggedBaseDir = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX); + + // create store dir to be deleted during restore + Path storeDir = Files.createTempDirectory(loggedBaseDir, "storeDir"); + StorageConfig storageConfig = mock(StorageConfig.class); + StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class); + when(storageManagerUtil.getStoreCheckpointDir(any(File.class), any(CheckpointId.class))) + .thenReturn(Paths.get(storeDir.toString(), "checkpointId").toString()); + when(storageManagerUtil.getTaskStoreDir( + eq(loggedBaseDir.toFile()), eq("storeName"), eq(taskName), eq(TaskMode.Active))) + .thenReturn(storeDir.toFile()); + BlobStoreUtil blobStoreUtil = mock(BlobStoreUtil.class); + DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class); + + // return immediately without restoring. + when(blobStoreUtil.restoreDir(eq(storeDir.toFile()), eq(dirIndex), any(Metadata.class))) + .thenReturn(CompletableFuture.completedFuture(null)); + when(dirDiffUtil.areSameDir(anySet(), anyBoolean())).thenReturn((arg1, arg2) -> true); + + BlobStoreRestoreManager.restoreStores(jobName, jobId, taskName, storesToRestore, prevStoreSnapshotIndexes, + loggedBaseDir.toFile(), storageConfig, metrics, + storageManagerUtil, blobStoreUtil, dirDiffUtil, EXECUTOR); + + // verify that the store directory restore was called and skipped (i.e. shouldRestore == true) + verify(blobStoreUtil, times(1)).restoreDir(eq(storeDir.toFile()), eq(dirIndex), any(Metadata.class)); + // verify that the store directory was deleted prior to restore + // (should still not exist at the end since restore is no-op) + assertFalse(storeDir.toFile().exists()); + } + + @Test + public void testRestoreDeletesCheckpointDirsIfRestoring() throws IOException { + String jobName = "testJobName"; + String jobId = "testJobId"; + TaskName taskName = mock(TaskName.class); + BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(new MetricsRegistryMap()); + metrics.initStoreMetrics(ImmutableList.of("storeName")); + List storesToRestore = ImmutableList.of("storeName"); + SnapshotIndex snapshotIndex = mock(SnapshotIndex.class); + Map> prevStoreSnapshotIndexes = + ImmutableMap.of("storeName", Pair.of("blobId", snapshotIndex)); + DirIndex dirIndex = BlobStoreTestUtil.createDirIndex("[a]"); + when(snapshotIndex.getDirIndex()).thenReturn(dirIndex); + CheckpointId checkpointId = CheckpointId.create(); + when(snapshotIndex.getSnapshotMetadata()) + .thenReturn(new SnapshotMetadata(checkpointId, "jobName", "jobId", "taskName", "storeName")); + + Path loggedBaseDir = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX); + + // create store dir to be deleted during restore + Path storeDir = Files.createTempDirectory(loggedBaseDir, "storeDir"); + Path storeCheckpointDir1 = Files.createTempDirectory(loggedBaseDir, "storeDir-" + checkpointId); + CheckpointId olderCheckpoint = CheckpointId.create(); + Path storeCheckpointDir2 = Files.createTempDirectory(loggedBaseDir, "storeDir-" + olderCheckpoint); + StorageConfig storageConfig = mock(StorageConfig.class); + StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class); + when(storageManagerUtil.getTaskStoreDir( + eq(loggedBaseDir.toFile()), eq("storeName"), eq(taskName), eq(TaskMode.Active))) + .thenReturn(storeDir.toFile()); + when(storageManagerUtil.getStoreCheckpointDir(eq(storeDir.toFile()), eq(checkpointId))) + .thenReturn(Paths.get(storeDir.toString(), checkpointId.toString()).toString()); + when(storageManagerUtil.getTaskStoreCheckpointDirs(any(File.class), anyString(), any(TaskName.class), any(TaskMode.class))) + .thenReturn(ImmutableList.of(storeCheckpointDir1.toFile(), storeCheckpointDir2.toFile())); + BlobStoreUtil blobStoreUtil = mock(BlobStoreUtil.class); + DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class); + + when(dirDiffUtil.areSameDir(anySet(), anyBoolean())).thenReturn((arg1, arg2) -> true); + // return immediately without restoring. + when(blobStoreUtil.restoreDir(eq(storeDir.toFile()), eq(dirIndex), any(Metadata.class))) + .thenReturn(CompletableFuture.completedFuture(null)); + + BlobStoreRestoreManager.restoreStores(jobName, jobId, taskName, storesToRestore, prevStoreSnapshotIndexes, + loggedBaseDir.toFile(), storageConfig, metrics, + storageManagerUtil, blobStoreUtil, dirDiffUtil, EXECUTOR); + + // verify that the store directory restore was called and skipped (i.e. shouldRestore == true) + verify(blobStoreUtil, times(1)).restoreDir(eq(storeDir.toFile()), eq(dirIndex), any(Metadata.class)); + // verify that the checkpoint directories were deleted prior to restore (should not exist at the end) + assertFalse(storeCheckpointDir1.toFile().exists()); + assertFalse(storeCheckpointDir2.toFile().exists()); + } + + @Test + public void testRestoreRetainsCheckpointDirsIfValid() throws IOException { + String jobName = "testJobName"; + String jobId = "testJobId"; + TaskName taskName = mock(TaskName.class); + BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(new MetricsRegistryMap()); + metrics.initStoreMetrics(ImmutableList.of("storeName")); + List storesToRestore = ImmutableList.of("storeName"); + SnapshotIndex snapshotIndex = mock(SnapshotIndex.class); + Map> prevStoreSnapshotIndexes = + ImmutableMap.of("storeName", Pair.of("blobId", snapshotIndex)); + DirIndex dirIndex = BlobStoreTestUtil.createDirIndex("[a]"); + when(snapshotIndex.getDirIndex()).thenReturn(dirIndex); + CheckpointId checkpointId = CheckpointId.create(); + when(snapshotIndex.getSnapshotMetadata()) + .thenReturn(new SnapshotMetadata(checkpointId, "jobName", "jobId", "taskName", "storeName")); + + Path loggedBaseDir = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX); + + // create store dir to be deleted during restore + Path storeDir = Files.createTempDirectory(loggedBaseDir, "storeDir-"); + + // create checkpoint dir so that shouldRestore = false (areSameDir == true later) + Path storeCheckpointDir = Files.createTempDirectory(loggedBaseDir, "storeDir-" + checkpointId + "-"); + // create a dummy file to verify after dir rename. + Path tempFile = Files.createTempFile(storeCheckpointDir, "tempFile-", null); + + StorageConfig storageConfig = mock(StorageConfig.class); + StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class); + when(storageManagerUtil.getTaskStoreDir( + eq(loggedBaseDir.toFile()), eq("storeName"), eq(taskName), eq(TaskMode.Active))) + .thenReturn(storeDir.toFile()); + when(storageManagerUtil.getStoreCheckpointDir(any(File.class), eq(checkpointId))) + .thenReturn(storeCheckpointDir.toString()); + when(storageManagerUtil.getTaskStoreCheckpointDirs(any(File.class), anyString(), any(TaskName.class), any(TaskMode.class))) + .thenReturn(ImmutableList.of(storeCheckpointDir.toFile())); + BlobStoreUtil blobStoreUtil = mock(BlobStoreUtil.class); + DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class); + + // ensures shouldRestore is not called + when(dirDiffUtil.areSameDir(anySet(), anyBoolean())).thenReturn((arg1, arg2) -> true); + // return immediately without restoring. + when(blobStoreUtil.restoreDir(eq(storeDir.toFile()), eq(dirIndex), any(Metadata.class))) + .thenReturn(CompletableFuture.completedFuture(null)); + + BlobStoreRestoreManager.restoreStores(jobName, jobId, taskName, storesToRestore, prevStoreSnapshotIndexes, + loggedBaseDir.toFile(), storageConfig, metrics, + storageManagerUtil, blobStoreUtil, dirDiffUtil, EXECUTOR); + + // verify that the store directory restore was not called (should have restored from checkpoint dir) + verify(blobStoreUtil, times(0)).restoreDir(eq(storeDir.toFile()), eq(dirIndex), any(Metadata.class)); + // verify that the checkpoint dir was renamed to store dir + assertFalse(storeCheckpointDir.toFile().exists()); + assertTrue(storeDir.toFile().exists()); + assertTrue(Files.exists(Paths.get(storeDir.toString(), tempFile.getFileName().toString()))); + } + + @Test + public void testRestoreSkipsStoresWithMissingCheckpointSCM() { + // store renamed from oldStoreName to newStoreName. No SCM for newStoreName in previous checkpoint. + String jobName = "testJobName"; + String jobId = "testJobId"; + TaskName taskName = mock(TaskName.class); + BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(new MetricsRegistryMap()); + metrics.initStoreMetrics(ImmutableList.of("newStoreName")); + List storesToRestore = ImmutableList.of("newStoreName"); // new store in config + SnapshotIndex snapshotIndex = mock(SnapshotIndex.class); + Map> prevStoreSnapshotIndexes = mock(Map.class); + when(prevStoreSnapshotIndexes.containsKey("newStoreName")).thenReturn(false); + DirIndex dirIndex = mock(DirIndex.class); + when(snapshotIndex.getDirIndex()).thenReturn(dirIndex); + CheckpointId checkpointId = CheckpointId.create(); + when(snapshotIndex.getSnapshotMetadata()) + .thenReturn(new SnapshotMetadata(checkpointId, "jobName", "jobId", "taskName", "storeName")); + Path loggedBaseDir = mock(Path.class); + + // create store dir to be deleted during restore + StorageConfig storageConfig = mock(StorageConfig.class); + StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class); + BlobStoreUtil blobStoreUtil = mock(BlobStoreUtil.class); + DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class); + + BlobStoreRestoreManager.restoreStores(jobName, jobId, taskName, storesToRestore, prevStoreSnapshotIndexes, + loggedBaseDir.toFile(), storageConfig, metrics, + storageManagerUtil, blobStoreUtil, dirDiffUtil, EXECUTOR); + + // verify that we checked the previously checkpointed SCMs. + verify(prevStoreSnapshotIndexes, times(1)).containsKey(eq("newStoreName")); + // verify that the store directory restore was never called + verify(blobStoreUtil, times(0)).restoreDir(any(File.class), any(DirIndex.class), any(Metadata.class)); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/serde/TestSnapshotIndexSerde.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/serde/TestSnapshotIndexSerde.java new file mode 100644 index 0000000000..8103a5bb34 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/serde/TestSnapshotIndexSerde.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.samza.storage.blobstore.serde; + +import org.apache.samza.storage.blobstore.index.DirIndex; +import org.apache.samza.storage.blobstore.index.SnapshotIndex; +import org.apache.samza.storage.blobstore.index.SnapshotMetadata; +import org.apache.samza.storage.blobstore.index.serde.SnapshotIndexSerde; +import org.apache.samza.storage.blobstore.util.BlobStoreTestUtil; +import java.io.IOException; +import java.nio.file.Path; +import java.util.Optional; +import org.apache.samza.checkpoint.CheckpointId; +import org.junit.Assert; +import org.junit.Test; + + +public class TestSnapshotIndexSerde { + @Test + public void testSnapshotIndexSerde() throws IOException { + // create local and remote snapshots + String local = "[a, b, c/1, d/1/2]"; + String remote = "[a, b, z, c/1/2, e/1]"; + + Path localSnapshot = BlobStoreTestUtil.createLocalDir(local); + DirIndex dirIndex = BlobStoreTestUtil.createDirIndex(remote); + SnapshotMetadata snapshotMetadata = + new SnapshotMetadata(CheckpointId.create(), "job", "123", "task", "store"); + SnapshotIndex testRemoteSnapshot = + new SnapshotIndex(System.currentTimeMillis(), snapshotMetadata, dirIndex, Optional.empty()); + + SnapshotIndexSerde snapshotIndexSerde = new SnapshotIndexSerde(); + byte[] serialized = snapshotIndexSerde.toBytes(testRemoteSnapshot); + SnapshotIndex deserialized = snapshotIndexSerde.fromBytes(serialized); + + Assert.assertNotNull(deserialized); + Assert.assertEquals(deserialized, testRemoteSnapshot); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/BlobStoreTestUtil.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/BlobStoreTestUtil.java new file mode 100644 index 0000000000..2d72451951 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/BlobStoreTestUtil.java @@ -0,0 +1,296 @@ +/* + * 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.samza.storage.blobstore.util; + +import com.google.common.collect.ImmutableList; +import org.apache.samza.storage.blobstore.diff.DirDiff; +import org.apache.samza.storage.blobstore.index.DirIndex; +import org.apache.samza.storage.blobstore.index.FileBlob; +import org.apache.samza.storage.blobstore.index.FileIndex; +import org.apache.samza.storage.blobstore.index.FileMetadata; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.zip.CRC32; +import java.util.zip.Checksum; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.samza.util.FileUtil; + +/** + * Test util methods to work with local dirs, {@link DirDiff}s and {@link DirIndex}es. + */ +public class BlobStoreTestUtil { + public static final String TEMP_DIR_PREFIX = "samza-blob-store-test-"; + + public static Path createLocalDir(String files) throws IOException { + Path tempDirPath = Files.createTempDirectory(TEMP_DIR_PREFIX); + File tempDirFile = tempDirPath.toFile(); + String tempDirPathString = tempDirPath.toAbsolutePath().toString(); + + if (files.length() == 2) return tempDirPath; + String[] paths = files.substring(1, files.length() - 1).split(","); + + for (String path: paths) { + path = path.trim(); + if (!path.contains("/")) { + Path filePath = Files.createFile(Paths.get(tempDirPathString, path)); + new FileUtil().writeToTextFile(filePath.toFile(), path, false); // file contents == file name + filePath.toFile().deleteOnExit(); + } else { + String dirs = path.substring(0, path.lastIndexOf("/")); + String file = path.substring(path.lastIndexOf("/") + 1); + Path directories = Files.createDirectories(Paths.get(tempDirPathString, dirs)); + if (!StringUtils.isBlank(file)) { // can be blank for empty directories + Path filePath = Paths.get(directories.toAbsolutePath().toString(), file); + Files.createFile(filePath); + new FileUtil().writeToTextFile(filePath.toFile(), file, false); // file contents == file name + filePath.toFile().deleteOnExit(); + } + } + } + + deleteDirRecursivelyOnExit(tempDirFile); + return tempDirPath; + } + + public static DirIndex createDirIndex(String files) throws IOException { + if (files.equals("[]")) { // empty dir + return new DirIndex(DirIndex.ROOT_DIR_NAME, Collections.emptyList(), Collections.emptyList(), + Collections.emptyList(), Collections.emptyList()); + } + + String[] paths = files.substring(1, files.length() - 1).split(","); + Arrays.sort(paths); + // actually create the directory structure in a temp dir so that file properties and checksums can be computed + Path localDir = createLocalDir(files); + DirTreeNode dirTree = createDirTree(localDir.toAbsolutePath().toString(), paths); + return createDirIndex(localDir.toAbsolutePath().toString(), dirTree); + } + + public static void getAllAddedInDiff(String basePath, DirDiff dirDiff, Set allAdded) { + for (File fileAdded: dirDiff.getFilesAdded()) { + allAdded.add(fileAdded.getAbsolutePath().substring(basePath.length() + 1)); + } + + for (DirDiff dirAdded: dirDiff.getSubDirsAdded()) { + getAllAddedInDiff(basePath, dirAdded, allAdded); + } + + for (DirDiff dirRetained: dirDiff.getSubDirsRetained()) { + getAllAddedInDiff(basePath, dirRetained, allAdded); + } + } + + public static void getAllRemovedInDiff(String basePath, DirDiff dirDiff, Set allRemoved) { + String prefix = basePath.isEmpty() ? basePath : basePath + "/"; + for (FileIndex fileRemoved: dirDiff.getFilesRemoved()) { + allRemoved.add(prefix + fileRemoved.getFileName()); + } + + for (DirIndex dirRemoved: dirDiff.getSubDirsRemoved()) { + getAllRemovedInRemovedSubDir(prefix + dirRemoved.getDirName(), dirRemoved, allRemoved); + } + + for (DirDiff dirRetained: dirDiff.getSubDirsRetained()) { + getAllRemovedInRetainedSubDir(prefix + dirRetained.getDirName(), dirRetained, allRemoved); + } + } + + public static void getAllRetainedInDiff(String basePath, DirDiff dirDiff, Set allRetained) { + String prefix = basePath.isEmpty() ? basePath : basePath + "/"; + for (FileIndex fileRetained: dirDiff.getFilesRetained()) { + allRetained.add(prefix + fileRetained.getFileName()); + } + + for (DirDiff dirRetained: dirDiff.getSubDirsRetained()) { + getAllRetainedInDiff(prefix + dirRetained.getDirName(), dirRetained, allRetained); + } + } + + public static void getAllPresentInIndex(String basePath, DirIndex dirIndex, Set allPresent) { + String prefix = basePath.isEmpty() ? basePath : basePath + "/"; + + for (FileIndex filePresent: dirIndex.getFilesPresent()) { + allPresent.add(prefix + filePresent.getFileName()); + } + + for (DirIndex dirPresent: dirIndex.getSubDirsPresent()) { + getAllPresentInIndex(prefix + dirPresent.getDirName(), dirPresent, allPresent); + } + } + + public static void getAllRemovedInIndex(String basePath, DirIndex dirIndex, Set allRemoved) { + String prefix = basePath.isEmpty() ? basePath : basePath + "/"; + + for (FileIndex fileRemoved: dirIndex.getFilesRemoved()) { + allRemoved.add(prefix + fileRemoved.getFileName()); + } + + for (DirIndex dirRemoved: dirIndex.getSubDirsRemoved()) { + getAllRemovedInRemovedSubDir(prefix + dirRemoved.getDirName(), dirRemoved, allRemoved); + } + + for (DirIndex dirPresent: dirIndex.getSubDirsPresent()) { + getAllRemovedInIndex(prefix + dirPresent.getDirName(), dirPresent, allRemoved); + } + } + + public static SortedSet getExpected(String expectedFiles) { + if (expectedFiles.length() == 2) return new TreeSet<>(); + String[] paths = expectedFiles.substring(1, expectedFiles.length() - 1).split(","); + SortedSet result = new TreeSet<>(); + for (String path: paths) { + result.add(path.trim()); + } + return result; + } + + private static DirIndex createDirIndex(String baseDir, DirTreeNode root) { + String dirName = root.fileName; + List filesPresent = new ArrayList<>(); + List subDirsPresent = new ArrayList<>(); + + List filesRemoved = Collections.emptyList(); + List subDirsRemoved = Collections.emptyList(); + + for (DirTreeNode child: root.children) { + if (!child.children.isEmpty()) { + subDirsPresent.add(createDirIndex(baseDir + "/" + child.fileName, child)); + } else { + filesPresent.add(createFileIndex(baseDir + "/" + child.fileName, child)); + } + } + + return new DirIndex(dirName, filesPresent, filesRemoved, subDirsPresent, subDirsRemoved); + } + + private static DirTreeNode createDirTree(String baseDir, String[] paths) { + DirTreeNode root = new DirTreeNode(); + root.fileName = DirIndex.ROOT_DIR_NAME; + + for (String path: paths) { + DirTreeNode pathRoot = root; + path = path.trim(); + String[] pathParts = path.split("/"); + for (String pathPart: pathParts) { + DirTreeNode childNode; + + Optional childNodeOptional = pathRoot.children.stream() + .filter(dtn -> dtn.fileName.equals(pathPart)).findFirst(); + + if (childNodeOptional.isPresent()) { + childNode = childNodeOptional.get(); + } else { + childNode = new DirTreeNode(); + childNode.fileName = pathPart; + pathRoot.children.add(childNode); + } + pathRoot = childNode; + } + } + + return root; + } + + public static File createTestCheckpointDirectory(String storePath, String checkpointId) throws IOException { + File checkpointDir = new File(storePath + "-" + checkpointId); + FileUtils.copyDirectory(new File(storePath), checkpointDir); + return checkpointDir; + } + + private static FileIndex createFileIndex(String filePath, DirTreeNode node) { + long checksum; + FileMetadata fileMetadata; + try { + Path path = Paths.get(filePath); + Checksum crc32 = new CRC32(); + byte[] fileBytes = Files.readAllBytes(path); + crc32.update(fileBytes, 0, fileBytes.length); + checksum = crc32.getValue(); + fileMetadata = FileMetadata.fromFile(path.toFile()); + } catch (Exception e) { + throw new RuntimeException(e); + } + return new FileIndex(node.fileName, ImmutableList.of(new FileBlob(node.fileName, 0)), fileMetadata, checksum); + } + + // recursively adds all present files to allRemoved in removed subdirs + private static void getAllRemovedInRemovedSubDir(String basePath, DirIndex dirIndex, Set allRemoved) { + String prefix = basePath.isEmpty() ? basePath : basePath + "/"; + for (FileIndex f: dirIndex.getFilesPresent()) { + allRemoved.add(prefix + f.getFileName()); + } + + for (DirIndex d: dirIndex.getSubDirsPresent()) { + getAllRemovedInRemovedSubDir(prefix + d.getDirName(), d, allRemoved); + } + } + + // only adds removed files in retained subdirs + private static void getAllRemovedInRetainedSubDir(String basePath, DirDiff dirDiff, Set allRemoved) { + String prefix = basePath.isEmpty() ? basePath : basePath + "/"; + for (FileIndex f: dirDiff.getFilesRemoved()) { + allRemoved.add(prefix + f.getFileName()); + } + + for (DirIndex dirRemoved: dirDiff.getSubDirsRemoved()) { + getAllRemovedInRemovedSubDir(prefix + dirRemoved.getDirName(), dirRemoved, allRemoved); + } + + for (DirDiff dirRetained: dirDiff.getSubDirsRetained()) { + getAllRemovedInRetainedSubDir(prefix + dirRetained.getDirName(), dirRetained, allRemoved); + } + } + + private static class DirTreeNode { + String fileName; + Set children = new HashSet<>(); + } + + private static class MockFileMetadata extends FileMetadata { + public MockFileMetadata() { + super(0, 0, 0, "owner", "group", "rwxrw-r--"); + } + } + + private static void deleteDirRecursivelyOnExit(File dir) { + dir.deleteOnExit(); + for (File f: dir.listFiles()) { + if (f.isDirectory()) { + deleteDirRecursivelyOnExit(f); + } else { + f.deleteOnExit(); + } + } + } +} diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestBlobStoreUtil.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestBlobStoreUtil.java new file mode 100644 index 0000000000..b22827e0a1 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestBlobStoreUtil.java @@ -0,0 +1,936 @@ +/* + * 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.samza.storage.blobstore.util; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.MoreExecutors; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.attribute.FileTime; +import java.nio.file.attribute.PosixFileAttributes; +import java.nio.file.attribute.PosixFilePermissions; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Random; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.zip.CRC32; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.commons.io.output.NullOutputStream; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.samza.SamzaException; +import org.apache.samza.checkpoint.Checkpoint; +import org.apache.samza.checkpoint.CheckpointId; +import org.apache.samza.checkpoint.CheckpointV2; +import org.apache.samza.storage.blobstore.BlobStoreManager; +import org.apache.samza.storage.blobstore.BlobStoreStateBackendFactory; +import org.apache.samza.storage.blobstore.Metadata; +import org.apache.samza.storage.blobstore.diff.DirDiff; +import org.apache.samza.storage.blobstore.exceptions.DeletedException; +import org.apache.samza.storage.blobstore.exceptions.RetriableException; +import org.apache.samza.storage.blobstore.index.DirIndex; +import org.apache.samza.storage.blobstore.index.FileBlob; +import org.apache.samza.storage.blobstore.index.FileIndex; +import org.apache.samza.storage.blobstore.index.FileMetadata; +import org.apache.samza.storage.blobstore.index.SnapshotIndex; +import org.apache.samza.storage.blobstore.index.SnapshotMetadata; +import org.apache.samza.util.FileUtil; +import org.apache.samza.util.FutureUtil; +import org.junit.Ignore; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.stubbing.Answer; + +import static org.junit.Assert.*; +import static org.mockito.Matchers.*; +import static org.mockito.Mockito.*; + + +public class TestBlobStoreUtil { + private static final ExecutorService EXECUTOR = MoreExecutors.newDirectExecutorService(); + private final CheckpointId checkpointId = CheckpointId.deserialize("1234-567"); + private final String jobName = "jobName"; + private final String jobId = "jobId"; + private final String taskName = "taskName"; + private final String storeName = "storeName"; + private final Metadata metadata = + new Metadata("payload-path", Optional.empty(), jobName, jobId, taskName, storeName); + + @Test + // TODO HIGH shesharm test with empty (0 byte) files + public void testPutDir() throws IOException, InterruptedException, ExecutionException { + BlobStoreManager blobStoreManager = mock(BlobStoreManager.class); + + // File, dir and recursive dir added, retained and removed in local + String local = "[a, c, z/1, y/1, p/m/1, q/n/1]"; + String remote = "[a, b, z/1, x/1, p/m/1, p/m/2, r/o/1]"; + String expectedAdded = "[c, y/1, q/n/1]"; + String expectedRetained = "[a, z/1, p/m/1]"; + String expectedRemoved = "[b, x/1, r/o/1, p/m/2]"; + SortedSet expectedAddedFiles = BlobStoreTestUtil.getExpected(expectedAdded); + SortedSet expectedRetainedFiles = BlobStoreTestUtil.getExpected(expectedRetained); + SortedSet expectedPresentFiles = new TreeSet<>(expectedAddedFiles); + expectedPresentFiles.addAll(expectedRetainedFiles); + SortedSet expectedRemovedFiles = BlobStoreTestUtil.getExpected(expectedRemoved); + + // Set up environment + Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local); + String basePath = localSnapshotDir.toAbsolutePath().toString(); + DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote); + SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName); + DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir, + (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName())); + + SortedSet allUploaded = new TreeSet<>(); + // Set up mocks + when(blobStoreManager.put(any(InputStream.class), any(Metadata.class))) + .thenAnswer((Answer>) invocation -> { + Metadata metadata = invocation.getArgumentAt(1, Metadata.class); + String path = metadata.getPayloadPath(); + allUploaded.add(path.substring(localSnapshotDir.toAbsolutePath().toString().length() + 1)); + return CompletableFuture.completedFuture(path); + }); + + // Execute + BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null); + CompletionStage dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata); + DirIndex dirIndex = null; + try { + // should be already complete. if not, future composition in putDir is broken. + dirIndex = dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + fail("Future returned from putDir should be already complete."); + } + + SortedSet allPresent = new TreeSet<>(); + SortedSet allRemoved = new TreeSet<>(); + BlobStoreTestUtil.getAllPresentInIndex("", dirIndex, allPresent); + BlobStoreTestUtil.getAllRemovedInIndex("", dirIndex, allRemoved); + + // Assert + assertEquals(expectedAddedFiles, allUploaded); + assertEquals(expectedPresentFiles, allPresent); + assertEquals(expectedRemovedFiles, allRemoved); + } + + @Test + public void testPutDirFailsIfAnyFileUploadFails() throws IOException, TimeoutException, InterruptedException { + BlobStoreManager blobStoreManager = mock(BlobStoreManager.class); + + // File, dir and recursive dir added, retained and removed in local + String local = "[a, b]"; + String remote = "[]"; + + // Set up environment + Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local); + String basePath = localSnapshotDir.toAbsolutePath().toString(); + DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote); + SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName); + DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir, + (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName())); + + // Set up mocks + SamzaException exception = new SamzaException("Error uploading file"); + CompletableFuture failedFuture = new CompletableFuture<>(); + failedFuture.completeExceptionally(exception); + when(blobStoreManager.put(any(InputStream.class), any(Metadata.class))) + .thenAnswer((Answer>) invocation -> { + Metadata metadata = invocation.getArgumentAt(1, Metadata.class); + String path = metadata.getPayloadPath(); + if (path.endsWith("a")) { + return CompletableFuture.completedFuture("aBlobId"); + } else { + return failedFuture; + } + }); + + // Execute + BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null); + CompletionStage dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata); + try { + // should be already complete. if not, future composition in putDir is broken. + dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + // Assert that the result future fails and that the cause is propagated correctly + assertEquals(exception, cause); + return; + } + + fail("DirIndex future should have been completed with an exception"); + } + + @Test + public void testPutDirFailsIfAnySubDirFileUploadFails() throws IOException, TimeoutException, InterruptedException { + BlobStoreManager blobStoreManager = mock(BlobStoreManager.class); + + // File, dir and recursive dir added, retained and removed in local + String local = "[a/1, b/2]"; + String remote = "[]"; + + // Set up environment + Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local); + String basePath = localSnapshotDir.toAbsolutePath().toString(); + DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote); + SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName); + DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir, + (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName())); + + // Set up mocks + SamzaException exception = new SamzaException("Error uploading file"); + CompletableFuture failedFuture = new CompletableFuture<>(); + failedFuture.completeExceptionally(exception); + when(blobStoreManager.put(any(InputStream.class), any(Metadata.class))) + .thenAnswer((Answer>) invocation -> { + Metadata metadata = invocation.getArgumentAt(1, Metadata.class); + String path = metadata.getPayloadPath(); + if (path.endsWith("1")) { + return CompletableFuture.completedFuture("a1BlobId"); + } else { + return failedFuture; + } + }); + + // Execute + BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null); + CompletionStage dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata); + try { + // should be already complete. if not, future composition in putDir is broken. + dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + // Assert that the result future fails and that the cause is propagated correctly + assertEquals(exception, cause); + return; + } + + fail("DirIndex future should have been completed with an exception"); + } + + @Test + public void testCleanup() throws IOException, ExecutionException, InterruptedException { + BlobStoreManager blobStoreManager = mock(BlobStoreManager.class); + + // File, dir and recursive dir added, retained and removed in local + // Using unique file names since test util uses only the file name (leaf node) + // as the mock blob id, not the full file path. + String local = "[a, c, z/1, y/2, p/m/3, q/n/4]"; + String remote = "[a, b, z/1, x/5, p/m/3, r/o/6]"; + String expectedRemoved = "[b, 5, 6]"; + // keep only the last character (the file name). + SortedSet expectedRemovedFiles = BlobStoreTestUtil.getExpected(expectedRemoved); + + // Set up environment + Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local); + String basePath = localSnapshotDir.toAbsolutePath().toString(); + DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote); + SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName); + DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir, + (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName())); + + BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null); + when(blobStoreManager.put(any(InputStream.class), any(Metadata.class))) + .thenReturn(CompletableFuture.completedFuture("blobId")); + CompletionStage dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata); + DirIndex dirIndex = null; + try { + // should be already complete. if not, future composition in putDir is broken. + dirIndex = dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + fail("Future returned from putDir should be already complete."); + } + + // Set up mocks + SortedSet allDeleted = new TreeSet<>(); + when(blobStoreManager.delete(anyString(), any(Metadata.class))) + .thenAnswer((Answer>) invocation -> { + String blobId = invocation.getArgumentAt(0, String.class); + allDeleted.add(blobId); + return CompletableFuture.completedFuture(null); + }); + + // Execute + CompletionStage cleanUpFuture = blobStoreUtil.cleanUpDir(dirIndex, metadata); + try { + // should be already complete. if not, future composition in putDir is broken. + cleanUpFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + fail("Future returned from putDir should be already complete."); + } + + // Assert + assertEquals(expectedRemovedFiles, allDeleted); + } + + @Test + public void testCleanUpFailsIfAnyFileDeleteFails() + throws IOException, TimeoutException, InterruptedException, ExecutionException { + BlobStoreManager blobStoreManager = mock(BlobStoreManager.class); + + // File, dir and recursive dir added, retained and removed in local + // Using unique file names since test util uses only the file name (leaf node) + // as the mock blob id, not the full file path. + String local = "[a, b]"; + String remote = "[c, d]"; + + // Set up environment + Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local); + String basePath = localSnapshotDir.toAbsolutePath().toString(); + DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote); + SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName); + DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir, + (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName())); + + BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null); + when(blobStoreManager.put(any(InputStream.class), any(Metadata.class))) + .thenReturn(CompletableFuture.completedFuture("blobId")); + CompletionStage dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata); + DirIndex dirIndex = null; + try { + // should be already complete. if not, future composition in putDir is broken. + dirIndex = dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + fail("Future returned from putDir should be already complete."); + } + + // Set up mocks + SamzaException exception = new SamzaException("Error deleting file"); + CompletableFuture failedFuture = new CompletableFuture<>(); + failedFuture.completeExceptionally(exception); + when(blobStoreManager.delete(anyString(), any(Metadata.class))) + .thenAnswer((Answer>) invocation -> { + String blobId = invocation.getArgumentAt(0, String.class); + if (blobId.equals("c")) { + return CompletableFuture.completedFuture(null); + } else { + return failedFuture; + } + }); + + // Execute + CompletionStage cleanUpFuture = blobStoreUtil.cleanUpDir(dirIndex, metadata); + try { + // should be already complete. if not, future composition in putDir is broken. + cleanUpFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + // Assert that the result future fails and that the cause is propagated correctly + assertEquals(exception, cause); + return; + } + + fail("Clean up future should have been completed with an exception"); + } + + @Test + public void testCleanUpFailsIfAnySubDirFileDeleteFails() + throws IOException, TimeoutException, InterruptedException, ExecutionException { + BlobStoreManager blobStoreManager = mock(BlobStoreManager.class); + + // File, dir and recursive dir added, retained and removed in local + // Using unique file names since test util uses only the file name (leaf node) + // as the mock blob id, not the full file path. + String local = "[a/1, b/2]"; + String remote = "[c/3, d/4]"; + + // Set up environment + Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local); + String basePath = localSnapshotDir.toAbsolutePath().toString(); + DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote); + SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName); + DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir, + (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName())); + + BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null); + when(blobStoreManager.put(any(InputStream.class), any(Metadata.class))) + .thenReturn(CompletableFuture.completedFuture("blobId")); + CompletionStage dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata); + DirIndex dirIndex = null; + try { + // should be already complete. if not, future composition in putDir is broken. + dirIndex = dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + fail("Future returned from putDir should be already complete."); + } + + // Set up mocks + SamzaException exception = new SamzaException("Error deleting file"); + CompletableFuture failedFuture = new CompletableFuture<>(); + failedFuture.completeExceptionally(exception); + when(blobStoreManager.delete(anyString(), any(Metadata.class))) + .thenAnswer((Answer>) invocation -> { + String blobId = invocation.getArgumentAt(0, String.class); + if (blobId.equals("3")) { // blob ID == file name (leaf node) in blob store test util + return CompletableFuture.completedFuture(null); + } else { + return failedFuture; + } + }); + + // Execute + CompletionStage cleanUpFuture = blobStoreUtil.cleanUpDir(dirIndex, metadata); + try { + // should be already complete. if not, future composition in putDir is broken. + cleanUpFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + // Assert that the result future fails and that the cause is propagated correctly + assertEquals(exception, cause); + return; + } + + fail("Clean up future should have been completed with an exception"); + } + + @Test + public void testRemoveTTL() throws IOException, ExecutionException, InterruptedException { + BlobStoreManager blobStoreManager = mock(BlobStoreManager.class); + + // File, dir and recursive dir added, retained and removed in local + // Using unique file names since test setup returns it as the blob id + String local = "[a, c, z/1, y/2, p/m/3, q/n/4]"; + String remote = "[a, b, z/1, x/5, p/m/3, r/o/6]"; + String expectedAdded = "[c, y/2, q/n/4]"; + String expectedRetained = "[a, z/1, p/m/3]"; + SortedSet expectedAddedFiles = BlobStoreTestUtil.getExpected(expectedAdded); + SortedSet expectedRetainedFiles = BlobStoreTestUtil.getExpected(expectedRetained); + SortedSet expectedPresentFiles = new TreeSet<>(expectedAddedFiles); + expectedPresentFiles.addAll(expectedRetainedFiles); + + // Set up environment + Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local); + String basePath = localSnapshotDir.toAbsolutePath().toString(); + DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote); + SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName); + DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir, + (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName())); + + when(blobStoreManager.put(any(InputStream.class), any(Metadata.class))) + .thenAnswer((Answer>) invocation -> { + Metadata metadata = invocation.getArgumentAt(1, Metadata.class); + String path = metadata.getPayloadPath(); + String fileName = path.substring(path.length() - 1); // use only the last character as file name + return CompletableFuture.completedFuture(fileName); + }); + + BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null); + CompletionStage dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata); + DirIndex dirIndex = null; + try { + // should be already complete. if not, future composition in putDir is broken. + dirIndex = dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + fail("Future returned from putDir should be already complete."); + } + + SnapshotIndex mockSnapshotIndex = mock(SnapshotIndex.class); + when(mockSnapshotIndex.getSnapshotMetadata()).thenReturn(snapshotMetadata); + when(mockSnapshotIndex.getDirIndex()).thenReturn(dirIndex); + + SortedSet allTTLRemoved = new TreeSet<>(); + when(blobStoreManager.removeTTL(anyString(), any(Metadata.class))) + .thenAnswer((Answer>) invocation -> { + String blobId = invocation.getArgumentAt(0, String.class); + allTTLRemoved.add(blobId); + return CompletableFuture.completedFuture(null); + }); + + // Execute + blobStoreUtil.removeTTL("snapshotIndexBlobId", mockSnapshotIndex, metadata); + + // Assert + SortedSet expectedBlobIds = new TreeSet<>(); + // test uses unique file name (last char) as the blob ID. + expectedPresentFiles.forEach(f -> expectedBlobIds.add(f.substring(f.length() - 1))); + expectedBlobIds.add("snapshotIndexBlobId"); + + assertEquals(expectedBlobIds, allTTLRemoved); + } + + @Test + public void testPutFileChecksumAndMetadata() throws IOException, ExecutionException, InterruptedException { + // Setup + SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName); + Path path = Files.createTempFile("samza-testPutFileChecksum-", ".tmp"); + FileUtil fileUtil = new FileUtil(); + fileUtil.writeToTextFile(path.toFile(), RandomStringUtils.random(1000), false); + long expectedChecksum = FileUtils.checksumCRC32(path.toFile()); + + BlobStoreManager blobStoreManager = mock(BlobStoreManager.class); + ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(Metadata.class); + when(blobStoreManager.put(any(InputStream.class), argumentCaptor.capture())).thenAnswer( + (Answer>) invocation -> { + InputStream inputStream = invocation.getArgumentAt(0, InputStream.class); + // consume input stream to ensure checksum is calculated + IOUtils.copy(inputStream, NullOutputStream.NULL_OUTPUT_STREAM); + return CompletableFuture.completedFuture("blobId"); + }); + + BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null); + + CompletionStage fileIndexFuture = blobStoreUtil.putFile(path.toFile(), snapshotMetadata); + FileIndex fileIndex = null; + try { + // should be already complete. if not, future composition in putFile is broken. + fileIndex = fileIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + fail("Future returned from putFile should be already complete."); + } + + // Assert + Metadata metadata = (Metadata) argumentCaptor.getValue(); + assertEquals(path.toAbsolutePath().toString(), metadata.getPayloadPath()); + assertEquals(path.toFile().length(), Long.valueOf(metadata.getPayloadSize()).longValue()); + assertEquals(expectedChecksum, fileIndex.getChecksum()); + } + + @Test + public void testAreSameFile() throws IOException { + FileUtil fileUtil = new FileUtil(); + // 1. test with sst file with same attributes + Path sstFile = Files.createTempFile("samza-testAreSameFiles-", ".sst"); + + PosixFileAttributes sstFileAttribs = Files.readAttributes(sstFile, PosixFileAttributes.class); + FileMetadata sstFileMetadata = new FileMetadata(sstFileAttribs.creationTime().toMillis(), + sstFileAttribs.lastModifiedTime().toMillis(), sstFileAttribs.size(), sstFileAttribs.owner().toString(), + sstFileAttribs.group().toString(), PosixFilePermissions.toString(sstFileAttribs.permissions())); + // checksum should be ignored for sst file. Set any dummy value + FileIndex sstFileIndex = new FileIndex(sstFile.getFileName().toString(), Collections.emptyList(), sstFileMetadata, 0L); + + assertTrue(DirDiffUtil.areSameFile(false).test(sstFile.toFile(), sstFileIndex)); + + // 2. test with sst file with different timestamps + // Update last modified time + Files.setLastModifiedTime(sstFile, FileTime.fromMillis(System.currentTimeMillis() + 1000L)); + assertTrue(DirDiffUtil.areSameFile(false).test(sstFile.toFile(), sstFileIndex)); + + // 3. test with non-sst files with same metadata and content + Path tmpFile = Files.createTempFile("samza-testAreSameFiles-", ".tmp"); + fileUtil.writeToTextFile(tmpFile.toFile(), RandomStringUtils.random(1000), false); + + PosixFileAttributes tmpFileAttribs = Files.readAttributes(tmpFile, PosixFileAttributes.class); + FileMetadata tmpFileMetadata = + new FileMetadata(tmpFileAttribs.creationTime().toMillis(), tmpFileAttribs.lastModifiedTime().toMillis(), + tmpFileAttribs.size(), tmpFileAttribs.owner().toString(), tmpFileAttribs.group().toString(), + PosixFilePermissions.toString(tmpFileAttribs.permissions())); + FileIndex tmpFileIndex = new FileIndex(tmpFile.getFileName().toString(), Collections.emptyList(), tmpFileMetadata, + FileUtils.checksumCRC32(tmpFile.toFile())); + + assertTrue(DirDiffUtil.areSameFile(false).test(tmpFile.toFile(), tmpFileIndex)); + + // 4. test with non-sst files with different attributes + // change lastModifiedTime of local file + FileTime prevLastModified = tmpFileAttribs.lastModifiedTime(); + Files.setLastModifiedTime(tmpFile, FileTime.fromMillis(System.currentTimeMillis() + 1000L)); + assertTrue(DirDiffUtil.areSameFile(false).test(tmpFile.toFile(), tmpFileIndex)); + + // change content/checksum of local file + Files.setLastModifiedTime(tmpFile, prevLastModified); // reset attributes to match with remote file + fileUtil.writeToTextFile(tmpFile.toFile(), RandomStringUtils.random(1000), false); //new content + assertFalse(DirDiffUtil.areSameFile(false).test(tmpFile.toFile(), tmpFileIndex)); + } + + @Test + public void testRestoreDirRestoresMultiPartFilesCorrectly() throws IOException { + Path restoreDirBasePath = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX); + + // remote file == 26 blobs, blob ids from a to z, blob contents from a to z, offsets 0 to 25. + DirIndex mockDirIndex = mock(DirIndex.class); + when(mockDirIndex.getDirName()).thenReturn(DirIndex.ROOT_DIR_NAME); + FileIndex mockFileIndex = mock(FileIndex.class); + when(mockFileIndex.getFileName()).thenReturn("1.sst"); + + // setup mock file attributes. create a temp file to get current user/group/permissions so that they + // match with restored files. + File tmpFile = Paths.get(restoreDirBasePath.toString(), "tempfile-" + new Random().nextInt()).toFile(); + tmpFile.createNewFile(); + PosixFileAttributes attrs = Files.readAttributes(tmpFile.toPath(), PosixFileAttributes.class); + FileMetadata fileMetadata = new FileMetadata(1234L, 1243L, 26, // ctime mtime does not matter. size == 26 + attrs.owner().getName(), attrs.group().getName(), PosixFilePermissions.toString(attrs.permissions())); + when(mockFileIndex.getFileMetadata()).thenReturn(fileMetadata); + Files.delete(tmpFile.toPath()); // delete so that it doesn't show up in restored dir contents. + + List mockFileBlobs = new ArrayList<>(); + StringBuilder fileContents = new StringBuilder(); + for (int i = 0; i < 26; i++) { + FileBlob mockFileBlob = mock(FileBlob.class); + char c = (char) ('a' + i); + fileContents.append(c); // blob contents == blobId + when(mockFileBlob.getBlobId()).thenReturn(String.valueOf(c)); + when(mockFileBlob.getOffset()).thenReturn(i); + mockFileBlobs.add(mockFileBlob); + } + when(mockFileIndex.getBlobs()).thenReturn(mockFileBlobs); + CRC32 checksum = new CRC32(); + checksum.update(fileContents.toString().getBytes()); + when(mockFileIndex.getChecksum()).thenReturn(checksum.getValue()); + when(mockDirIndex.getFilesPresent()).thenReturn(ImmutableList.of(mockFileIndex)); + + BlobStoreManager mockBlobStoreManager = mock(BlobStoreManager.class); + when(mockBlobStoreManager.get(anyString(), any(OutputStream.class), any(Metadata.class))).thenAnswer( + (Answer>) invocationOnMock -> { + String blobId = invocationOnMock.getArgumentAt(0, String.class); + OutputStream outputStream = invocationOnMock.getArgumentAt(1, OutputStream.class); + // blob contents = blob id + outputStream.write(blobId.getBytes()); + + // force flush so that the checksum calculation later uses the full file contents. + ((FileOutputStream) outputStream).getFD().sync(); + return CompletableFuture.completedFuture(null); + }); + + BlobStoreUtil blobStoreUtil = new BlobStoreUtil(mockBlobStoreManager, EXECUTOR, null, null); + blobStoreUtil.restoreDir(restoreDirBasePath.toFile(), mockDirIndex, metadata).join(); + + assertTrue( + new DirDiffUtil().areSameDir(Collections.emptySet(), false).test(restoreDirBasePath.toFile(), mockDirIndex)); + } + + @Test + public void testRestoreDirRetriesFileRestoreOnRetriableExceptions() throws IOException { + Path restoreDirBasePath = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX); + + DirIndex mockDirIndex = mock(DirIndex.class); + when(mockDirIndex.getDirName()).thenReturn(DirIndex.ROOT_DIR_NAME); + FileIndex mockFileIndex = mock(FileIndex.class); + when(mockFileIndex.getFileName()).thenReturn("1.sst"); + + // setup mock file attributes. create a temp file to get current user/group/permissions so that they + // match with restored files. + File tmpFile = Paths.get(restoreDirBasePath.toString(), "tempfile-" + new Random().nextInt()).toFile(); + tmpFile.createNewFile(); + byte[] fileContents = "fileContents".getBytes(); + PosixFileAttributes attrs = Files.readAttributes(tmpFile.toPath(), PosixFileAttributes.class); + FileMetadata fileMetadata = + new FileMetadata(1234L, 1243L, fileContents.length, // ctime mtime does not matter. size == 26 + attrs.owner().getName(), attrs.group().getName(), PosixFilePermissions.toString(attrs.permissions())); + when(mockFileIndex.getFileMetadata()).thenReturn(fileMetadata); + Files.delete(tmpFile.toPath()); // delete so that it doesn't show up in restored dir contents. + + List mockFileBlobs = new ArrayList<>(); + FileBlob mockFileBlob = mock(FileBlob.class); + when(mockFileBlob.getBlobId()).thenReturn("fileBlobId"); + when(mockFileBlob.getOffset()).thenReturn(0); + mockFileBlobs.add(mockFileBlob); + when(mockFileIndex.getBlobs()).thenReturn(mockFileBlobs); + + CRC32 checksum = new CRC32(); + checksum.update(fileContents); + when(mockFileIndex.getChecksum()).thenReturn(checksum.getValue()); + when(mockDirIndex.getFilesPresent()).thenReturn(ImmutableList.of(mockFileIndex)); + + BlobStoreManager mockBlobStoreManager = mock(BlobStoreManager.class); + when(mockBlobStoreManager.get(anyString(), any(OutputStream.class), any(Metadata.class))).thenAnswer( + (Answer>) invocationOnMock -> { // first try, retriable error + String blobId = invocationOnMock.getArgumentAt(0, String.class); + OutputStream outputStream = invocationOnMock.getArgumentAt(1, OutputStream.class); + // write garbage data on first retry to verify that final file contents are correct + outputStream.write("bad-data".getBytes()); + ((FileOutputStream) outputStream).getFD().sync(); + return FutureUtil.failedFuture(new RetriableException()); // retriable error + }).thenAnswer((Answer>) invocationOnMock -> { // 2nd try + String blobId = invocationOnMock.getArgumentAt(0, String.class); + OutputStream outputStream = invocationOnMock.getArgumentAt(1, OutputStream.class); + // write correct data on first retry to verify that final file contents are correct + outputStream.write(fileContents); + ((FileOutputStream) outputStream).getFD().sync(); + return CompletableFuture.completedFuture(null); // success + }); + + BlobStoreUtil blobStoreUtil = new BlobStoreUtil(mockBlobStoreManager, EXECUTOR, null, null); + blobStoreUtil.restoreDir(restoreDirBasePath.toFile(), mockDirIndex, metadata).join(); + + assertTrue( + new DirDiffUtil().areSameDir(Collections.emptySet(), false).test(restoreDirBasePath.toFile(), mockDirIndex)); + } + + @Test + public void testRestoreDirFailsRestoreOnNonRetriableExceptions() throws IOException { + Path restoreDirBasePath = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX); + + DirIndex mockDirIndex = mock(DirIndex.class); + when(mockDirIndex.getDirName()).thenReturn(DirIndex.ROOT_DIR_NAME); + FileIndex mockFileIndex = mock(FileIndex.class); + when(mockFileIndex.getFileName()).thenReturn("1.sst"); + + // setup mock file attributes. create a temp file to get current user/group/permissions so that they + // match with restored files. + File tmpFile = Paths.get(restoreDirBasePath.toString(), "tempfile-" + new Random().nextInt()).toFile(); + tmpFile.createNewFile(); + byte[] fileContents = "fileContents".getBytes(); + PosixFileAttributes attrs = Files.readAttributes(tmpFile.toPath(), PosixFileAttributes.class); + FileMetadata fileMetadata = + new FileMetadata(1234L, 1243L, fileContents.length, // ctime mtime does not matter. size == 26 + attrs.owner().getName(), attrs.group().getName(), PosixFilePermissions.toString(attrs.permissions())); + when(mockFileIndex.getFileMetadata()).thenReturn(fileMetadata); + Files.delete(tmpFile.toPath()); // delete so that it doesn't show up in restored dir contents. + + List mockFileBlobs = new ArrayList<>(); + FileBlob mockFileBlob = mock(FileBlob.class); + when(mockFileBlob.getBlobId()).thenReturn("fileBlobId"); + when(mockFileBlob.getOffset()).thenReturn(0); + mockFileBlobs.add(mockFileBlob); + when(mockFileIndex.getBlobs()).thenReturn(mockFileBlobs); + + CRC32 checksum = new CRC32(); + checksum.update(fileContents); + when(mockFileIndex.getChecksum()).thenReturn(checksum.getValue()); + when(mockDirIndex.getFilesPresent()).thenReturn(ImmutableList.of(mockFileIndex)); + + BlobStoreManager mockBlobStoreManager = mock(BlobStoreManager.class); + when(mockBlobStoreManager.get(anyString(), any(OutputStream.class), any(Metadata.class))).thenReturn( + FutureUtil.failedFuture(new IllegalArgumentException())) // non retriable error + .thenAnswer((Answer>) invocationOnMock -> { + String blobId = invocationOnMock.getArgumentAt(0, String.class); + OutputStream outputStream = invocationOnMock.getArgumentAt(1, OutputStream.class); + outputStream.write(fileContents); + + // force flush so that the checksum calculation later uses the full file contents. + ((FileOutputStream) outputStream).getFD().sync(); + return CompletableFuture.completedFuture(null); + }); + + BlobStoreUtil blobStoreUtil = new BlobStoreUtil(mockBlobStoreManager, EXECUTOR, null, null); + try { + blobStoreUtil.restoreDir(restoreDirBasePath.toFile(), mockDirIndex, metadata).join(); + fail("Should have failed on non-retriable errors during file restore"); + } catch (CompletionException e) { + assertTrue(e.getCause() instanceof IllegalArgumentException); + } + } + + @Test + @Ignore // TODO remove + public void testRestoreDirRecreatesEmptyFilesAndDirs() throws IOException { + String prevSnapshotFiles = "[a, b, z/1, y/1, p/m/1, q/n/1]"; + DirIndex dirIndex = BlobStoreTestUtil.createDirIndex(prevSnapshotFiles); + String localSnapshotFiles = "[a, b, z/1, y/1, p/m/1, q/n/1]"; + Path localSnapshot = BlobStoreTestUtil.createLocalDir(localSnapshotFiles); + BlobStoreManager mockBlobStoreManager = mock(BlobStoreManager.class); + when(mockBlobStoreManager.get(anyString(), any(OutputStream.class), any(Metadata.class))).thenAnswer( + (Answer>) invocationOnMock -> { + String blobId = invocationOnMock.getArgumentAt(0, String.class); + OutputStream outputStream = invocationOnMock.getArgumentAt(1, OutputStream.class); + // blob contents = blob id + outputStream.write(blobId.getBytes()); + return CompletableFuture.completedFuture(null); + }); + boolean result = new DirDiffUtil().areSameDir(new TreeSet<>(), false).test(localSnapshot.toFile(), dirIndex); + assertFalse(result); + //ToDo complete + } + + @Test + public void testRestoreDirVerifiesFileChecksums() { + // ToDo shesharma restore dir only restores SST files. Since other metadata files are in ignore list, + // no checksum matching would be done? Check later. + } + + @Test + public void testRestoreDirCreatesCorrectDirectoryStructure() throws IOException { + String prevSnapshotFiles = "[a, b, z/1, y/1, p/m/1, q/n/1]"; + DirIndex dirIndex = BlobStoreTestUtil.createDirIndex(prevSnapshotFiles); + + BlobStoreManager mockBlobStoreManager = mock(BlobStoreManager.class); + when(mockBlobStoreManager.get(anyString(), any(OutputStream.class), any(Metadata.class))).thenAnswer( + (Answer>) invocationOnMock -> { + String blobId = invocationOnMock.getArgumentAt(0, String.class); + OutputStream outputStream = invocationOnMock.getArgumentAt(1, OutputStream.class); + // blob contents = blob id + outputStream.write(blobId.getBytes()); + return CompletableFuture.completedFuture(null); + }); + + Path restoreDirBasePath = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX); + BlobStoreUtil blobStoreUtil = new BlobStoreUtil(mockBlobStoreManager, EXECUTOR, null, null); + blobStoreUtil.restoreDir(restoreDirBasePath.toFile(), dirIndex, metadata).join(); + + assertTrue(new DirDiffUtil().areSameDir(Collections.emptySet(), false).test(restoreDirBasePath.toFile(), dirIndex)); + } + + /** + * Tests related to {@link BlobStoreUtil#getStoreSnapshotIndexes} + */ + + @Test + public void testGetSSIReturnsEmptyMapForNullCheckpoint() { + BlobStoreUtil blobStoreUtil = + new BlobStoreUtil(mock(BlobStoreManager.class), MoreExecutors.newDirectExecutorService(), null, null); + Map> snapshotIndexes = + blobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", null); + assertTrue(snapshotIndexes.isEmpty()); + } + + @Test(expected = SamzaException.class) + public void testGetSSIThrowsExceptionForCheckpointV1() { + Checkpoint mockCheckpoint = mock(Checkpoint.class); + when(mockCheckpoint.getVersion()).thenReturn((short) 1); + BlobStoreUtil blobStoreUtil = + new BlobStoreUtil(mock(BlobStoreManager.class), MoreExecutors.newDirectExecutorService(), null, null); + blobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", mockCheckpoint); + } + + @Test + public void testGetSSIReturnsEmptyMapIfNoEntryForBlobStoreBackendFactory() { + CheckpointV2 mockCheckpoint = mock(CheckpointV2.class); + when(mockCheckpoint.getVersion()).thenReturn((short) 2); + when(mockCheckpoint.getStateCheckpointMarkers()).thenReturn( + ImmutableMap.of("com.OtherStateBackendFactory", ImmutableMap.of("storeName", "otherSCM"))); + + BlobStoreUtil blobStoreUtil = + new BlobStoreUtil(mock(BlobStoreManager.class), MoreExecutors.newDirectExecutorService(), null, null); + Map> snapshotIndexes = + blobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", mockCheckpoint); + assertTrue(snapshotIndexes.isEmpty()); + } + + @Test + public void testGetSSIReturnsEmptyMapIfNoStoreForBlobStoreBackendFactory() { + CheckpointV2 mockCheckpoint = mock(CheckpointV2.class); + when(mockCheckpoint.getVersion()).thenReturn((short) 2); + when(mockCheckpoint.getStateCheckpointMarkers()).thenReturn( + ImmutableMap.of(BlobStoreStateBackendFactory.class.getName(), ImmutableMap.of())); + + BlobStoreUtil blobStoreUtil = + new BlobStoreUtil(mock(BlobStoreManager.class), MoreExecutors.newDirectExecutorService(), null, null); + Map> snapshotIndexes = + blobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", mockCheckpoint); + assertTrue(snapshotIndexes.isEmpty()); + } + + @Test(expected = SamzaException.class) + public void testGetSSIThrowsExceptionOnSyncBlobStoreErrors() { + Checkpoint checkpoint = createCheckpointV2(BlobStoreStateBackendFactory.class.getName(), + ImmutableMap.of("storeName", "snapshotIndexBlobId")); + BlobStoreUtil mockBlobStoreUtil = mock(BlobStoreUtil.class); + when(mockBlobStoreUtil.getSnapshotIndex(anyString(), any(Metadata.class))).thenThrow(new RuntimeException()); + when(mockBlobStoreUtil.getStoreSnapshotIndexes(anyString(), anyString(), anyString(), + any(Checkpoint.class))).thenCallRealMethod(); + mockBlobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", checkpoint); + } + + @Test + public void testGetSSISkipsStoresWithSnapshotIndexAlreadyDeleted() { + Checkpoint checkpoint = createCheckpointV2(BlobStoreStateBackendFactory.class.getName(), + ImmutableMap.of("storeName1", "snapshotIndexBlobId1", "storeName2", "snapshotIndexBlobId2")); + SnapshotIndex store1SnapshotIndex = mock(SnapshotIndex.class); + BlobStoreUtil mockBlobStoreUtil = mock(BlobStoreUtil.class); + + CompletableFuture failedFuture = FutureUtil.failedFuture(new DeletedException()); + when(mockBlobStoreUtil.getSnapshotIndex(eq("snapshotIndexBlobId1"), any(Metadata.class))).thenReturn( + CompletableFuture.completedFuture(store1SnapshotIndex)); + when(mockBlobStoreUtil.getSnapshotIndex(eq("snapshotIndexBlobId2"), any(Metadata.class))).thenReturn(failedFuture); + when(mockBlobStoreUtil.getStoreSnapshotIndexes(anyString(), anyString(), anyString(), + any(Checkpoint.class))).thenCallRealMethod(); + + Map> snapshotIndexes = + mockBlobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", checkpoint); + assertEquals(1, snapshotIndexes.size()); + assertEquals("snapshotIndexBlobId1", snapshotIndexes.get("storeName1").getLeft()); + assertEquals(store1SnapshotIndex, snapshotIndexes.get("storeName1").getRight()); + } + + @Test + public void testGetSSIThrowsExceptionIfAnyNonIgnoredAsyncBlobStoreErrors() { + Checkpoint checkpoint = createCheckpointV2(BlobStoreStateBackendFactory.class.getName(), + ImmutableMap.of("storeName1", "snapshotIndexBlobId1", "storeName2", "snapshotIndexBlobId2")); + SnapshotIndex store1SnapshotIndex = mock(SnapshotIndex.class); + BlobStoreUtil mockBlobStoreUtil = mock(BlobStoreUtil.class); + when(mockBlobStoreUtil.getStoreSnapshotIndexes(anyString(), anyString(), anyString(), + any(Checkpoint.class))).thenCallRealMethod(); + RuntimeException nonIgnoredException = new RuntimeException(); + CompletableFuture failedFuture = FutureUtil.failedFuture(nonIgnoredException); + when(mockBlobStoreUtil.getSnapshotIndex(eq("snapshotIndexBlobId1"), any(Metadata.class))).thenReturn( + FutureUtil.failedFuture(new DeletedException())); // should fail even if some errors are ignored + when(mockBlobStoreUtil.getSnapshotIndex(eq("snapshotIndexBlobId2"), any(Metadata.class))).thenReturn(failedFuture); + + try { + mockBlobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", checkpoint); + fail("Should have thrown an exception"); + } catch (Exception e) { + Throwable cause = + FutureUtil.unwrapExceptions(CompletionException.class, FutureUtil.unwrapExceptions(SamzaException.class, e)); + assertEquals(nonIgnoredException, cause); + } + } + + @Test + public void testGetSSIReturnsCorrectSCMSnapshotIndexPair() { + String storeName = "storeName"; + String otherStoreName = "otherStoreName"; + String storeSnapshotIndexBlobId = "snapshotIndexBlobId"; + String otherStoreSnapshotIndexBlobId = "otherSnapshotIndexBlobId"; + SnapshotIndex mockStoreSnapshotIndex = mock(SnapshotIndex.class); + SnapshotIndex mockOtherStooreSnapshotIndex = mock(SnapshotIndex.class); + + CheckpointV2 checkpoint = createCheckpointV2(BlobStoreStateBackendFactory.class.getName(), + ImmutableMap.of(storeName, storeSnapshotIndexBlobId, otherStoreName, otherStoreSnapshotIndexBlobId)); + + BlobStoreUtil mockBlobStoreUtil = mock(BlobStoreUtil.class); + + when(mockBlobStoreUtil.getSnapshotIndex(eq(storeSnapshotIndexBlobId), any(Metadata.class))).thenReturn( + CompletableFuture.completedFuture(mockStoreSnapshotIndex)); + when(mockBlobStoreUtil.getSnapshotIndex(eq(otherStoreSnapshotIndexBlobId), any(Metadata.class))).thenReturn( + CompletableFuture.completedFuture(mockOtherStooreSnapshotIndex)); + when(mockBlobStoreUtil.getStoreSnapshotIndexes(anyString(), anyString(), anyString(), + any(Checkpoint.class))).thenCallRealMethod(); + + Map> snapshotIndexes = + mockBlobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", checkpoint); + + assertEquals(storeSnapshotIndexBlobId, snapshotIndexes.get(storeName).getKey()); + assertEquals(mockStoreSnapshotIndex, snapshotIndexes.get(storeName).getValue()); + assertEquals(otherStoreSnapshotIndexBlobId, snapshotIndexes.get(otherStoreName).getKey()); + assertEquals(mockOtherStooreSnapshotIndex, snapshotIndexes.get(otherStoreName).getValue()); + verify(mockBlobStoreUtil, times(2)).getSnapshotIndex(anyString(), any(Metadata.class)); + } + + private CheckpointV2 createCheckpointV2(String stateBackendFactory, Map storeSnapshotIndexBlobIds) { + CheckpointId checkpointId = CheckpointId.create(); + Map> factoryStoreSCMs = new HashMap<>(); + Map storeSCMs = new HashMap<>(); + for (Map.Entry entry : storeSnapshotIndexBlobIds.entrySet()) { + storeSCMs.put(entry.getKey(), entry.getValue()); + } + + factoryStoreSCMs.put(stateBackendFactory, storeSCMs); + return new CheckpointV2(checkpointId, ImmutableMap.of(), factoryStoreSCMs); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtil.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtil.java new file mode 100644 index 0000000000..fa08670426 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtil.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.storage.blobstore.util; + +import org.apache.samza.storage.blobstore.diff.DirDiff; +import org.apache.samza.storage.blobstore.index.DirIndex; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collection; +import java.util.SortedSet; +import java.util.TreeSet; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.junit.Assert.assertEquals; + + +@RunWith(value = Parameterized.class) +public class TestDirDiffUtil { + + private final String local; + private final String remote; + private final String expectedAdded; + private final String expectedRetained; + private final String expectedRemoved; + private final String description; + + public TestDirDiffUtil(String local, String remote, + String expectedAdded, String expectedRetained, String expectedRemoved, + String description) { + this.local = local; + this.remote = remote; + this.expectedAdded = expectedAdded; + this.expectedRetained = expectedRetained; + this.expectedRemoved = expectedRemoved; + this.description = description; + } + + // TODO HIGH shesharm test with empty subdirectories + @Parameterized.Parameters(name = "testGetDirDiff: {5}") + public static Collection data() { + return Arrays.asList(new Object[][]{ + // Local Remote Expected Added Expected Retained Expected Removed Description + // -------------------------------- -------------------------------- ----------------- ------------------ ----------------- ------------------------------------------------------------------ + {"[]", "[]", "[]", "[]", "[]", "Nothing in local or remote"}, + {"[a]", "[]", "[a]", "[]", "[]", "New file in local"}, + {"[z/1]", "[]", "[z/1]", "[]", "[]", "New dir in local"}, + {"[z/i/1]", "[]", "[z/i/1]", "[]", "[]", "New recursive dir in local"}, + {"[a, z/1]", "[]", "[a, z/1]", "[]", "[]", "New file and dir in local"}, + {"[a, z/1, y/j/1]", "[]", "[a, z/1, y/j/1]", "[]", "[]", "New file, dir and recursive dir in local"}, + {"[a]", "[a]", "[]", "[a]", "[]", "File retained in local"}, + {"[z/1]", "[z/1]", "[]", "[z/1]", "[]", "Dir retained in local"}, + {"[z/i/1]", "[z/i/1]", "[]", "[z/i/1]", "[]", "Recursive dir retained in local"}, + {"[a, z/1]", "[a, z/1]", "[]", "[a, z/1]", "[]", "File and dir retained in local"}, + {"[a, z/1, y/j/1]", "[a, z/1, y/j/1]", "[]", "[a, z/1, y/j/1]", "[]", "File, dir and recursive dir retained in local"}, + {"[]", "[a]", "[]", "[]", "[a]", "File removed in local"}, + {"[]", "[z/1]", "[]", "[]", "[z/1]", "Dir removed in local"}, + {"[]", "[z/i/1]", "[]", "[]", "[z/i/1]", "Recursive dir removed in local"}, + {"[]", "[a, z/1]", "[]", "[]", "[a, z/1]", "File and dir removed in local"}, + {"[]", "[a, z/1, y/j/1]", "[]", "[]", "[a, z/1, y/j/1]", "File, dir and recursive dir removed in local"}, + {"[b]", "[a]", "[b]", "[]", "[a]", "File added and removed in local"}, + {"[y/1]", "[z/1]", "[y/1]", "[]", "[z/1]", "Dir added and removed in local"}, + {"[y/j/1]", "[z/i/1]", "[y/j/1]", "[]", "[z/i/1]", "Recursive dir added and removed in local"}, + {"[b, y/1]", "[a, z/1]", "[b, y/1]", "[]", "[a, z/1]", "File and dir added and removed in local"}, + {"[b, y/1, x/k/1]", "[a, z/1, w/m/1]", "[b, y/1, x/k/1]", "[]", "[a, z/1, w/m/1]", "File, dir and recursive dir added and removed in local"}, + {"[a, c]", "[a]", "[c]", "[a]", "[]", "File added and retained in local"}, + {"[z/1, y/1]", "[z/1]", "[y/1]", "[z/1]", "[]", "Dir added and retained in local"}, + {"[z/i/1, y/j/1]", "[z/i/1]", "[y/j/1]", "[z/i/1]", "[]", "Recursive dir added and retained in local"}, + {"[a, c, z/1, y/1]", "[a, z/1]", "[c, y/1]", "[a, z/1]", "[]", "File and dir added and retained in local"}, + {"[a, c, z/1, y/1, p/m/1, q/n/1]", "[a, z/1, p/m/1]", "[c, y/1, q/n/1]", "[a, z/1, p/m/1]", "[]", "File, dir and recursive dir added and retained in local"}, + {"[a, c]", "[a, b]", "[c]", "[a]", "[b]", "File added, retained and removed in local"}, + {"[z/1, y/1]", "[z/1, x/1]", "[y/1]", "[z/1]", "[x/1]", "Dir added, retained and removed in local"}, + {"[z/1, z/3]", "[z/1, z/2]", "[z/3]", "[z/1]", "[z/2]", "File added, retained and removed in dir in local"}, + {"[z/i/1, y/j/1]", "[z/i/1, x/k/1]", "[y/j/1]", "[z/i/1]", "[x/k/1]", "Recursive dir added, retained and removed in local"}, + {"[a, c, z/1, y/1]", "[a, b, z/1, x/1]", "[c, y/1]", "[a, z/1]", "[b, x/1]", "File and dir added, retained and removed in local"}, + {"[a, c, z/1, y/1, p/m/1, q/n/1]", "[a, b, z/1, x/1, p/m/1, r/o/1]", "[c, y/1, q/n/1]", "[a, z/1, p/m/1]", "[b, x/1, r/o/1]", "File, dir and recursive dir added, retained and removed in local"}, + {"[a, c, z/1, p/m/1, p/m/2, q/n/1]", "[a, b, z/1, x/1, p/m/1, r/o/1]", "[c, p/m/2, q/n/1]", "[a, z/1, p/m/1]", "[b, x/1, r/o/1]", "File, File in recursive subdir, dir and recursive dir added, retained and removed in local"} + }); + } + + @Test + public void testGetDirDiff() throws IOException { + // Setup + Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(this.local); + String basePath = localSnapshotDir.toAbsolutePath().toString(); + DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(this.remote); + + // Execute + DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir, + (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName())); + + SortedSet allAdded = new TreeSet<>(); + SortedSet allRemoved = new TreeSet<>(); + SortedSet allRetained = new TreeSet<>(); + BlobStoreTestUtil.getAllAddedInDiff(basePath, dirDiff, allAdded); + BlobStoreTestUtil.getAllRemovedInDiff("", dirDiff, allRemoved); + BlobStoreTestUtil.getAllRetainedInDiff("", dirDiff, allRetained); + + // Assert + SortedSet expectedAddedFiles = BlobStoreTestUtil.getExpected(this.expectedAdded); + SortedSet expectedRetainedFiles = BlobStoreTestUtil.getExpected(this.expectedRetained); + SortedSet expectedRemovedFiles = BlobStoreTestUtil.getExpected(this.expectedRemoved); + assertEquals(expectedAddedFiles, allAdded); + assertEquals(expectedRetainedFiles, allRetained); + assertEquals(expectedRemovedFiles, allRemoved); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtilMisc.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtilMisc.java new file mode 100644 index 0000000000..e61297bf4b --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtilMisc.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.storage.blobstore.util; + +import org.apache.samza.storage.blobstore.diff.DirDiff; +import org.apache.samza.storage.blobstore.index.DirIndex; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.SortedSet; +import java.util.TreeSet; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class TestDirDiffUtilMisc { + + + /** + * Test the case when a file has been modified locally. I.e., when isSameFile returns false for a local file with + * the same name as the remote file. The file should be marked for both deletion and addition. + */ + @Test + public void testGetDirDiffWhenIsSameFileReturnsFalseForSameFileName() throws IOException { + String local = "[a]"; + String remote = "[a]"; + String expectedAdded = "[a]"; + String expectedRetained = "[]"; + String expectedRemoved = "[a]"; + + Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local); + String basePath = localSnapshotDir.toAbsolutePath().toString(); + DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote); + + // Execute + DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir, + (localFile, remoteFile) -> false); + + SortedSet allAdded = new TreeSet<>(); + SortedSet allRemoved = new TreeSet<>(); + SortedSet allRetained = new TreeSet<>(); + BlobStoreTestUtil.getAllAddedInDiff(basePath, dirDiff, allAdded); + BlobStoreTestUtil.getAllRemovedInDiff("", dirDiff, allRemoved); + BlobStoreTestUtil.getAllRetainedInDiff("", dirDiff, allRetained); + + // Assert + SortedSet expectedAddedFiles = BlobStoreTestUtil.getExpected(expectedAdded); + SortedSet expectedRetainedFiles = BlobStoreTestUtil.getExpected(expectedRetained); + SortedSet expectedRemovedFiles = BlobStoreTestUtil.getExpected(expectedRemoved); + assertEquals(expectedAddedFiles, allAdded); + assertEquals(expectedRetainedFiles, allRetained); + assertEquals(expectedRemovedFiles, allRemoved); + } +} diff --git a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbOptionsHelper.java b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbOptionsHelper.java index b7c47e1779..0908a977d1 100644 --- a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbOptionsHelper.java +++ b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbOptionsHelper.java @@ -20,13 +20,17 @@ package org.apache.samza.storage.kv; import java.io.File; +import org.apache.commons.lang3.StringUtils; import org.apache.samza.config.Config; import org.apache.samza.storage.StorageEngineFactory; import org.apache.samza.storage.StorageManagerUtil; import org.rocksdb.BlockBasedTableConfig; +import org.rocksdb.CompactionOptionsUniversal; +import org.rocksdb.CompactionStopStyle; import org.rocksdb.CompactionStyle; import org.rocksdb.CompressionType; import org.rocksdb.Options; +import org.rocksdb.WALRecoveryMode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,9 +40,30 @@ public class RocksDbOptionsHelper { private static final Logger log = LoggerFactory.getLogger(RocksDbOptionsHelper.class); + // TODO HIGH pmaheshw Add these to RockdDBTableDescriptor + public static final String ROCKSDB_WAL_ENABLED = "rocksdb.wal.enabled"; private static final String ROCKSDB_COMPRESSION = "rocksdb.compression"; private static final String ROCKSDB_BLOCK_SIZE_BYTES = "rocksdb.block.size.bytes"; + + private static final String ROCKSDB_COMPACTION_NUM_LEVELS = "rocksdb.compaction.num.levels"; + private static final String ROCKSDB_COMPACTION_LEVEL0_FILE_NUM_COMPACTION_TRIGGER = + "rocksdb.compaction.level0.file.num.compaction.trigger"; + private static final String ROCKSDB_COMPACTION_MAX_BACKGROUND_COMPACTIONS = "rocksdb.compaction.max.background.compactions"; + private static final String ROCKSDB_COMPACTION_TARGET_FILE_SIZE_BASE = "rocksdb.compaction.target.file.size.base"; + private static final String ROCKSDB_COMPACTION_TARGET_FILE_SIZE_MULTIPLIER = "rocksdb.compaction.target.file.size.multiplier"; + private static final String ROCKSDB_COMPACTION_STYLE = "rocksdb.compaction.style"; + private static final String ROCKSDB_COMPACTION_UNIVERSAL_MAX_SIZE_AMPLIFICATION_PERCENT = + "rocksdb.compaction.universal.max.size.amplification.percent"; + private static final String ROCKSDB_COMPACTION_UNIVERSAL_SIZE_RATIO = + "rocksdb.compaction.universal.size.ratio"; + private static final String ROCKSDB_COMPACTION_UNIVERSAL_MIN_MERGE_WIDTH = + "rocksdb.compaction.universal.min.merge.width"; + private static final String ROCKSDB_COMPACTION_UNIVERSAL_MAX_MERGE_WIDTH = + "rocksdb.compaction.universal.max.merge.width"; + private static final String ROCKSDB_COMPACTION_UNIVERSAL_COMPACTION_STOP_STYLE = + "rocksdb.compaction.universal.compaction.stop.style"; + private static final String ROCKSDB_NUM_WRITE_BUFFERS = "rocksdb.num.write.buffers"; private static final String ROCKSDB_MAX_LOG_FILE_SIZE_BYTES = "rocksdb.max.log.file.size.bytes"; private static final String ROCKSDB_KEEP_LOG_FILE_NUM = "rocksdb.keep.log.file.num"; @@ -47,6 +72,12 @@ public class RocksDbOptionsHelper { public static Options options(Config storeConfig, int numTasksForContainer, File storeDir, StorageEngineFactory.StoreMode storeMode) { Options options = new Options(); + + if (storeConfig.getBoolean(ROCKSDB_WAL_ENABLED, false)) { + options.setManualWalFlush(true); // store.flush() will flushWAL(sync = true) instead + options.setWalRecoveryMode(WALRecoveryMode.AbsoluteConsistency); + } + Long writeBufSize = storeConfig.getLong("container.write.buffer.size.bytes", 32 * 1024 * 1024); // Cache size and write buffer size are specified on a per-container basis. options.setWriteBufferSize((int) (writeBufSize / numTasksForContainer)); @@ -84,23 +115,7 @@ public static Options options(Config storeConfig, int numTasksForContainer, File tableOptions.setBlockCacheSize(blockCacheSize).setBlockSize(blockSize); options.setTableFormatConfig(tableOptions); - CompactionStyle compactionStyle = CompactionStyle.UNIVERSAL; - String compactionStyleInConfig = storeConfig.get(ROCKSDB_COMPACTION_STYLE, "universal"); - switch (compactionStyleInConfig) { - case "universal": - compactionStyle = CompactionStyle.UNIVERSAL; - break; - case "fifo": - compactionStyle = CompactionStyle.FIFO; - break; - case "level": - compactionStyle = CompactionStyle.LEVEL; - break; - default: - log.warn("Unknown rocksdb.compaction.style " + compactionStyleInConfig + - ", overwriting to " + compactionStyle.name()); - } - options.setCompactionStyle(compactionStyle); + setCompactionOptions(storeConfig, options); options.setMaxWriteBufferNumber(storeConfig.getInt(ROCKSDB_NUM_WRITE_BUFFERS, 3)); options.setCreateIfMissing(true); @@ -125,6 +140,81 @@ public static Options options(Config storeConfig, int numTasksForContainer, File return options; } + private static void setCompactionOptions(Config storeConfig, Options options) { + if (storeConfig.containsKey(ROCKSDB_COMPACTION_NUM_LEVELS)) { + options.setNumLevels(storeConfig.getInt(ROCKSDB_COMPACTION_NUM_LEVELS)); + } + + if (storeConfig.containsKey(ROCKSDB_COMPACTION_LEVEL0_FILE_NUM_COMPACTION_TRIGGER)) { + int level0FileNumCompactionTrigger = storeConfig.getInt(ROCKSDB_COMPACTION_LEVEL0_FILE_NUM_COMPACTION_TRIGGER); + options.setLevel0FileNumCompactionTrigger(level0FileNumCompactionTrigger); + } + + if (storeConfig.containsKey(ROCKSDB_COMPACTION_MAX_BACKGROUND_COMPACTIONS)) { + options.setMaxBackgroundCompactions(storeConfig.getInt(ROCKSDB_COMPACTION_MAX_BACKGROUND_COMPACTIONS)); + } + + if (storeConfig.containsKey(ROCKSDB_COMPACTION_TARGET_FILE_SIZE_BASE)) { + options.setTargetFileSizeBase(storeConfig.getLong(ROCKSDB_COMPACTION_TARGET_FILE_SIZE_BASE)); + } + + if (storeConfig.containsKey(ROCKSDB_COMPACTION_TARGET_FILE_SIZE_MULTIPLIER)) { + options.setTargetFileSizeBase(storeConfig.getLong(ROCKSDB_COMPACTION_TARGET_FILE_SIZE_MULTIPLIER)); + } + + CompactionStyle compactionStyle = CompactionStyle.UNIVERSAL; + String compactionStyleInConfig = storeConfig.get(ROCKSDB_COMPACTION_STYLE, "universal"); + switch (compactionStyleInConfig) { + case "universal": + compactionStyle = CompactionStyle.UNIVERSAL; + break; + case "fifo": + compactionStyle = CompactionStyle.FIFO; + break; + case "level": + compactionStyle = CompactionStyle.LEVEL; + break; + default: + log.warn("Unknown rocksdb.compaction.style " + compactionStyleInConfig + + ", overwriting to " + compactionStyle.name()); + } + options.setCompactionStyle(compactionStyle); + + // Universal compaction options + if (compactionStyle.equals(CompactionStyle.UNIVERSAL)) { + CompactionOptionsUniversal compactionOptions = new CompactionOptionsUniversal(); + + if (storeConfig.containsKey(ROCKSDB_COMPACTION_UNIVERSAL_MAX_SIZE_AMPLIFICATION_PERCENT)) { + int val = storeConfig.getInt(ROCKSDB_COMPACTION_UNIVERSAL_MAX_SIZE_AMPLIFICATION_PERCENT); + compactionOptions.setMaxSizeAmplificationPercent(val); + } + + if (storeConfig.containsKey(ROCKSDB_COMPACTION_UNIVERSAL_SIZE_RATIO)) { + int val = storeConfig.getInt(ROCKSDB_COMPACTION_UNIVERSAL_SIZE_RATIO); + compactionOptions.setSizeRatio(val); + } + + if (storeConfig.containsKey(ROCKSDB_COMPACTION_UNIVERSAL_MIN_MERGE_WIDTH)) { + int val = storeConfig.getInt(ROCKSDB_COMPACTION_UNIVERSAL_MIN_MERGE_WIDTH); + compactionOptions.setMinMergeWidth(val); + } + + if (storeConfig.containsKey(ROCKSDB_COMPACTION_UNIVERSAL_MAX_MERGE_WIDTH)) { + int val = storeConfig.getInt(ROCKSDB_COMPACTION_UNIVERSAL_MAX_MERGE_WIDTH); + compactionOptions.setMaxMergeWidth(val); + } + + if (storeConfig.containsKey(ROCKSDB_COMPACTION_UNIVERSAL_COMPACTION_STOP_STYLE)) { + String stopStyle = storeConfig.get(ROCKSDB_COMPACTION_UNIVERSAL_COMPACTION_STOP_STYLE); + if (StringUtils.isNotBlank(stopStyle)) { + compactionOptions.setStopStyle(CompactionStopStyle.valueOf(stopStyle)); + } + } + + options.setCompactionOptionsUniversal(compactionOptions); + } + } + public static Long getBlockCacheSize(Config storeConfig, int numTasksForContainer) { long cacheSize = storeConfig.getLong("container.cache.size.bytes", 100 * 1024 * 1024L); return cacheSize / numTasksForContainer; diff --git a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala index e1f44d218f..afc289ad75 100644 --- a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala +++ b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala @@ -50,7 +50,13 @@ class RocksDbKeyValueStorageEngineFactory [K, V] extends BaseKeyValueStorageEngi () => RocksDbOptionsHelper.getBlockCacheSize(storageConfigSubset, numTasksForContainer)) val rocksDbOptions = RocksDbOptionsHelper.options(storageConfigSubset, numTasksForContainer, storeDir, storeMode) - val rocksDbWriteOptions = new WriteOptions().setDisableWAL(true) + val rocksDbWriteOptions = new WriteOptions() + + if (!storageConfigSubset.getBoolean(RocksDbOptionsHelper.ROCKSDB_WAL_ENABLED, false)) { + // if WAL not enabled, explicitly disable it + rocksDbWriteOptions.setDisableWAL(true) + } + val rocksDbFlushOptions = new FlushOptions().setWaitForFlush(true) val rocksDb = new RocksDbKeyValueStore( storeDir, diff --git a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala index 9d5ddfd6d2..6f5a7f2f8f 100644 --- a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala +++ b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala @@ -19,6 +19,8 @@ package org.apache.samza.storage.kv +import com.google.common.annotations.VisibleForTesting + import java.io.File import java.nio.file.{Path, Paths} import java.util.concurrent.TimeUnit @@ -125,7 +127,7 @@ class RocksDbKeyValueStore( // lazy val here is important because the store directories do not exist yet, it can only be opened // after the directories are created, which happens much later from now. - private lazy val db = RocksDbKeyValueStore.openDB(dir, options, storeConfig, isLoggedStore, storeName, metrics) + @VisibleForTesting lazy val db = RocksDbKeyValueStore.openDB(dir, options, storeConfig, isLoggedStore, storeName, metrics) private val lexicographic = new LexicographicComparator() /** @@ -236,13 +238,17 @@ class RocksDbKeyValueStore( def flush(): Unit = ifOpen { metrics.flushes.inc trace("Flushing store: %s" format storeName) - db.flush(flushOptions) + if (storeConfig.getBoolean(RocksDbOptionsHelper.ROCKSDB_WAL_ENABLED, false)) { + db.flushWal(true) + } else { + db.flush(flushOptions) + } trace("Flushed store: %s" format storeName) } override def checkpoint(id: CheckpointId): Optional[Path] = { val checkpoint = Checkpoint.create(db) - val checkpointPath = StorageManagerUtil.getCheckpointDirPath(dir, id) + val checkpointPath = new StorageManagerUtil().getStoreCheckpointDir(dir, id) checkpoint.createCheckpoint(checkpointPath) Optional.of(Paths.get(checkpointPath)) } diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java index 704e0cbd8a..d5059b341c 100644 --- a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java @@ -106,7 +106,7 @@ public StorageEngine getStorageEngine(String storeName, storePropertiesBuilder.setPersistedToDisk(true); } // The store is durable iff it is backed by the task backup manager - List storeBackupManager = storageConfig.getStoreBackupManagerClassName(storeName); + List storeBackupManager = storageConfig.getStoreBackupFactory(storeName); storePropertiesBuilder.setIsDurable(!storeBackupManager.isEmpty()); int batchSize = storageConfigSubset.getInt(WRITE_BATCH_SIZE, DEFAULT_WRITE_BATCH_SIZE); diff --git a/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseKeyValueStorageEngineFactory.java b/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseKeyValueStorageEngineFactory.java index 05b106d28d..457fcba8da 100644 --- a/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseKeyValueStorageEngineFactory.java +++ b/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseKeyValueStorageEngineFactory.java @@ -139,7 +139,7 @@ public void testInMemoryKeyValueStore() { @Test public void testDurableKeyValueStore() { Config config = new MapConfig(BASE_CONFIG, DISABLE_CACHE, - ImmutableMap.of(String.format(StorageConfig.STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME), + ImmutableMap.of(String.format(StorageConfig.STORE_BACKUP_FACTORIES, STORE_NAME), "backendFactory,backendFactory2")); StorageEngine storageEngine = callGetStorageEngine(config, null); KeyValueStorageEngine keyValueStorageEngine = baseStorageEngineValidation(storageEngine); From c2b77f099d062b2c303be932dfe4386818c0a4ac Mon Sep 17 00:00:00 2001 From: shekhars-li <72765053+shekhars-li@users.noreply.github.com> Date: Fri, 28 May 2021 10:16:53 -0700 Subject: [PATCH 05/20] Fix StorageConfig bug from newly introduced BlobStoreRestore factory config (#1503) --- .../apache/samza/config/StorageConfig.java | 6 +++-- .../KafkaChangelogStateBackendFactory.java | 12 ++++++++++ .../samza/container/SamzaContainer.scala | 17 -------------- .../samza/config/TestStorageConfig.java | 22 +++++++++++++++++++ 4 files changed, 38 insertions(+), 19 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java index aa8df958fb..7535d658c5 100644 --- a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java @@ -74,7 +74,8 @@ public class StorageConfig extends MapConfig { KAFKA_STATE_BACKEND_FACTORY); public static final String STORE_BACKUP_FACTORIES = STORE_PREFIX + "%s.backup.factories"; // TODO BLOCKER dchen make this per store - public static final String STORE_RESTORE_FACTORY = STORE_PREFIX + "restore.factory"; + public static final String RESTORE_FACTORY_SUFFIX = "restore.factory"; + public static final String STORE_RESTORE_FACTORY = STORE_PREFIX + RESTORE_FACTORY_SUFFIX; static final String CHANGELOG_SYSTEM = "job.changelog.system"; static final String CHANGELOG_DELETE_RETENTION_MS = STORE_PREFIX + "%s.changelog.delete.retention.ms"; @@ -100,7 +101,8 @@ public List getStoreNames() { for (String key : subConfig.keySet()) { if (key.endsWith(SIDE_INPUT_PROCESSOR_FACTORY_SUFFIX)) { storeNames.add(key.substring(0, key.length() - SIDE_INPUT_PROCESSOR_FACTORY_SUFFIX.length())); - } else if (key.endsWith(FACTORY_SUFFIX)) { + } else if (key.endsWith(FACTORY_SUFFIX) && !key.equals(RESTORE_FACTORY_SUFFIX)) { + // TODO HIGH dchen STORE_RESTORE_FACTORY added here to be ignored. Update/remove after changing restore factory -> factories storeNames.add(key.substring(0, key.length() - FACTORY_SUFFIX.length())); } } diff --git a/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java b/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java index 54f20da719..be419b6c65 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java +++ b/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java @@ -48,6 +48,18 @@ public class KafkaChangelogStateBackendFactory implements StateBackendFactory { private StreamMetadataCache streamCache; + /* + * This keeps track of the changelog SSPs that are associated with the whole container. This is used so that we can + * prefetch the metadata about the all of the changelog SSPs associated with the container whenever we need the + * metadata about some of the changelog SSPs. + * An example use case is when Samza writes offset files for stores ({@link TaskStorageManager}). Each task is + * responsible for its own offset file, but if we can do prefetching, then most tasks will already have cached + * metadata by the time they need the offset metadata. + * Note: By using all changelog streams to build the sspsToPrefetch, any fetches done for persisted stores will + * include the ssps for non-persisted stores, so this is slightly suboptimal. However, this does not increase the + * actual number of calls to the {@link SystemAdmin}, and we can decouple this logic from the per-task objects (e.g. + * {@link TaskStorageManager}). + */ private SSPMetadataCache sspCache; @Override diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 1e169285db..b3360f2b39 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -346,23 +346,6 @@ object SamzaContainer extends Logging { info("Got change log system streams: %s" format storeChangelogs) - /* - * This keeps track of the changelog SSPs that are associated with the whole container. This is used so that we can - * prefetch the metadata about the all of the changelog SSPs associated with the container whenever we need the - * metadata about some of the changelog SSPs. - * An example use case is when Samza writes offset files for stores ({@link TaskStorageManager}). Each task is - * responsible for its own offset file, but if we can do prefetching, then most tasks will already have cached - * metadata by the time they need the offset metadata. - * Note: By using all changelog streams to build the sspsToPrefetch, any fetches done for persisted stores will - * include the ssps for non-persisted stores, so this is slightly suboptimal. However, this does not increase the - * actual number of calls to the {@link SystemAdmin}, and we can decouple this logic from the per-task objects (e.g. - * {@link TaskStorageManager}). - */ - val changelogSSPMetadataCache = new SSPMetadataCache(systemAdmins, - Duration.ofSeconds(5), - SystemClock.instance, - getChangelogSSPsForContainer(containerModel, storeChangelogs).asJava) - val intermediateStreams = streamConfig .getStreamIds() .asScala diff --git a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java index ca7196a6b0..91f1fa1cfd 100644 --- a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java +++ b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java @@ -70,6 +70,28 @@ public void testGetStoreNames() { assertEquals(expectedStoreNames, ImmutableSet.copyOf(actual)); } + /** + * Test verifies that the {@link StorageConfig#STORE_RESTORE_FACTORY} which matches pattern for store.%s.factory + * is not picked up as in store names list + */ + @Test + public void testGetStoreNamesIgnoreStateRestoreFactory() { + // empty config, so no stores + assertEquals(Collections.emptyList(), new StorageConfig(new MapConfig()).getStoreNames()); + + Set expectedStoreNames = ImmutableSet.of(STORE_NAME0, STORE_NAME1); + // has stores + StorageConfig storageConfig = new StorageConfig(new MapConfig( + ImmutableMap.of(String.format(StorageConfig.FACTORY, STORE_NAME0), "store0.factory.class", + String.format(StorageConfig.FACTORY, STORE_NAME1), "store1.factory.class", + STORE_RESTORE_FACTORY, "org.apache.class"))); + + List actual = storageConfig.getStoreNames(); + // ordering shouldn't matter + assertEquals(2, actual.size()); + assertEquals(expectedStoreNames, ImmutableSet.copyOf(actual)); + } + @Test public void testGetChangelogStream() { // empty config, so no changelog stream From 354620f0356c14c02637baab80a76f8c2d59e11d Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Wed, 2 Jun 2021 12:38:03 -0700 Subject: [PATCH 06/20] SnapshotIndexSerde serialization fixes RB=2624444 RB=2624448 G=samza-reviewers R=shesharm A=shesharm,bkumaras --- .../serializers/JsonCheckpointIdMixin.java | 42 ------------------- .../index/serde/SnapshotIndexSerde.java | 8 ++-- 2 files changed, 3 insertions(+), 47 deletions(-) delete mode 100644 samza-api/src/main/java/org/apache/samza/serializers/JsonCheckpointIdMixin.java diff --git a/samza-api/src/main/java/org/apache/samza/serializers/JsonCheckpointIdMixin.java b/samza-api/src/main/java/org/apache/samza/serializers/JsonCheckpointIdMixin.java deleted file mode 100644 index 752e329c1c..0000000000 --- a/samza-api/src/main/java/org/apache/samza/serializers/JsonCheckpointIdMixin.java +++ /dev/null @@ -1,42 +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.samza.serializers; - -import org.apache.samza.checkpoint.CheckpointId; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - - -/** - * A mix-in Jackson class to convert {@link CheckpointId} to/from JSON. - */ -public abstract class JsonCheckpointIdMixin { - @JsonCreator - private JsonCheckpointIdMixin( - @JsonProperty("millis") long millis, - @JsonProperty("nanos") long nanos) { - } - - @JsonProperty("millis") - abstract long getMillis(); - - @JsonProperty("nanos") - abstract long getNanos(); -} diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/SnapshotIndexSerde.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/SnapshotIndexSerde.java index 2e0e50c678..9e316ada23 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/SnapshotIndexSerde.java +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/SnapshotIndexSerde.java @@ -24,9 +24,8 @@ import com.fasterxml.jackson.databind.ObjectWriter; import com.fasterxml.jackson.datatype.jdk8.Jdk8Module; import org.apache.samza.SamzaException; -import org.apache.samza.checkpoint.CheckpointId; -import org.apache.samza.serializers.JsonCheckpointIdMixin; import org.apache.samza.serializers.Serde; +import org.apache.samza.serializers.model.SamzaObjectMapper; import org.apache.samza.storage.blobstore.index.DirIndex; import org.apache.samza.storage.blobstore.index.FileBlob; import org.apache.samza.storage.blobstore.index.FileIndex; @@ -37,7 +36,7 @@ public class SnapshotIndexSerde implements Serde { - private final static ObjectMapper MAPPER = new ObjectMapper(); + private final static ObjectMapper MAPPER = SamzaObjectMapper.getObjectMapper(); private TypeReference typeReference; private final ObjectWriter objectWriter; @@ -48,8 +47,7 @@ public SnapshotIndexSerde() { .addMixIn(DirIndex.class, JsonDirIndexMixin.class) .addMixIn(FileIndex.class, JsonFileIndexMixin.class) .addMixIn(FileMetadata.class, JsonFileMetadataMixin.class) - .addMixIn(FileBlob.class, JsonFileBlobMixin.class) - .addMixIn(CheckpointId.class, JsonCheckpointIdMixin.class); + .addMixIn(FileBlob.class, JsonFileBlobMixin.class); this.typeReference = new TypeReference() { }; this.objectWriter = MAPPER.writerFor(typeReference); From 48a878567ec50986cabee9d27386c8ca1d5c0b2f Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Tue, 1 Jun 2021 09:54:19 -0700 Subject: [PATCH 07/20] Checkpoint-tool changes for checkpoint v2 RB=2621726 G=samza-reviewers A=pmaheshw --- .../blobstore/BlobStoreRestoreManager.java | 2 +- .../samza/checkpoint/CheckpointTool.scala | 29 +++++++--- .../file/FileSystemCheckpointManager.scala | 3 ++ .../samza/container/SamzaContainer.scala | 13 ----- .../storage/ContainerStorageManager.java | 2 +- .../samza/config/TestStorageConfig.java | 30 ++++++++--- .../samza/checkpoint/TestCheckpointTool.scala | 54 +++++++++++++++++-- 7 files changed, 99 insertions(+), 34 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java index 052e49da89..fa0beb099a 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java @@ -168,7 +168,7 @@ protected BlobStoreUtil createBlobStoreUtil(BlobStoreManager blobStoreManager, E /** * Deletes blob store contents for stores that were present in the last checkpoint but are either no longer - * present in job configs (removed by user since last deploymetn) or are no longer configured to be backed + * present in job configs (removed by user since last deployment) or are no longer configured to be backed * up using blob stores. * * This method blocks until all the necessary store contents and snapshot index blobs have been marked for deletion. diff --git a/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala b/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala index 92b4e593cd..15b71c341b 100644 --- a/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala +++ b/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala @@ -185,22 +185,37 @@ class CheckpointTool(newOffsets: TaskNameToCheckpointMap, coordinatorStreamStore taskNames.foreach(checkpointManager.register) checkpointManager.start() - // TODO dchen make add support for checkpointv2 + val checkpointReadVersion = taskConfig.getCheckpointReadVersion + val defaultCheckpoint = if (checkpointReadVersion == 1) { + new CheckpointV1(new java.util.HashMap[SystemStreamPartition, String]()) + } else if (checkpointReadVersion == 2) { + new CheckpointV2(CheckpointId.create(), new java.util.HashMap[SystemStreamPartition, String](), + new java.util.HashMap[String, util.Map[String, String]]()) + } else { + throw new SamzaException("Unrecognized checkpoint read version: " + checkpointReadVersion) + } + val lastCheckpoints = taskNames.map(taskName => { taskName -> Option(checkpointManager.readLastCheckpoint(taskName)) - .getOrElse(new CheckpointV1(new java.util.HashMap[SystemStreamPartition, String]())) - .getOffsets - .asScala - .toMap + .getOrElse(defaultCheckpoint) }).toMap - lastCheckpoints.foreach(lcp => logCheckpoint(lcp._1, lcp._2, "Current checkpoint for task: "+ lcp._1)) + lastCheckpoints.foreach(lcp => logCheckpoint(lcp._1, lcp._2.getOffsets.asScala.toMap, + "Current checkpoint for task: " + lcp._1)) if (newOffsets != null) { newOffsets.foreach { case (taskName: TaskName, offsets: Map[SystemStreamPartition, String]) => logCheckpoint(taskName, offsets, "New offset to be written for task: " + taskName) - val checkpoint = new CheckpointV1(offsets.asJava) + val checkpoint = if (checkpointReadVersion == 1) { + new CheckpointV1(offsets.asJava) + } else if (checkpointReadVersion == 2) { + val lastSCMs = lastCheckpoints.getOrElse(taskName, defaultCheckpoint) + .asInstanceOf[CheckpointV2].getStateCheckpointMarkers + new CheckpointV2(CheckpointId.create(), offsets.asJava, lastSCMs) + } else { + throw new SamzaException("Unrecognized checkpoint read version: " + checkpointReadVersion) + } checkpointManager.writeCheckpoint(taskName, checkpoint) info(s"Updated the checkpoint of the task: $taskName to: $offsets") } diff --git a/samza-core/src/main/scala/org/apache/samza/checkpoint/file/FileSystemCheckpointManager.scala b/samza-core/src/main/scala/org/apache/samza/checkpoint/file/FileSystemCheckpointManager.scala index b740a130e9..a068e2adc5 100644 --- a/samza-core/src/main/scala/org/apache/samza/checkpoint/file/FileSystemCheckpointManager.scala +++ b/samza-core/src/main/scala/org/apache/samza/checkpoint/file/FileSystemCheckpointManager.scala @@ -43,6 +43,9 @@ class FileSystemCheckpointManager( def getCheckpointFile(taskName: TaskName) = getFile(jobName, taskName, "checkpoints") def writeCheckpoint(taskName: TaskName, checkpoint: Checkpoint) { + if (!checkpoint.isInstanceOf[CheckpointV1]) { + throw new SamzaException("Unsupported checkpoint version: " + checkpoint.getVersion) + } val bytes = serde.toBytes(checkpoint.asInstanceOf[CheckpointV1]) val fos = new FileOutputStream(getCheckpointFile(taskName)) diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index b3360f2b39..17167b1036 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -684,19 +684,6 @@ object SamzaContainer extends Logging { containerStorageManager = containerStorageManager, diagnosticsManager = diagnosticsManager) } - - /** - * Builds the set of SSPs for all changelogs on this container. - */ - @VisibleForTesting - private[container] def getChangelogSSPsForContainer(containerModel: ContainerModel, - changeLogSystemStreams: util.Map[String, SystemStream]): Set[SystemStreamPartition] = { - containerModel.getTasks.values().asScala - .map(taskModel => taskModel.getChangelogPartition) - .flatMap(changelogPartition => changeLogSystemStreams.asScala.map { case (_, systemStream) => - new SystemStreamPartition(systemStream, changelogPartition) }) - .toSet - } } class SamzaContainer( diff --git a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java index 5cc79b3c7b..08ae375e0a 100644 --- a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java +++ b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java @@ -467,7 +467,7 @@ private Map> createTaskStores(Set s // add created store to map taskStores.get(taskName).put(storeName, storageEngine); - LOG.info("Created non side input store store {} in read-write mode for task {}", storeName, taskName); + LOG.info("Created task store {} in read-write mode for task {} in path {}", storeName, taskName, storeDirectory.getAbsolutePath()); } } return taskStores; diff --git a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java index 91f1fa1cfd..3a0debce98 100644 --- a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java +++ b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java @@ -30,6 +30,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import org.apache.samza.SamzaException; +import org.apache.samza.system.SystemStream; import org.junit.Test; import static org.apache.samza.config.StorageConfig.*; @@ -101,57 +102,72 @@ public void testGetChangelogStream() { StorageConfig storageConfig = new StorageConfig( new MapConfig(ImmutableMap.of(String.format(StorageConfig.CHANGELOG_STREAM, STORE_NAME0), ""))); assertEquals(Optional.empty(), storageConfig.getChangelogStream(STORE_NAME0)); + assertEquals(Collections.emptyMap(), storageConfig.getStoreChangelogs()); // store has full changelog system-stream defined storageConfig = new StorageConfig(new MapConfig( ImmutableMap.of(String.format(StorageConfig.CHANGELOG_STREAM, STORE_NAME0), - "changelog-system.changelog-stream0"))); + "changelog-system.changelog-stream0", String.format(FACTORY, STORE_NAME0), "store0.factory.class"))); assertEquals(Optional.of("changelog-system.changelog-stream0"), storageConfig.getChangelogStream(STORE_NAME0)); + assertEquals(ImmutableMap.of(STORE_NAME0, new SystemStream("changelog-system", "changelog-stream0")), storageConfig.getStoreChangelogs()); // store has changelog stream defined, but system comes from job.changelog.system storageConfig = new StorageConfig(new MapConfig( ImmutableMap.of(String.format(StorageConfig.CHANGELOG_STREAM, STORE_NAME0), "changelog-stream0", + String.format(FACTORY, STORE_NAME0), "store0.factory.class" , StorageConfig.CHANGELOG_SYSTEM, "changelog-system"))); assertEquals(Optional.of("changelog-system.changelog-stream0"), storageConfig.getChangelogStream(STORE_NAME0)); + assertEquals(ImmutableMap.of(STORE_NAME0, new SystemStream("changelog-system", "changelog-stream0")), storageConfig.getStoreChangelogs()); // batch mode: create unique stream name storageConfig = new StorageConfig(new MapConfig( ImmutableMap.of(String.format(StorageConfig.CHANGELOG_STREAM, STORE_NAME0), - "changelog-system.changelog-stream0", ApplicationConfig.APP_MODE, + "changelog-system.changelog-stream0", String.format(FACTORY, STORE_NAME0), "store0.factory.class", + ApplicationConfig.APP_MODE, ApplicationConfig.ApplicationMode.BATCH.name().toLowerCase(), ApplicationConfig.APP_RUN_ID, "run-id"))); assertEquals(Optional.of("changelog-system.changelog-stream0-run-id"), storageConfig.getChangelogStream(STORE_NAME0)); + assertEquals(ImmutableMap.of(STORE_NAME0, new SystemStream("changelog-system", "changelog-stream0-run-id")), storageConfig.getStoreChangelogs()); // job has no changelog stream defined storageConfig = new StorageConfig(new MapConfig( ImmutableMap.of(StorageConfig.CHANGELOG_SYSTEM, "changelog-system", JobConfig.JOB_DEFAULT_SYSTEM, - "should-not-be-used"))); + "should-not-be-used", String.format(FACTORY, STORE_NAME0), "store0.factory.class"))); assertEquals(Optional.empty(), storageConfig.getChangelogStream(STORE_NAME0)); + assertEquals(Collections.emptyMap(), storageConfig.getStoreChangelogs()); // job.changelog.system takes precedence over job.default.system when changelog is specified as just streamName storageConfig = new StorageConfig(new MapConfig( ImmutableMap.of(StorageConfig.CHANGELOG_SYSTEM, "changelog-system", JobConfig.JOB_DEFAULT_SYSTEM, - "should-not-be-used", String.format(CHANGELOG_STREAM, STORE_NAME0), "streamName"))); + "should-not-be-used", String.format(CHANGELOG_STREAM, STORE_NAME0), "streamName", + String.format(FACTORY, STORE_NAME0), "store0.factory.class"))); assertEquals("changelog-system.streamName", storageConfig.getChangelogStream(STORE_NAME0).get()); + assertEquals(ImmutableMap.of(STORE_NAME0, new SystemStream("changelog-system", "streamName")), storageConfig.getStoreChangelogs()); // job.changelog.system takes precedence over job.default.system when changelog is specified as {systemName}.{streamName} storageConfig = new StorageConfig(new MapConfig( ImmutableMap.of(StorageConfig.CHANGELOG_SYSTEM, "changelog-system", JobConfig.JOB_DEFAULT_SYSTEM, - "should-not-be-used", String.format(CHANGELOG_STREAM, STORE_NAME0), "changelog-system.streamName"))); + "should-not-be-used", String.format(CHANGELOG_STREAM, STORE_NAME0), "changelog-system.streamName", + String.format(FACTORY, STORE_NAME0), "store0.factory.class"))); assertEquals("changelog-system.streamName", storageConfig.getChangelogStream(STORE_NAME0).get()); + assertEquals(ImmutableMap.of(STORE_NAME0, new SystemStream("changelog-system", "streamName")), storageConfig.getStoreChangelogs()); // systemName specified using stores.{storeName}.changelog = {systemName}.{streamName} should take precedence even // when job.changelog.system and job.default.system are specified storageConfig = new StorageConfig(new MapConfig( ImmutableMap.of(StorageConfig.CHANGELOG_SYSTEM, "default-changelog-system", JobConfig.JOB_DEFAULT_SYSTEM, "default-system", - String.format(CHANGELOG_STREAM, STORE_NAME0), "nondefault-changelog-system.streamName"))); + String.format(CHANGELOG_STREAM, STORE_NAME0), "nondefault-changelog-system.streamName", + String.format(FACTORY, STORE_NAME0), "store0.factory.class"))); assertEquals("nondefault-changelog-system.streamName", storageConfig.getChangelogStream(STORE_NAME0).get()); + assertEquals(ImmutableMap.of(STORE_NAME0, new SystemStream("nondefault-changelog-system", "streamName")), storageConfig.getStoreChangelogs()); // fall back to job.default.system if job.changelog.system is not specified storageConfig = new StorageConfig(new MapConfig( - ImmutableMap.of(JobConfig.JOB_DEFAULT_SYSTEM, "default-system", String.format(CHANGELOG_STREAM, STORE_NAME0), "streamName"))); + ImmutableMap.of(JobConfig.JOB_DEFAULT_SYSTEM, "default-system", String.format(CHANGELOG_STREAM, STORE_NAME0), + "streamName", String.format(FACTORY, STORE_NAME0), "store0.factory.class"))); assertEquals("default-system.streamName", storageConfig.getChangelogStream(STORE_NAME0).get()); + assertEquals(ImmutableMap.of(STORE_NAME0, new SystemStream("default-system", "streamName")), storageConfig.getStoreChangelogs()); } @Test(expected = SamzaException.class) diff --git a/samza-core/src/test/scala/org/apache/samza/checkpoint/TestCheckpointTool.scala b/samza-core/src/test/scala/org/apache/samza/checkpoint/TestCheckpointTool.scala index 5bcbdcd2a2..16778e40fe 100644 --- a/samza-core/src/test/scala/org/apache/samza/checkpoint/TestCheckpointTool.scala +++ b/samza-core/src/test/scala/org/apache/samza/checkpoint/TestCheckpointTool.scala @@ -19,8 +19,7 @@ package org.apache.samza.checkpoint -import java.util.Properties - +import java.util.{Collections, Properties} import org.apache.samza.Partition import org.apache.samza.checkpoint.CheckpointTool.{CheckpointToolCommandLine, TaskNameToCheckpointMap} import org.apache.samza.checkpoint.TestCheckpointTool.{MockCheckpointManagerFactory, MockSystemFactory} @@ -32,9 +31,10 @@ import org.apache.samza.execution.JobPlanner import org.apache.samza.metrics.MetricsRegistry import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata import org.apache.samza.system._ +import org.junit.Assert.{assertEquals, assertNotNull} import org.junit.{Before, Test} import org.mockito.Matchers._ -import org.mockito.Mockito +import org.mockito.{ArgumentCaptor, Matchers, Mockito} import org.mockito.Mockito._ import org.scalatest.junit.AssertionsForJUnit import org.scalatest.mockito.MockitoSugar @@ -67,8 +67,10 @@ class TestCheckpointTool extends AssertionsForJUnit with MockitoSugar { val tn0 = new TaskName("Partition 0") val tn1 = new TaskName("Partition 1") + val tn2 = new TaskName("Partition 2") val p0 = new Partition(0) val p1 = new Partition(1) + val p2 = new Partition(2) @Before def setup() { @@ -83,8 +85,9 @@ class TestCheckpointTool extends AssertionsForJUnit with MockitoSugar { ).asJava) config = JobPlanner.generateSingleJobConfig(userDefinedConfig) val metadata = new SystemStreamMetadata("foo", Map[Partition, SystemStreamPartitionMetadata]( - new Partition(0) -> new SystemStreamPartitionMetadata("0", "100", "101"), - new Partition(1) -> new SystemStreamPartitionMetadata("0", "200", "201") + p0 -> new SystemStreamPartitionMetadata("0", "100", "101"), + p1 -> new SystemStreamPartitionMetadata("0", "200", "201"), + p2 -> new SystemStreamPartitionMetadata("0", "300", "301") ).asJava) TestCheckpointTool.checkpointManager = mock[CheckpointManager] TestCheckpointTool.systemAdmin = mock[SystemAdmin] @@ -94,6 +97,11 @@ class TestCheckpointTool extends AssertionsForJUnit with MockitoSugar { .thenReturn(new CheckpointV1(Map(new SystemStreamPartition("test", "foo", p0) -> "1234").asJava)) when(TestCheckpointTool.checkpointManager.readLastCheckpoint(tn1)) .thenReturn(new CheckpointV1(Map(new SystemStreamPartition("test", "foo", p1) -> "4321").asJava)) + when(TestCheckpointTool.checkpointManager.readLastCheckpoint(tn2)) + .thenReturn(new CheckpointV2(null, + Map(new SystemStreamPartition("test", "foo", p2) -> "5678").asJava, + Map("BackupFactory"-> Map("StoreName"-> "offset").asJava).asJava + )) } @Test @@ -118,6 +126,42 @@ class TestCheckpointTool extends AssertionsForJUnit with MockitoSugar { .writeCheckpoint(tn1, new CheckpointV1(Map(new SystemStreamPartition("test", "foo", p1) -> "43").asJava)) } + @Test + def testOverwriteCheckpointV2() { + // Skips the v1 checkpoints for the task + when(TestCheckpointTool.checkpointManager.readLastCheckpoint(tn0)) + .thenReturn(null) + when(TestCheckpointTool.checkpointManager.readLastCheckpoint(tn1)) + .thenReturn(null) + + val toOverwrite = Map( + tn0 -> Map(new SystemStreamPartition("test", "foo", p0) -> "42"), + tn1 -> Map(new SystemStreamPartition("test", "foo", p1) -> "43"), + tn2 -> Map(new SystemStreamPartition("test", "foo", p2) -> "45")) + + val checkpointV2Config = new MapConfig(config, Map(TaskConfig.CHECKPOINT_READ_VERSION -> "2").asJava) + + val argument = ArgumentCaptor.forClass(classOf[CheckpointV2]) + val checkpointTool = CheckpointTool(checkpointV2Config, toOverwrite) + + checkpointTool.run() + verify(TestCheckpointTool.checkpointManager) + .writeCheckpoint(Matchers.same(tn0), argument.capture()) + assertNotNull(argument.getValue.getCheckpointId) + assertEquals(Map(new SystemStreamPartition("test", "foo", p0) -> "42").asJava, argument.getValue.getOffsets) + assertEquals(Collections.emptyMap(), argument.getValue.getStateCheckpointMarkers) + verify(TestCheckpointTool.checkpointManager) + .writeCheckpoint(Matchers.same(tn1), argument.capture()) + assertNotNull(argument.getValue.getCheckpointId) + assertEquals(Map(new SystemStreamPartition("test", "foo", p1) -> "43").asJava, argument.getValue.getOffsets) + assertEquals(Collections.emptyMap(), argument.getValue.getStateCheckpointMarkers) + verify(TestCheckpointTool.checkpointManager) + .writeCheckpoint(Matchers.same(tn2), argument.capture()) + assertNotNull(argument.getValue.getCheckpointId) + assertEquals(Map(new SystemStreamPartition("test", "foo", p2) -> "45").asJava, argument.getValue.getOffsets) + assertEquals(Map("BackupFactory"-> Map("StoreName"-> "offset").asJava).asJava, argument.getValue.getStateCheckpointMarkers) + } + @Test def testGrouping(): Unit = { val config : java.util.Properties = new Properties() From 20631555fc1bb7d663def3e5d42d03e1ae56a318 Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Thu, 3 Jun 2021 19:00:36 -0700 Subject: [PATCH 08/20] HOT FIX: KafkaConsumerProxy without any registered TopicPartitions --- .../org/apache/samza/system/kafka/KafkaConsumerProxy.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaConsumerProxy.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaConsumerProxy.java index 8fd21fe92b..9cf8eae3c0 100644 --- a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaConsumerProxy.java +++ b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaConsumerProxy.java @@ -154,7 +154,9 @@ public void start() { if (topicPartitionToSSP.size() == 0) { String msg = String.format("Cannot start KafkaConsumerProxy without any registered TopicPartitions for %s", systemName); LOG.error(msg); - throw new SamzaException(msg); + // TODO HIGH dchen the consumer init for blob stores only goes through this path as well if it has changelogs for backup + // need to isolate the changes + //throw new SamzaException(msg); } } From 271ef6469ca4bf264830a885df5d64e64cb9addb Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Mon, 7 Jun 2021 11:49:37 -0700 Subject: [PATCH 09/20] Fix async cleanup TaskMetric RB=2632353 G=samza-reviewers R=pmaheshw,shesharm A=pmaheshw --- .../scala/org/apache/samza/container/TaskInstance.scala | 8 ++++---- .../org/apache/samza/container/TestTaskInstance.scala | 9 +++++++++ 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala index 8a872dedda..5f3570401c 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala @@ -356,14 +356,11 @@ class TaskInstance( val checkpointWriteFuture: CompletableFuture[util.Map[String, util.Map[String, String]]] = uploadSCMsFuture.thenApplyAsync(writeCheckpoint(checkpointId, inputOffsets), commitThreadPool) - val cleanupStartTimeNs = System.nanoTime() val cleanUpFuture: CompletableFuture[Void] = checkpointWriteFuture.thenComposeAsync(cleanUp(checkpointId), commitThreadPool) cleanUpFuture.whenComplete(new BiConsumer[Void, Throwable] { override def accept(v: Void, throwable: Throwable): Unit = { - if (throwable == null) { - metrics.asyncCleanupNs.update(System.nanoTime() - cleanupStartTimeNs) - } else { + if (throwable != null) { warn("Commit cleanup did not complete successfully for taskName: %s checkpointId: %s with error msg: %s" format (taskName, checkpointId, throwable.getMessage)) } @@ -428,6 +425,7 @@ class TaskInstance( override def apply(uploadSCMs: util.Map[String, util.Map[String, String]]): CompletableFuture[Void] = { // Perform cleanup on unused checkpoints debug("Cleaning up old checkpoint state for taskName: %s checkpointId: %s" format(taskName, checkpointId)) + val cleanUpStartTime = System.nanoTime() try { commitManager.cleanUp(checkpointId, uploadSCMs) } catch { @@ -438,6 +436,8 @@ class TaskInstance( throw new SamzaException( "Failed to remove old checkpoint state for taskName: %s checkpointId: %s." format(taskName, checkpointId), e) + } finally { + metrics.asyncCleanupNs.update(System.nanoTime() - cleanUpStartTime) } } } diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala index 606f86dfac..9d2d325b8c 100644 --- a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala +++ b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala @@ -728,10 +728,19 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { taskInstance.commit // async stage will be run by caller due to direct executor + val asyncCommitTimeCaptor = ArgumentCaptor.forClass(classOf[Long]) + val uploadTimeCaptor = ArgumentCaptor.forClass(classOf[Long]) + val cleanUpTimeCaptor = ArgumentCaptor.forClass(classOf[Long]) + verify(commitsCounter).inc() verify(snapshotTimer).update(anyLong()) verify(uploadTimer).update(anyLong()) verify(commitTimer).update(anyLong()) + verify(commitAsyncTimer).update(asyncCommitTimeCaptor.capture()) + verify(uploadTimer).update(uploadTimeCaptor.capture()) + verify(cleanUpTimer).update(cleanUpTimeCaptor.capture()) + + assertTrue((cleanUpTimeCaptor.getValue + uploadTimeCaptor.getValue) < asyncCommitTimeCaptor.getValue) taskInstance.commit From 7d61a7d9de9429f64664586144bff66ecc3852ae Mon Sep 17 00:00:00 2001 From: shekhars-li Date: Tue, 8 Jun 2021 13:58:32 -0700 Subject: [PATCH 10/20] Added lastCommitNs and commitTimedOut metrics, updated default timeout to 30 mins RB=2634862 RB=2634867 G=samza-reviewers R=pmaheshw,dchen1 A=dchen1 --- .../src/main/java/org/apache/samza/config/TaskConfig.java | 2 +- .../main/scala/org/apache/samza/container/TaskInstance.scala | 2 ++ .../scala/org/apache/samza/container/TaskInstanceMetrics.scala | 3 ++- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/config/TaskConfig.java b/samza-core/src/main/java/org/apache/samza/config/TaskConfig.java index 8db1d2a5b6..cca01f5206 100644 --- a/samza-core/src/main/java/org/apache/samza/config/TaskConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/TaskConfig.java @@ -63,7 +63,7 @@ public class TaskConfig extends MapConfig { // COMMIT_MAX_DELAY_MS have passed since the pending commit start. if the pending commit // does not complete within this timeout, the container will shut down. public static final String COMMIT_TIMEOUT_MS = "task.commit.timeout.ms"; - static final long DEFAULT_COMMIT_TIMEOUT_MS = Duration.ofMinutes(1).toMillis(); + static final long DEFAULT_COMMIT_TIMEOUT_MS = Duration.ofMinutes(30).toMillis(); // how long to wait for a clean shutdown public static final String TASK_SHUTDOWN_MS = "task.shutdown.ms"; diff --git a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala index 5f3570401c..f91aae1ee3 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala @@ -290,6 +290,7 @@ class TaskInstance( if (!commitInProgress.tryAcquire(commitTimeoutMs, TimeUnit.MILLISECONDS)) { val timeSinceLastCommit = System.currentTimeMillis() - lastCommitStartTimeMs + metrics.commitsTimedOut.set(metrics.commitsTimedOut.getValue + 1) throw new SamzaException("Timeout waiting for pending commit for taskName: %s to finish. " + "%s ms have elapsed since the pending commit started. Max allowed commit delay is %s ms " + "and commit timeout beyond that is %s ms" format (taskName, timeSinceLastCommit, @@ -377,6 +378,7 @@ class TaskInstance( } }) + metrics.lastCommitNs.set(System.nanoTime() - commitStartNs) metrics.commitSyncNs.update(System.nanoTime() - commitStartNs) debug("Finishing sync stage of commit for taskName: %s checkpointId: %s" format (taskName, checkpointId)) } diff --git a/samza-core/src/main/scala/org/apache/samza/container/TaskInstanceMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/TaskInstanceMetrics.scala index f13e37a678..54d3665253 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/TaskInstanceMetrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/TaskInstanceMetrics.scala @@ -38,9 +38,10 @@ class TaskInstanceMetrics( val pendingMessages = newGauge("pending-messages", 0) val messagesInFlight = newGauge("messages-in-flight", 0) val asyncCallbackCompleted = newCounter("async-callback-complete-calls") - + val commitsTimedOut = newGauge("commits-timed-out", 0) val commitsSkipped = newGauge("commits-skipped", 0) val commitNs = newTimer("commit-ns") + val lastCommitNs = newGauge("last-commit-ns", 0L) val commitSyncNs = newTimer("commit-sync-ns") val commitAsyncNs = newTimer("commit-async-ns") val snapshotNs = newTimer("snapshot-ns") From 7f5ff3a9e472a0352b2f329423b5f5a58916cf6d Mon Sep 17 00:00:00 2001 From: shekhars-li Date: Fri, 11 Jun 2021 16:31:54 -0700 Subject: [PATCH 11/20] Updated log line in BlobStoreUtil --- .../apache/samza/storage/blobstore/util/BlobStoreUtil.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java index 9c8b61f522..b312b368a1 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java @@ -456,7 +456,7 @@ CompletableFuture getFile(List fileBlobs, File fileToRestore, Me * @return A future containing the {@link FileIndex} for the uploaded file. */ @VisibleForTesting - CompletableFuture putFile(File file, SnapshotMetadata snapshotMetadata) { + public CompletableFuture putFile(File file, SnapshotMetadata snapshotMetadata) { if (file == null || !file.isFile()) { String message = file != null ? "Dir or Symbolic link" : "null"; throw new SamzaException(String.format("Required a non-null parameter of type file, provided: %s", message)); @@ -483,7 +483,7 @@ CompletableFuture putFile(File file, SnapshotMetadata snapshotMetadat fileBlobFuture = blobStoreManager.put(inputStream, metadata) .thenApplyAsync(id -> { - LOG.trace("Put complete. Closing input stream for file: {}.", file.getPath()); + LOG.trace("Put complete. Received Blob ID {}. Closing input stream for file: {}.", id, file.getPath()); try { finalInputStream.close(); } catch (Exception e) { From 35c0dbc96d87001ced5e0c0272e28b7d2a8c8602 Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Thu, 17 Jun 2021 18:02:25 -0700 Subject: [PATCH 12/20] Checkpoint version config precedence list RB=2652787 RB=2652789 G=samza-reviewers R=pmaheshw,shesharm A=pmaheshw --- .../samza/storage/TaskRestoreManager.java | 2 +- .../org/apache/samza/config/TaskConfig.java | 16 ++++-- .../samza/startpoint/StartpointManager.java | 2 +- .../TransactionalStateTaskRestoreManager.java | 2 +- .../samza/checkpoint/CheckpointTool.scala | 3 +- .../samza/checkpoint/TestCheckpointTool.scala | 2 +- .../samza/container/TestTaskInstance.scala | 25 ++++++++- .../kafka/KafkaCheckpointLogKey.java | 6 ++ .../kafka/KafkaCheckpointManager.scala | 49 +++++++++++++---- .../kafka/TestKafkaCheckpointManager.scala | 55 ++++++++++++++++++- .../CheckpointVersionIntegrationTest.java | 2 +- 11 files changed, 141 insertions(+), 23 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/storage/TaskRestoreManager.java b/samza-api/src/main/java/org/apache/samza/storage/TaskRestoreManager.java index 999325e309..1f5a5ce4d0 100644 --- a/samza-api/src/main/java/org/apache/samza/storage/TaskRestoreManager.java +++ b/samza-api/src/main/java/org/apache/samza/storage/TaskRestoreManager.java @@ -47,7 +47,7 @@ public interface TaskRestoreManager { void restore() throws InterruptedException; /** - * Closes all initiated ressources include storage engines + * Closes all initiated resources include storage engines */ void close(); } diff --git a/samza-core/src/main/java/org/apache/samza/config/TaskConfig.java b/samza-core/src/main/java/org/apache/samza/config/TaskConfig.java index cca01f5206..06a8727480 100644 --- a/samza-core/src/main/java/org/apache/samza/config/TaskConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/TaskConfig.java @@ -125,8 +125,8 @@ public class TaskConfig extends MapConfig { public static final List DEFAULT_CHECKPOINT_WRITE_VERSIONS = ImmutableList.of("1", "2"); // checkpoint version to read during container startup - public static final String CHECKPOINT_READ_VERSION = "task.checkpoint.read.version"; - public static final short DEFAULT_CHECKPOINT_READ_VERSION = 1; + public static final String CHECKPOINT_READ_VERSIONS = "task.checkpoint.read.versions"; + public static final List DEFAULT_CHECKPOINT_READ_VERSIONS = ImmutableList.of("1"); public static final String TRANSACTIONAL_STATE_CHECKPOINT_ENABLED = "task.transactional.state.checkpoint.enabled"; private static final boolean DEFAULT_TRANSACTIONAL_STATE_CHECKPOINT_ENABLED = true; @@ -348,8 +348,16 @@ public List getCheckpointWriteVersions() { .stream().map(Short::valueOf).collect(Collectors.toList()); } - public short getCheckpointReadVersion() { - return getShort(CHECKPOINT_READ_VERSION, DEFAULT_CHECKPOINT_READ_VERSION); + public List getCheckpointReadVersions() { + List checkpointReadPriorityList = getList(CHECKPOINT_READ_VERSIONS, DEFAULT_CHECKPOINT_READ_VERSIONS) + .stream().map(Short::valueOf).collect(Collectors.toList()); + if (checkpointReadPriorityList.isEmpty()) { + // if the user explicitly defines the checkpoint read list to be empty + throw new IllegalArgumentException("No checkpoint read versions defined for job. " + + "Please remove the task.checkpoint.read.versions or define valid checkpoint versions"); + } else { + return checkpointReadPriorityList; + } } public boolean getTransactionalStateCheckpointEnabled() { diff --git a/samza-core/src/main/java/org/apache/samza/startpoint/StartpointManager.java b/samza-core/src/main/java/org/apache/samza/startpoint/StartpointManager.java index c1722889ae..083b483d6b 100644 --- a/samza-core/src/main/java/org/apache/samza/startpoint/StartpointManager.java +++ b/samza-core/src/main/java/org/apache/samza/startpoint/StartpointManager.java @@ -244,7 +244,7 @@ public void deleteAllStartpoints() { * The Startpoints that are written to with {@link #writeStartpoint(SystemStreamPartition, Startpoint)} and with * {@link #writeStartpoint(SystemStreamPartition, TaskName, Startpoint)} are moved from a "read-write" namespace * to a "fan out" namespace. - * This method is not atomic or thread-safe. The intent is for the Samza Processor's coordinator to use this + * This method is not atomic nor thread-safe. The intent is for the Samza Processor's coordinator to use this * method to assign the Startpoints to the appropriate tasks. * @param taskToSSPs Determines which {@link TaskName} each {@link SystemStreamPartition} maps to. * @return The set of active {@link TaskName}s that were fanned out to. diff --git a/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java index 20f653c76e..f33bb5b34f 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java @@ -611,7 +611,7 @@ private Map getCheckpointedChangelogOffsets( } else if (checkpoint instanceof CheckpointV1) { // If the checkpoint v1 is used, we need to fetch the changelog SSPs in the inputOffsets in order to get the // store offset. - Map checkpointedOffsets = ((CheckpointV1) checkpoint).getOffsets(); + Map checkpointedOffsets = checkpoint.getOffsets(); storeChangelogs.forEach((storeName, systemStream) -> { Partition changelogPartition = taskModel.getChangelogPartition(); SystemStreamPartition storeChangelogSSP = new SystemStreamPartition(systemStream, changelogPartition); diff --git a/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala b/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala index 15b71c341b..1f98b22b52 100644 --- a/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala +++ b/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala @@ -185,7 +185,8 @@ class CheckpointTool(newOffsets: TaskNameToCheckpointMap, coordinatorStreamStore taskNames.foreach(checkpointManager.register) checkpointManager.start() - val checkpointReadVersion = taskConfig.getCheckpointReadVersion + // Get preferred read version for the checkpoint application + val checkpointReadVersion = taskConfig.getCheckpointReadVersions.get(0) val defaultCheckpoint = if (checkpointReadVersion == 1) { new CheckpointV1(new java.util.HashMap[SystemStreamPartition, String]()) } else if (checkpointReadVersion == 2) { diff --git a/samza-core/src/test/scala/org/apache/samza/checkpoint/TestCheckpointTool.scala b/samza-core/src/test/scala/org/apache/samza/checkpoint/TestCheckpointTool.scala index 16778e40fe..85b2011ddb 100644 --- a/samza-core/src/test/scala/org/apache/samza/checkpoint/TestCheckpointTool.scala +++ b/samza-core/src/test/scala/org/apache/samza/checkpoint/TestCheckpointTool.scala @@ -139,7 +139,7 @@ class TestCheckpointTool extends AssertionsForJUnit with MockitoSugar { tn1 -> Map(new SystemStreamPartition("test", "foo", p1) -> "43"), tn2 -> Map(new SystemStreamPartition("test", "foo", p2) -> "45")) - val checkpointV2Config = new MapConfig(config, Map(TaskConfig.CHECKPOINT_READ_VERSION -> "2").asJava) + val checkpointV2Config = new MapConfig(config, Map(TaskConfig.CHECKPOINT_READ_VERSIONS -> "2").asJava) val argument = ArgumentCaptor.forClass(classOf[CheckpointV2]) val checkpointTool = CheckpointTool(checkpointV2Config, toOverwrite) diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala index 9d2d325b8c..e357fe5e0a 100644 --- a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala +++ b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala @@ -227,6 +227,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { when(this.metrics.snapshotNs).thenReturn(snapshotTimer) val commitTimer = mock[Timer] when(this.metrics.commitNs).thenReturn(commitTimer) + val lastCommitGauge = mock[Gauge[Long]] + when(this.metrics.lastCommitNs).thenReturn(lastCommitGauge) val commitSyncTimer = mock[Timer] when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) val commitAsyncTimer = mock[Timer] @@ -320,6 +322,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { when(this.metrics.snapshotNs).thenReturn(snapshotTimer) val commitTimer = mock[Timer] when(this.metrics.commitNs).thenReturn(commitTimer) + val lastCommitGauge = mock[Gauge[Long]] + when(this.metrics.lastCommitNs).thenReturn(lastCommitGauge) val commitSyncTimer = mock[Timer] when(this.metrics.commitSyncNs).thenReturn(commitSyncTimer) val commitAsyncTimer = mock[Timer] @@ -389,6 +393,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { when(this.metrics.asyncUploadNs).thenReturn(uploadTimer) val skippedCounter = mock[Gauge[Int]] when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + val lastCommitGauge = mock[Gauge[Long]] + when(this.metrics.lastCommitNs).thenReturn(lastCommitGauge) val inputOffsets = Map(SYSTEM_STREAM_PARTITION -> "4").asJava val stateCheckpointMarkers: util.Map[String, String] = new util.HashMap[String, String]() @@ -460,6 +466,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) val skippedCounter = mock[Gauge[Int]] when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + val lastCommitGauge = mock[Gauge[Long]] + when(this.metrics.lastCommitNs).thenReturn(lastCommitGauge) val inputOffsets = new util.HashMap[SystemStreamPartition, String]() inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") @@ -510,6 +518,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) val skippedCounter = mock[Gauge[Int]] when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + val lastCommitGauge = mock[Gauge[Long]] + when(this.metrics.lastCommitNs).thenReturn(lastCommitGauge) val inputOffsets = new util.HashMap[SystemStreamPartition, String]() inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") @@ -560,6 +570,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) val skippedCounter = mock[Gauge[Int]] when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + val lastCommitGauge = mock[Gauge[Long]] + when(this.metrics.lastCommitNs).thenReturn(lastCommitGauge) val inputOffsets = new util.HashMap[SystemStreamPartition, String]() inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") @@ -611,6 +623,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) val skippedCounter = mock[Gauge[Int]] when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + val lastCommitGauge = mock[Gauge[Long]] + when(this.metrics.lastCommitNs).thenReturn(lastCommitGauge) val inputOffsets = new util.HashMap[SystemStreamPartition, String]() inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") @@ -662,6 +676,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) val skippedCounter = mock[Gauge[Int]] when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + val lastCommitGauge = mock[Gauge[Long]] + when(this.metrics.lastCommitNs).thenReturn(lastCommitGauge) val inputOffsets = new util.HashMap[SystemStreamPartition, String]() inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") @@ -714,6 +730,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) val skippedCounter = mock[Gauge[Int]] when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + val lastCommitGauge = mock[Gauge[Long]] + when(this.metrics.lastCommitNs).thenReturn(lastCommitGauge) val inputOffsets = new util.HashMap[SystemStreamPartition, String]() inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") @@ -772,6 +790,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) val skippedCounter = mock[Gauge[Int]] when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + val lastCommitGauge = mock[Gauge[Long]] + when(this.metrics.lastCommitNs).thenReturn(lastCommitGauge) val inputOffsets = new util.HashMap[SystemStreamPartition, String]() inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") @@ -803,7 +823,6 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { verify(commitsCounter, times(1)).inc() // should only have been incremented once on the initial commit verify(snapshotTimer).update(anyLong()) - verify(uploadTimer).update(anyLong()) verifyZeroInteractions(commitTimer) cleanUpFuture.complete(null) // just to unblock shared executor @@ -827,6 +846,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) val skippedCounter = mock[Gauge[Int]] when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + val lastCommitGauge = mock[Gauge[Long]] + when(this.metrics.lastCommitNs).thenReturn(lastCommitGauge) val inputOffsets = new util.HashMap[SystemStreamPartition, String]() inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") @@ -888,6 +909,8 @@ class TestTaskInstance extends AssertionsForJUnit with MockitoSugar { when(this.metrics.asyncCleanupNs).thenReturn(cleanUpTimer) val skippedCounter = mock[Gauge[Int]] when(this.metrics.commitsSkipped).thenReturn(skippedCounter) + val lastCommitGauge = mock[Gauge[Long]] + when(this.metrics.lastCommitNs).thenReturn(lastCommitGauge) val inputOffsets = new util.HashMap[SystemStreamPartition, String]() inputOffsets.put(SYSTEM_STREAM_PARTITION,"4") diff --git a/samza-kafka/src/main/java/org/apache/samza/checkpoint/kafka/KafkaCheckpointLogKey.java b/samza-kafka/src/main/java/org/apache/samza/checkpoint/kafka/KafkaCheckpointLogKey.java index 0f0b071c5c..01c22f55ac 100644 --- a/samza-kafka/src/main/java/org/apache/samza/checkpoint/kafka/KafkaCheckpointLogKey.java +++ b/samza-kafka/src/main/java/org/apache/samza/checkpoint/kafka/KafkaCheckpointLogKey.java @@ -19,6 +19,8 @@ package org.apache.samza.checkpoint.kafka; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import java.util.Map; import org.apache.samza.container.TaskName; /** @@ -28,6 +30,10 @@ public class KafkaCheckpointLogKey { public static final String CHECKPOINT_V1_KEY_TYPE = "checkpoint"; public static final String CHECKPOINT_V2_KEY_TYPE = "checkpoint-v2"; + public static final Map CHECKPOINT_KEY_VERSIONS = ImmutableMap.of( + CHECKPOINT_V1_KEY_TYPE, (short) 1, + CHECKPOINT_V2_KEY_TYPE, (short) 2 + ); /** * The SystemStreamPartitionGrouperFactory configured for this job run. Since, checkpoints of different * groupers are not compatible, we persist and validate them across job runs. diff --git a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala index a83a34ba2c..e719439b2a 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala @@ -34,6 +34,7 @@ import org.apache.samza.system.kafka.KafkaStreamSpec import org.apache.samza.util.Logging import org.apache.samza.{Partition, SamzaException} +import java.{lang, util} import scala.collection.mutable /** @@ -80,7 +81,7 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, // for active containers, this will be set to true, while false for standby containers. val stopConsumerAfterFirstRead: Boolean = new TaskConfig(config).getCheckpointManagerConsumerStopAfterFirstRead - val checkpointReadVersion: Short = new TaskConfig(config).getCheckpointReadVersion + val checkpointReadVersions: util.List[lang.Short] = new TaskConfig(config).getCheckpointReadVersions /** * Create checkpoint stream prior to start. @@ -287,16 +288,17 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, val msgBytes = checkpointEnvelope.getMessage.asInstanceOf[Array[Byte]] try { // if checkpoint key version does not match configured checkpoint version to read, skip the message. - if (checkpointReadVersion == CheckpointV1.CHECKPOINT_VERSION && - KafkaCheckpointLogKey.CHECKPOINT_V1_KEY_TYPE.equals(checkpointKey.getType)) { - val msgBytes = checkpointEnvelope.getMessage.asInstanceOf[Array[Byte]] - val checkpoint = checkpointV1MsgSerde.fromBytes(msgBytes) - checkpoints.put(checkpointKey.getTaskName, checkpoint) - } else if (checkpointReadVersion == CheckpointV2.CHECKPOINT_VERSION && - KafkaCheckpointLogKey.CHECKPOINT_V2_KEY_TYPE.equals(checkpointKey.getType)) { - val checkpoint = checkpointV2MsgSerde.fromBytes(msgBytes) - checkpoints.put(checkpointKey.getTaskName, checkpoint) - } // else ignore and skip the message + if (checkpointReadVersions.contains( + KafkaCheckpointLogKey.CHECKPOINT_KEY_VERSIONS.get(checkpointKey.getType))) { + if (!checkpoints.contains(checkpointKey.getTaskName) || + shouldOverrideCheckpoint(checkpoints.get(checkpointKey.getTaskName), checkpointKey)) { + checkpoints.put(checkpointKey.getTaskName, deserializeCheckpoint(checkpointKey, msgBytes)) + } // else ignore the de-prioritized checkpoint + } else { + // Ignore and skip the unknown checkpoint key type. We do not want to throw any exceptions for this case + // for forwards compatibility with new checkpoints versions in the checkpoint topic + warn(s"Ignoring unknown checkpoint key type for checkpoint key: $checkpointKey") + } } catch { case e: Exception => if (validateCheckpoint) { @@ -372,4 +374,29 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, case _ => throw new SamzaException("Unknown checkpoint version: " + checkpoint.getVersion) } } + + private def shouldOverrideCheckpoint(currentCheckpoint: Option[Checkpoint], + newCheckpointKey: KafkaCheckpointLogKey): Boolean = { + val newCheckpointVersion = KafkaCheckpointLogKey.CHECKPOINT_KEY_VERSIONS.get(newCheckpointKey.getType) + if (newCheckpointVersion == null) { + // Unknown checkpoint version + throw new IllegalArgumentException("Unknown checkpoint key type: " + newCheckpointKey.getType + + " for checkpoint key: " + newCheckpointKey) + } + // Override checkpoint if the current checkpoint does not exist or if new checkpoint has a higher restore + // priority than the currently written checkpoint + currentCheckpoint.isEmpty || + checkpointReadVersions.indexOf(newCheckpointVersion) <= + checkpointReadVersions.indexOf(currentCheckpoint.get.getVersion) + } + + private def deserializeCheckpoint(checkpointKey: KafkaCheckpointLogKey, checkpointMsgBytes: Array[Byte]): Checkpoint = { + if (KafkaCheckpointLogKey.CHECKPOINT_V1_KEY_TYPE.equals(checkpointKey.getType)) { + checkpointV1MsgSerde.fromBytes(checkpointMsgBytes) + } else if (KafkaCheckpointLogKey.CHECKPOINT_V2_KEY_TYPE.equals(checkpointKey.getType)) { + checkpointV2MsgSerde.fromBytes(checkpointMsgBytes) + } else { + throw new IllegalArgumentException("Unknown checkpoint key type: " + checkpointKey.getType) + } + } } diff --git a/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala index f9402921d8..866904b7d8 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala @@ -194,7 +194,7 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { .put(s"systems.$checkpointSystemName.producer.bootstrap.servers", brokerList) .put(s"systems.$checkpointSystemName.consumer.zookeeper.connect", zkConnect) .put("task.checkpoint.system", checkpointSystemName) - .put(TaskConfig.CHECKPOINT_READ_VERSION, "2") + .put(TaskConfig.CHECKPOINT_READ_VERSIONS, "2") .build()) // Skips reading any v1 checkpoints @@ -210,6 +210,59 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { assertEquals(checkpointV2, readCheckpoint(checkpointTopic, taskName, overrideConfig)) } + @Test + def testCheckpointV1AndV2WriteAndReadV1V2PrecedenceList(): Unit = { + val checkpointTopic = "checkpoint-topic-1" + val kcm1 = createKafkaCheckpointManager(checkpointTopic) + kcm1.register(taskName) + kcm1.createResources + kcm1.start + kcm1.stop + + // check that start actually creates the topic with log compaction enabled + val topicConfig = adminZkClient.getAllTopicConfigs().getOrElse(checkpointTopic, new Properties()) + + assertEquals(topicConfig, new KafkaConfig(config).getCheckpointTopicProperties()) + assertEquals("compact", topicConfig.get("cleanup.policy")) + assertEquals("26214400", topicConfig.get("segment.bytes")) + + // read before topic exists should result in a null checkpoint + val readCp = readCheckpoint(checkpointTopic, taskName) + assertNull(readCp) + + val checkpointV1 = new CheckpointV1(ImmutableMap.of(ssp, "offset-1")) + val checkpointV2 = new CheckpointV2(CheckpointId.create(), ImmutableMap.of(ssp, "offset-2"), + ImmutableMap.of("factory1", ImmutableMap.of("store1", "changelogOffset"))) + + val overrideConfig = new MapConfig(new ImmutableMap.Builder[String, String]() + .put(JobConfig.JOB_NAME, "some-job-name") + .put(JobConfig.JOB_ID, "i001") + .put(s"systems.$checkpointSystemName.samza.factory", classOf[KafkaSystemFactory].getCanonicalName) + .put(s"systems.$checkpointSystemName.producer.bootstrap.servers", brokerList) + .put(s"systems.$checkpointSystemName.consumer.zookeeper.connect", zkConnect) + .put("task.checkpoint.system", checkpointSystemName) + .put(TaskConfig.CHECKPOINT_READ_VERSIONS, "2,1") + .build()) + + // Still reads any v1 checkpoints due to precedence list + writeCheckpoint(checkpointTopic, taskName, checkpointV1) + assertEquals(checkpointV1, readCheckpoint(checkpointTopic, taskName, overrideConfig)) + + // writing a v2 checkpoint would allow reading it back + writeCheckpoint(checkpointTopic, taskName, checkpointV2) + assertEquals(checkpointV2, readCheckpoint(checkpointTopic, taskName, overrideConfig)) + + // writing v1 checkpoint is still skipped + writeCheckpoint(checkpointTopic, taskName, checkpointV1) + assertEquals(checkpointV2, readCheckpoint(checkpointTopic, taskName, overrideConfig)) + + val newCheckpointV2 = new CheckpointV2(CheckpointId.create(), ImmutableMap.of(ssp, "offset-3"), + ImmutableMap.of("factory1", ImmutableMap.of("store1", "changelogOffset"))) + // writing v2 returns a new checkpoint v2 + writeCheckpoint(checkpointTopic, taskName, newCheckpointV2) + assertEquals(newCheckpointV2, readCheckpoint(checkpointTopic, taskName, overrideConfig)) + } + @Test def testCheckpointValidationSkipped(): Unit = { val checkpointTopic = "checkpoint-topic-1" diff --git a/samza-test/src/test/java/org/apache/samza/checkpoint/CheckpointVersionIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/checkpoint/CheckpointVersionIntegrationTest.java index d0f1c2f0b6..66eb79c551 100644 --- a/samza-test/src/test/java/org/apache/samza/checkpoint/CheckpointVersionIntegrationTest.java +++ b/samza-test/src/test/java/org/apache/samza/checkpoint/CheckpointVersionIntegrationTest.java @@ -84,7 +84,7 @@ public void testStopCheckpointV1V2AndRestartCheckpointV2() { Arrays.asList(null, null, "98", "99", "4", "5", "5"); List expectedInitialStoreContentsOnSecondRun = Arrays.asList("1", "2", "3"); Map configOverrides = new HashMap<>(CONFIGS); - configOverrides.put(TaskConfig.CHECKPOINT_READ_VERSION, "2"); + configOverrides.put(TaskConfig.CHECKPOINT_READ_VERSIONS, "2"); secondRun(CHANGELOG_TOPIC, expectedChangelogMessagesAfterSecondRun, expectedInitialStoreContentsOnSecondRun, configOverrides); } From 7676da930a9b504be77d4010747be8ae9a20628b Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Tue, 29 Jun 2021 21:52:52 -0700 Subject: [PATCH 13/20] Per store restore manager config precendence RB=2675917 --- .../apache/samza/config/StorageConfig.java | 48 ++++-- .../apache/samza/storage/StorageRecovery.java | 9 +- .../samza/container/SamzaContainer.scala | 25 ++- .../storage/ContainerStorageManager.java | 145 +++++++++++++----- .../samza/config/TestStorageConfig.java | 48 ++++++ .../storage/TestContainerStorageManager.java | 4 +- .../storage/TestTaskStorageManager.scala | 3 +- 7 files changed, 222 insertions(+), 60 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java index 7535d658c5..07ecd422bc 100644 --- a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java @@ -73,9 +73,10 @@ public class StorageConfig extends MapConfig { public static final List DEFAULT_BACKUP_FACTORIES = ImmutableList.of( KAFKA_STATE_BACKEND_FACTORY); public static final String STORE_BACKUP_FACTORIES = STORE_PREFIX + "%s.backup.factories"; - // TODO BLOCKER dchen make this per store - public static final String RESTORE_FACTORY_SUFFIX = "restore.factory"; - public static final String STORE_RESTORE_FACTORY = STORE_PREFIX + RESTORE_FACTORY_SUFFIX; + public static final String RESTORE_FACTORIES_SUFFIX = "restore.factories"; + public static final String STORE_RESTORE_FACTORY = STORE_PREFIX + "%s." + RESTORE_FACTORIES_SUFFIX; + public static final String JOB_RESTORE_FACTORIES = STORE_PREFIX + RESTORE_FACTORIES_SUFFIX; + public static final List DEFAULT_RESTORE_FACTORIES = ImmutableList.of(KAFKA_STATE_BACKEND_FACTORY); static final String CHANGELOG_SYSTEM = "job.changelog.system"; static final String CHANGELOG_DELETE_RETENTION_MS = STORE_PREFIX + "%s.changelog.delete.retention.ms"; @@ -101,8 +102,7 @@ public List getStoreNames() { for (String key : subConfig.keySet()) { if (key.endsWith(SIDE_INPUT_PROCESSOR_FACTORY_SUFFIX)) { storeNames.add(key.substring(0, key.length() - SIDE_INPUT_PROCESSOR_FACTORY_SUFFIX.length())); - } else if (key.endsWith(FACTORY_SUFFIX) && !key.equals(RESTORE_FACTORY_SUFFIX)) { - // TODO HIGH dchen STORE_RESTORE_FACTORY added here to be ignored. Update/remove after changing restore factory -> factories + } else if (key.endsWith(FACTORY_SUFFIX)) { storeNames.add(key.substring(0, key.length() - FACTORY_SUFFIX.length())); } } @@ -305,10 +305,6 @@ public Set getBackupFactories() { .collect(Collectors.toSet()); } - public String getRestoreFactory() { - return get(STORE_RESTORE_FACTORY, KAFKA_STATE_BACKEND_FACTORY); - } - public List getStoresWithBackupFactory(String backendFactoryName) { return getStoreNames().stream() .filter((storeName) -> getStoreBackupFactory(storeName) @@ -316,16 +312,44 @@ public List getStoresWithBackupFactory(String backendFactoryName) { .collect(Collectors.toList()); } - // TODO BLOCKER dchen update when making restore managers per store + public List getJobStoreRestoreFactory() { + return getList(JOB_RESTORE_FACTORIES, new ArrayList<>()); + } + + /** + * Restore state backend factory follows the precedence: + * + * 1. If stores.store-name.restore.factories is set for the store-name, that value is used + * 2. If stores.restore.factories is set for the job, that value is used + * 3. If stores.store-name.changelog is set for store-name, the default Kafka changelog state backend factory + * 4. Otherwise no restore factories will be configured for the store + * + * @return List of restore factories for the store in order of restoration precedence + */ + public List getStoreRestoreFactories(String storeName) { + List storeRestoreManagers = getList(String.format(STORE_RESTORE_FACTORY, storeName), + getJobStoreRestoreFactory()); + // for backwards compatibility if changelog is enabled, we use default Kafka backup factory + if (storeRestoreManagers.isEmpty() && getChangelogStream(storeName).isPresent()) { + storeRestoreManagers = DEFAULT_RESTORE_FACTORIES; + } + return storeRestoreManagers; + } + + public Set getRestoreFactories() { + return getStoreNames().stream() + .flatMap(((storesName) -> getStoreRestoreFactories(storesName).stream())) + .collect(Collectors.toSet()); + } + public List getStoresWithRestoreFactory(String backendFactoryName) { return getStoreNames().stream() - .filter((storeName) -> getRestoreFactory().equals(backendFactoryName)) + .filter((storeName) -> getStoreRestoreFactories(storeName).contains(backendFactoryName)) .collect(Collectors.toList()); } /** * Helper method to get if logged store dirs should be deleted regardless of their contents. - * @return */ public boolean cleanLoggedStoreDirsOnStart(String storeName) { return getBoolean(String.format(CLEAN_LOGGED_STOREDIRS_ON_START, storeName), false); diff --git a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java index 8091e482ef..f1d7ce2d29 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java +++ b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java @@ -24,6 +24,8 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; import org.apache.samza.SamzaException; import org.apache.samza.checkpoint.CheckpointManager; import org.apache.samza.config.Config; @@ -215,7 +217,10 @@ private Map> getSerdes() { */ @SuppressWarnings("rawtypes") private void getContainerStorageManagers() { - String factoryClass = new StorageConfig(jobConfig).getRestoreFactory(); + Set factoryClasses = new StorageConfig(jobConfig).getRestoreFactories(); + Map stateBackendFactories = factoryClasses.stream().collect( + Collectors.toMap(factoryClass -> factoryClass, + factoryClass -> ReflectionUtil.getObj(factoryClass, StateBackendFactory.class))); Clock clock = SystemClock.instance(); StreamMetadataCache streamMetadataCache = new StreamMetadataCache(systemAdmins, 5000, clock); // don't worry about prefetching for this; looks like the tool doesn't flush to offset files anyways @@ -242,7 +247,7 @@ private void getContainerStorageManagers() { new SamzaContainerMetrics(containerModel.getId(), new MetricsRegistryMap(), ""), JobContextImpl.fromConfigWithDefaults(jobConfig, jobModel), containerContext, - ReflectionUtil.getObj(factoryClass, StateBackendFactory.class), + stateBackendFactories, new HashMap<>(), storeBaseDir, storeBaseDir, diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index 17167b1036..e6b0d7834c 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -28,9 +28,9 @@ import java.util import java.util.concurrent._ import java.util.function.Consumer import java.util.{Base64, Optional} - import com.google.common.annotations.VisibleForTesting import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.commons.lang.builder.ReflectionToStringBuilder import org.apache.samza.SamzaException import org.apache.samza.checkpoint.{CheckpointListener, OffsetManager, OffsetManagerMetrics} import org.apache.samza.clustermanager.StandbyTaskUtil @@ -502,9 +502,20 @@ object SamzaContainer extends Logging { val loggedStorageBaseDir = getLoggedStorageBaseDir(jobConfig, defaultStoreBaseDir) info("Got base directory for logged data stores: %s" format loggedStorageBaseDir) - // TODO dchen should we enforce restore factories to be subset of backup factories? - val stateStorageBackendRestoreFactory = ReflectionUtil - .getObj(storageConfig.getRestoreFactory(), classOf[StateBackendFactory]) + val backupFactoryNames = storageConfig.getBackupFactories + val restoreFactoryNames = storageConfig.getRestoreFactories + + // Restore factories should be a subset of backup factories + if (!backupFactoryNames.containsAll(restoreFactoryNames)) { + throw new SamzaException("Restore state backend factories is not a subset of backup state backend factories") + } + + val stateStorageBackendBackupFactories = backupFactoryNames.asScala.map( + ReflectionUtil.getObj(_, classOf[StateBackendFactory]) + ) + val stateStorageBackendRestoreFactories = restoreFactoryNames.asScala.map( + factoryName => factoryName -> ReflectionUtil.getObj(factoryName, classOf[StateBackendFactory]) + ).toMap.asJava val containerStorageManager = new ContainerStorageManager( checkpointManager, @@ -521,7 +532,7 @@ object SamzaContainer extends Logging { samzaContainerMetrics, jobContext, containerContext, - stateStorageBackendRestoreFactory, + stateStorageBackendRestoreFactories, taskCollectors.asJava, loggedStorageBaseDir, nonLoggedStorageBaseDir, @@ -530,9 +541,7 @@ object SamzaContainer extends Logging { storeWatchPaths.addAll(containerStorageManager.getStoreDirectoryPaths) - val stateStorageBackendBackupFactories = storageConfig.getBackupFactories().asScala.map( - ReflectionUtil.getObj(_, classOf[StateBackendFactory]) - ) + // Create taskInstances val taskInstances: Map[TaskName, TaskInstance] = taskModels diff --git a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java index 08ae375e0a..cd26d58504 100644 --- a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java +++ b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java @@ -44,6 +44,7 @@ import org.apache.samza.SamzaException; import org.apache.samza.checkpoint.Checkpoint; import org.apache.samza.checkpoint.CheckpointManager; +import org.apache.samza.checkpoint.CheckpointV2; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.StorageConfig; @@ -120,7 +121,6 @@ public class ContainerStorageManager { private static final int RESTORE_THREAD_POOL_SHUTDOWN_TIMEOUT_SECONDS = 60; /** Maps containing relevant per-task objects */ - private final Map taskRestoreManagers; private final Map taskInstanceMetrics; private final Map taskInstanceCollectors; private final Map> inMemoryStores; // subset of taskStores after #start() @@ -131,6 +131,8 @@ public class ContainerStorageManager { private final Map changelogSystemStreams; // Map of changelog system-streams indexed by store name private final Map> serdes; // Map of Serde objects indexed by serde name (specified in config) private final SystemAdmins systemAdmins; + private final Clock clock; + private final Map stateBackendFactories; private final StreamMetadataCache streamMetadataCache; private final SamzaContainerMetrics samzaContainerMetrics; @@ -153,7 +155,7 @@ public class ContainerStorageManager { private final Set sideInputStoreNames; private final Map sspSideInputHandlers; private SystemConsumers sideInputSystemConsumers; - private volatile Map sideInputTaskLatches; // Used by the sideInput-read thread to signal to the main thread + private final Map sideInputTaskLatches; // Used by the sideInput-read thread to signal to the main thread private volatile boolean shouldShutdown = false; private RunLoop sideInputRunLoop; @@ -183,7 +185,7 @@ public ContainerStorageManager( SamzaContainerMetrics samzaContainerMetrics, JobContext jobContext, ContainerContext containerContext, - StateBackendFactory stateBackendFactory, + Map stateBackendFactories, Map taskInstanceCollectors, File loggedStoreBaseDirectory, File nonLoggedStoreBaseDirectory, @@ -203,6 +205,8 @@ public ContainerStorageManager( LOG.info("Starting with changelogSystemStreams = {} taskSideInputStoreSSPs = {}", this.changelogSystemStreams, this.taskSideInputStoreSSPs); + this.clock = clock; + this.stateBackendFactories = stateBackendFactories; this.storageEngineFactories = storageEngineFactories; this.serdes = serdes; this.loggedStoreBaseDirectory = loggedStoreBaseDirectory; @@ -260,16 +264,13 @@ public ContainerStorageManager( JobConfig jobConfig = new JobConfig(config); int restoreThreadPoolSize = Math.min( - Math.max(containerModel.getTasks().size() * 2, jobConfig.getRestoreThreadPoolSize()), + Math.max(containerModel.getTasks().size() * stateBackendFactories.size() + 1, + jobConfig.getRestoreThreadPoolSize()), jobConfig.getRestoreThreadPoolMaxSize() ); this.restoreExecutor = Executors.newFixedThreadPool(restoreThreadPoolSize, new ThreadFactoryBuilder().setDaemon(true).setNameFormat(RESTORE_THREAD_NAME).build()); - // creating task restore managers - this.taskRestoreManagers = createTaskRestoreManagers(stateBackendFactory, clock, - this.samzaContainerMetrics); - this.sspSideInputHandlers = createSideInputHandlers(clock); // create SystemConsumers for consuming from taskSideInputSSPs, if sideInputs are being used @@ -393,27 +394,82 @@ private static Map createStoreIndexedMap(Map createTaskRestoreManagers(StateBackendFactory factory, Clock clock, - SamzaContainerMetrics samzaContainerMetrics) { - Map taskRestoreManagers = new HashMap<>(); + private Map createTaskRestoreManagers(Map factories, + Map> backendFactoryStoreNames, Clock clock, SamzaContainerMetrics samzaContainerMetrics, TaskName taskName, + TaskModel taskModel) { + // Get the factories for the task based on the stores of the tasks to be restored from the factory + Map backendFactoryRestoreManagers = new HashMap<>(); // backendFactoryName -> restoreManager + MetricsRegistry taskMetricsRegistry = + taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry() : new MetricsRegistryMap(); - containerModel.getTasks().forEach((taskName, taskModel) -> { - MetricsRegistry taskMetricsRegistry = - taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry() : new MetricsRegistryMap(); - Set nonSideInputStoreNames = storageEngineFactories.keySet().stream() - .filter(storeName -> !sideInputStoreNames.contains(storeName)) - .collect(Collectors.toSet()); + backendFactoryStoreNames.forEach((factoryName, storeNames) -> { + StateBackendFactory factory = factories.get(factoryName); KafkaChangelogRestoreParams kafkaChangelogRestoreParams = new KafkaChangelogRestoreParams(storeConsumers, inMemoryStores.get(taskName), systemAdmins.getSystemAdmins(), storageEngineFactories, serdes, - taskInstanceCollectors.get(taskName), nonSideInputStoreNames); + taskInstanceCollectors.get(taskName), storeNames); + TaskRestoreManager restoreManager = factory.getRestoreManager(jobContext, containerContext, taskModel, restoreExecutor, + taskMetricsRegistry, config, clock, loggedStoreBaseDirectory, nonLoggedStoreBaseDirectory, + kafkaChangelogRestoreParams); - taskRestoreManagers.put(taskName, - factory.getRestoreManager(jobContext, containerContext, taskModel, restoreExecutor, - taskMetricsRegistry, config, clock, loggedStoreBaseDirectory, nonLoggedStoreBaseDirectory, - kafkaChangelogRestoreParams)); - samzaContainerMetrics.addStoresRestorationGauge(taskName); + backendFactoryRestoreManagers.put(factoryName, restoreManager); }); - return taskRestoreManagers; + samzaContainerMetrics.addStoresRestorationGauge(taskName); + return backendFactoryRestoreManagers; + } + + /** + * Return a map of backend factory names to set of stores that should be restored using it + */ + private Map> getBackendFactoryStoreNames(Checkpoint checkpoint, Set storeNames, + StorageConfig storageConfig) { + Map> backendFactoryStoreNames = new HashMap<>(); // backendFactoryName -> set(storeNames) + + if (checkpoint != null && checkpoint.getVersion() == 1) { + // Default to changelog backend factory when using checkpoint v1 for backwards compatibility + backendFactoryStoreNames.put(StorageConfig.KAFKA_STATE_BACKEND_FACTORY, storeNames); + } else if (checkpoint == null || checkpoint.getVersion() == 2) { + // Extract the state checkpoint markers if checkpoint exists + Map> stateCheckpointMarkers = checkpoint == null ? Collections.emptyMap() : + ((CheckpointV2) checkpoint).getStateCheckpointMarkers(); + + // Find stores associated to each state backend factory + storeNames.forEach(storeName -> { + List storeFactories = storageConfig.getStoreRestoreFactories(storeName); + boolean storeCheckpointFound = false; + + // Search the ordered list for the first matched state backend factory in the checkpoint + // If the checkpoint does not exist or state checkpoint markers does not exist, we match the first configured + // restore manager + for (String factoryName : storeFactories) { + if (stateCheckpointMarkers.containsKey(factoryName) && + stateCheckpointMarkers.get(factoryName).containsKey(storeName)) { + if (!backendFactoryStoreNames.containsKey(factoryName)) { + backendFactoryStoreNames.put(factoryName, new HashSet<>()); + } + backendFactoryStoreNames.get(factoryName).add(storeName); + storeCheckpointFound = true; + break; + } + } + + if (storeFactories.isEmpty()) { + // If the restore factory is not configured for the store and the store does not have a changelog topic + LOG.warn("non-Side input store: {}, does not have a configured restore factories nor store changelogs," + + "restore for the store will be skipped", + storeName); + } else if (!storeCheckpointFound) { // restore factories configured but no checkpoints found + // Use first configured restore factory + String factoryName = storeFactories.get(0); + if (!backendFactoryStoreNames.containsKey(factoryName)) { + backendFactoryStoreNames.put(factoryName, new HashSet<>()); + } + backendFactoryStoreNames.get(factoryName).add(storeName); + } + }); + } else { + throw new SamzaException(String.format("Unsupported checkpoint version %s", checkpoint.getVersion())); + } + return backendFactoryStoreNames; } // Helper method to filter active Tasks from the container model @@ -646,30 +702,50 @@ public void start() throws SamzaException, InterruptedException { private void restoreStores() throws InterruptedException { LOG.info("Store Restore started"); Set activeTasks = getTasks(containerModel, TaskMode.Active).keySet(); + // TODO dchen verify davinci lifecycle + // Find all non-side input stores + Set nonSideInputStoreNames = storageEngineFactories.keySet() + .stream() + .filter(storeName -> !sideInputStoreNames.contains(storeName)) + .collect(Collectors.toSet()); - // initialize each TaskStorageManager - this.taskRestoreManagers.forEach((taskName, taskRestoreManager) -> { + // Obtain the checkpoints for each task + Map> taskRestoreManagers = new HashMap<>(); + Map taskCheckpoints = new HashMap<>(); + containerModel.getTasks().forEach((taskName, taskModel) -> { Checkpoint taskCheckpoint = null; if (checkpointManager != null && activeTasks.contains(taskName)) { // only pass in checkpoints for active tasks taskCheckpoint = checkpointManager.readLastCheckpoint(taskName); LOG.info("Obtained checkpoint: {} for state restore for taskName: {}", taskCheckpoint, taskName); } - taskRestoreManager.init(taskCheckpoint); + taskCheckpoints.put(taskName, taskCheckpoint); + Map> backendFactoryStoreNames = getBackendFactoryStoreNames(taskCheckpoint, nonSideInputStoreNames, + new StorageConfig(config)); + Map taskStoreRestoreManagers = createTaskRestoreManagers(stateBackendFactories, + backendFactoryStoreNames, clock, samzaContainerMetrics, taskName, taskModel); + taskRestoreManagers.put(taskName, taskStoreRestoreManagers); }); + // Initialize each TaskStorageManager + taskRestoreManagers.forEach((taskName, restoreManagers) -> + restoreManagers.forEach((factoryName, taskRestoreManager) -> + taskRestoreManager.init(taskCheckpoints.get(taskName)) + ) + ); + // Start each store consumer once this.storeConsumers.values().stream().distinct().forEach(SystemConsumer::start); - List taskRestoreFutures = new ArrayList<>(this.taskRestoreManagers.entrySet().size()); + List taskRestoreFutures = new ArrayList<>(); // Submit restore callable for each taskInstance - this.taskRestoreManagers.forEach((taskInstance, taskRestoreManager) -> { - taskRestoreFutures.add(restoreExecutor.submit( - new TaskRestoreCallable(this.samzaContainerMetrics, taskInstance, taskRestoreManager))); - }); + taskRestoreManagers.forEach((taskInstance, restoreManagersMap) -> + // Submit for each restore factory + restoreManagersMap.forEach((factoryName, taskRestoreManager) -> taskRestoreFutures.add(restoreExecutor.submit( + new TaskRestoreCallable(this.samzaContainerMetrics, taskInstance, taskRestoreManager))))); - // loop-over the future list to wait for each thread to finish, catch any exceptions during restore and throw + // Loop-over the future list to wait for each thread to finish, catch any exceptions during restore and throw // as samza exceptions for (Future future : taskRestoreFutures) { try { @@ -689,9 +765,6 @@ private void restoreStores() throws InterruptedException { this.storeConsumers.values().stream().distinct().forEach(SystemConsumer::stop); // Now create persistent non side input stores in read-write mode, leave non-persistent stores as-is - Set nonSideInputStoreNames = storageEngineFactories.keySet().stream() - .filter(storeName -> !sideInputStoreNames.contains(storeName)) - .collect(Collectors.toSet()); this.taskStores = createTaskStores(nonSideInputStoreNames, this.containerModel, jobContext, containerContext, storageEngineFactories, serdes, taskInstanceMetrics, taskInstanceCollectors); // Add in memory stores diff --git a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java index 3a0debce98..afbd9f8a13 100644 --- a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java +++ b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java @@ -431,4 +431,52 @@ public void testGetChangelogMinCompactionLagMs() { configMap.put(String.format(CHANGELOG_MIN_COMPACTION_LAG_MS, STORE_NAME0), String.valueOf(storeSpecificLagOverride)); assertEquals(storeSpecificLagOverride, new StorageConfig(new MapConfig(configMap)).getChangelogMinCompactionLagMs(STORE_NAME0)); } + + @Test + public void testGetRestoreManagers() { + String storeName = "store1"; + String storeName2 = "store2"; + Map configMap = new HashMap<>(); + configMap.put(String.format(FACTORY, storeName), "store1.factory.class"); + configMap.put(String.format(FACTORY, storeName2), "store2.factory.class"); + + // empty config, return no restore managers + assertEquals(Collections.emptySet(), new StorageConfig(new MapConfig(configMap)).getRestoreFactories()); + assertEquals(Collections.emptyList(), new StorageConfig(new MapConfig(configMap)).getStoreRestoreFactories(storeName)); + assertEquals(Collections.emptyList(), new StorageConfig(new MapConfig(configMap)).getStoreRestoreFactories(storeName2)); + + // changelog set, should default to kafka state backend restore + String changelogStreamOverride = "changelogStream"; + configMap.put(String.format(CHANGELOG_STREAM, storeName), changelogStreamOverride); + configMap.put(String.format(CHANGELOG_STREAM, storeName2), changelogStreamOverride); + configMap.put(StorageConfig.CHANGELOG_SYSTEM, "changelog-system"); + configMap.put(String.format(StorageConfig.CHANGELOG_STREAM, storeName), "changelog-stream0"); + assertEquals(ImmutableSet.of(KAFKA_STATE_BACKEND_FACTORY), new StorageConfig(new MapConfig(configMap)).getRestoreFactories()); + assertEquals(DEFAULT_RESTORE_FACTORIES, new StorageConfig(new MapConfig(configMap)).getStoreRestoreFactories(storeName)); + assertEquals(DEFAULT_RESTORE_FACTORIES, new StorageConfig(new MapConfig(configMap)).getStoreRestoreFactories(storeName2)); + + // job restore manager config set should override to job backend factory + String jobRestoreFactory1 = "jobBackendRestoreFactory1"; + String jobRestoreFactory2 = "jobBackendRestoreFactory2"; + String jobRestoreFactoryOverride = jobRestoreFactory1 + "," + jobRestoreFactory2; + configMap.put(JOB_RESTORE_FACTORIES, jobRestoreFactoryOverride); + assertEquals(ImmutableSet.of(jobRestoreFactory1, jobRestoreFactory2), + new StorageConfig(new MapConfig(configMap)).getRestoreFactories()); + assertEquals(ImmutableList.of(jobRestoreFactory1, jobRestoreFactory2), + new StorageConfig(new MapConfig(configMap)).getStoreRestoreFactories(storeName)); + assertEquals(ImmutableList.of(jobRestoreFactory1, jobRestoreFactory2), + new StorageConfig(new MapConfig(configMap)).getStoreRestoreFactories(storeName2)); + + // store specific restore managers set + String storeRestoreFactory1 = "storeBackendRestoreFactory1"; + String storeRestoreFactory2 = "storeBackendRestoreFactory2"; + String storeRestoreFactoryOverride = storeRestoreFactory1 + "," + storeRestoreFactory2; + configMap.put(String.format(STORE_RESTORE_FACTORY, storeName), storeRestoreFactoryOverride); + assertEquals(ImmutableSet.of(jobRestoreFactory1, jobRestoreFactory2, storeRestoreFactory1, storeRestoreFactory2), + new StorageConfig(new MapConfig(configMap)).getRestoreFactories()); + assertEquals(ImmutableList.of(storeRestoreFactory1, storeRestoreFactory2), + new StorageConfig(new MapConfig(configMap)).getStoreRestoreFactories(storeName)); + assertEquals(ImmutableList.of(jobRestoreFactory1, jobRestoreFactory2), + new StorageConfig(new MapConfig(configMap)).getStoreRestoreFactories(storeName2)); + } } diff --git a/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java b/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java index f381f3d497..e0e5b476cb 100644 --- a/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java +++ b/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java @@ -18,6 +18,7 @@ */ package org.apache.samza.storage; +import com.google.common.collect.ImmutableMap; import java.io.File; import java.util.Arrays; import java.util.HashMap; @@ -28,6 +29,7 @@ import org.apache.samza.checkpoint.CheckpointV1; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; +import org.apache.samza.config.StorageConfig; import org.apache.samza.config.TaskConfig; import org.apache.samza.container.SamzaContainerMetrics; import org.apache.samza.container.TaskInstance; @@ -246,7 +248,7 @@ public Void answer(InvocationOnMock invocation) { samzaContainerMetrics, mock(JobContext.class), mockContainerContext, - backendFactory, + ImmutableMap.of(StorageConfig.KAFKA_STATE_BACKEND_FACTORY, backendFactory), mock(Map.class), DEFAULT_LOGGED_STORE_BASE_DIR, DEFAULT_STORE_BASE_DIR, diff --git a/samza-kafka/src/test/java/org/apache/samza/storage/TestTaskStorageManager.scala b/samza-kafka/src/test/java/org/apache/samza/storage/TestTaskStorageManager.scala index 167bc78e67..b4a2c3581b 100644 --- a/samza-kafka/src/test/java/org/apache/samza/storage/TestTaskStorageManager.scala +++ b/samza-kafka/src/test/java/org/apache/samza/storage/TestTaskStorageManager.scala @@ -800,7 +800,8 @@ class TaskStorageManagerBuilder extends MockitoSugar { Mockito.mock(classOf[SamzaContainerMetrics]), mockJobContext, mockContainerContext, - new mockKafkaChangelogBackendManager(changeLogSystemStreams), + ImmutableMap.of(StorageConfig.KAFKA_STATE_BACKEND_FACTORY, + new mockKafkaChangelogBackendManager(changeLogSystemStreams)), new HashMap[TaskName, TaskInstanceCollector].asJava, loggedStoreBaseDir, TaskStorageManagerBuilder.defaultStoreBaseDir, From 5379f97ae1e93f9af218fc3771d8d85ef4cce3e6 Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Wed, 30 Jun 2021 16:21:07 -0700 Subject: [PATCH 14/20] Job level state backend configs RB=2675917 G=samza-reviewers R=pmaheshw,shesharm A=pmaheshw,shesharm --- .../storage/KafkaChangelogRestoreParams.java | 9 +- .../samza/storage/StateBackendFactory.java | 2 + .../apache/samza/config/StorageConfig.java | 56 +++++-- .../KafkaChangelogStateBackendFactory.java | 5 +- ...nTransactionalStateTaskRestoreManager.java | 1 - .../blobstore/BlobStoreRestoreManager.java | 10 +- .../BlobStoreStateBackendFactory.java | 4 +- .../samza/container/SamzaContainer.scala | 10 +- .../storage/ContainerStorageManager.java | 75 +++++---- .../samza/config/TestStorageConfig.java | 115 ++++++++++++-- .../TestBlobStoreRestoreManager.java | 10 +- .../storage/TestContainerStorageManager.java | 145 +++++++++++++++++- .../kv/BaseKeyValueStorageEngineFactory.java | 4 +- 13 files changed, 359 insertions(+), 87 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/storage/KafkaChangelogRestoreParams.java b/samza-api/src/main/java/org/apache/samza/storage/KafkaChangelogRestoreParams.java index 961a85bb93..728b10f2f1 100644 --- a/samza-api/src/main/java/org/apache/samza/storage/KafkaChangelogRestoreParams.java +++ b/samza-api/src/main/java/org/apache/samza/storage/KafkaChangelogRestoreParams.java @@ -36,7 +36,6 @@ public class KafkaChangelogRestoreParams { private final Map> storageEngineFactories; private final Map> serdes; private final MessageCollector collector; - private final Set storeNames; public KafkaChangelogRestoreParams( Map storeConsumers, @@ -44,15 +43,13 @@ public KafkaChangelogRestoreParams( Map systemAdmins, Map> storageEngineFactories, Map> serdes, - MessageCollector collector, - Set storeNames) { + MessageCollector collector) { this.storeConsumers = storeConsumers; this.inMemoryStores = inMemoryStores; this.systemAdmins = systemAdmins; this.storageEngineFactories = storageEngineFactories; this.serdes = serdes; this.collector = collector; - this.storeNames = storeNames; } public Map getStoreConsumers() { @@ -78,8 +75,4 @@ public Map> getSerdes() { public MessageCollector getCollector() { return collector; } - - public Set getStoreNames() { - return storeNames; - } } diff --git a/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java b/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java index f51d414815..91f3df3d04 100644 --- a/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java +++ b/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java @@ -20,6 +20,7 @@ package org.apache.samza.storage; import java.io.File; +import java.util.Set; import java.util.concurrent.ExecutorService; import org.apache.samza.config.Config; import org.apache.samza.context.ContainerContext; @@ -51,6 +52,7 @@ TaskRestoreManager getRestoreManager(JobContext jobContext, TaskModel taskModel, ExecutorService restoreExecutor, MetricsRegistry metricsRegistry, + Set storesToRestore, Config config, Clock clock, File loggedStoreBaseDir, diff --git a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java index 07ecd422bc..35d6003cfa 100644 --- a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java @@ -72,9 +72,10 @@ public class StorageConfig extends MapConfig { "org.apache.samza.storage.KafkaChangelogStateBackendFactory"; public static final List DEFAULT_BACKUP_FACTORIES = ImmutableList.of( KAFKA_STATE_BACKEND_FACTORY); + public static final String JOB_BACKUP_FACTORIES = STORE_PREFIX + "backup.factories"; public static final String STORE_BACKUP_FACTORIES = STORE_PREFIX + "%s.backup.factories"; public static final String RESTORE_FACTORIES_SUFFIX = "restore.factories"; - public static final String STORE_RESTORE_FACTORY = STORE_PREFIX + "%s." + RESTORE_FACTORIES_SUFFIX; + public static final String STORE_RESTORE_FACTORIES = STORE_PREFIX + "%s." + RESTORE_FACTORIES_SUFFIX; public static final String JOB_RESTORE_FACTORIES = STORE_PREFIX + RESTORE_FACTORIES_SUFFIX; public static final List DEFAULT_RESTORE_FACTORIES = ImmutableList.of(KAFKA_STATE_BACKEND_FACTORY); @@ -290,48 +291,73 @@ public int getNumPersistentStores() { .count(); } - public List getStoreBackupFactory(String storeName) { - List storeBackupManagers = getList(String.format(STORE_BACKUP_FACTORIES, storeName), new ArrayList<>()); - // For backwards compatibility if the changelog is enabled, we use default kafka backup factory - if (storeBackupManagers.isEmpty() && getChangelogStream(storeName).isPresent()) { - storeBackupManagers = DEFAULT_BACKUP_FACTORIES; + private List getJobStoreBackupFactories() { + return getList(JOB_BACKUP_FACTORIES, new ArrayList<>()); + } + + /** + * Backup state backend factory follows the precedence: + * + * 1. If stores.store-name.backup.factories config key exists the store-name, that value is used + * 2. If stores.store-name.changelog is set for store-name, the default Kafka changelog state backend factory + * 3. If stores.backup.factories is set for the job, that value is used + * 4. Otherwise no backup factories will be configured for the store + * + * Note: that 2 takes precedence over 3 enables job based migration off of Changelog restores + * @return List of backup factories for the store in order of backup precedence + */ + public List getStoreBackupFactories(String storeName) { + List storeBackupManagers; + if (containsKey(String.format(STORE_BACKUP_FACTORIES, storeName))) { + storeBackupManagers = getList(String.format(STORE_BACKUP_FACTORIES, storeName), new ArrayList<>()); + } else { + storeBackupManagers = getJobStoreBackupFactories(); + // For backwards compatibility if the changelog is enabled, we use default kafka backup factory + if (storeBackupManagers.isEmpty() && getChangelogStream(storeName).isPresent()) { + storeBackupManagers = DEFAULT_BACKUP_FACTORIES; + } } return storeBackupManagers; } public Set getBackupFactories() { return getStoreNames().stream() - .flatMap((storeName) -> getStoreBackupFactory(storeName).stream()) + .flatMap((storeName) -> getStoreBackupFactories(storeName).stream()) .collect(Collectors.toSet()); } public List getStoresWithBackupFactory(String backendFactoryName) { return getStoreNames().stream() - .filter((storeName) -> getStoreBackupFactory(storeName) + .filter((storeName) -> getStoreBackupFactories(storeName) .contains(backendFactoryName)) .collect(Collectors.toList()); } - public List getJobStoreRestoreFactory() { + private List getJobStoreRestoreFactories() { return getList(JOB_RESTORE_FACTORIES, new ArrayList<>()); } /** * Restore state backend factory follows the precedence: * - * 1. If stores.store-name.restore.factories is set for the store-name, that value is used + * 1. If stores.store-name.restore.factories config key exists for the store-name, that value is used * 2. If stores.restore.factories is set for the job, that value is used * 3. If stores.store-name.changelog is set for store-name, the default Kafka changelog state backend factory * 4. Otherwise no restore factories will be configured for the store * + * Note that 2 takes precedence over 3 enables job based migration off of Changelog restores * @return List of restore factories for the store in order of restoration precedence */ public List getStoreRestoreFactories(String storeName) { - List storeRestoreManagers = getList(String.format(STORE_RESTORE_FACTORY, storeName), - getJobStoreRestoreFactory()); - // for backwards compatibility if changelog is enabled, we use default Kafka backup factory - if (storeRestoreManagers.isEmpty() && getChangelogStream(storeName).isPresent()) { - storeRestoreManagers = DEFAULT_RESTORE_FACTORIES; + List storeRestoreManagers; + if (containsKey(String.format(STORE_RESTORE_FACTORIES, storeName))) { + storeRestoreManagers = getList(String.format(STORE_RESTORE_FACTORIES, storeName), new ArrayList<>()); + } else { + storeRestoreManagers = getJobStoreRestoreFactories(); + // for backwards compatibility if changelog is enabled, we use default Kafka backup factory + if (storeRestoreManagers.isEmpty() && getChangelogStream(storeName).isPresent()) { + storeRestoreManagers = DEFAULT_RESTORE_FACTORIES; + } } return storeRestoreManagers; } diff --git a/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java b/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java index be419b6c65..84339968e0 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java +++ b/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java @@ -91,6 +91,7 @@ public TaskRestoreManager getRestoreManager(JobContext jobContext, TaskModel taskModel, ExecutorService restoreExecutor, MetricsRegistry metricsRegistry, + Set storesToRestore, Config config, Clock clock, File loggedStoreBaseDir, @@ -108,7 +109,7 @@ public TaskRestoreManager getRestoreManager(JobContext jobContext, if (new TaskConfig(config).getTransactionalStateRestoreEnabled()) { return new TransactionalStateTaskRestoreManager( - kafkaChangelogRestoreParams.getStoreNames(), + storesToRestore, jobContext, containerContext, taskModel, @@ -128,7 +129,7 @@ public TaskRestoreManager getRestoreManager(JobContext jobContext, ); } else { return new NonTransactionalStateTaskRestoreManager( - kafkaChangelogRestoreParams.getStoreNames(), + storesToRestore, jobContext, containerContext, taskModel, diff --git a/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java index 35fa375dfd..33af3677bc 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java @@ -326,7 +326,6 @@ private String getStartingOffset(SystemStreamPartition systemStreamPartition, Sy return storageManagerUtil.getStartingOffset(systemStreamPartition, systemAdmin, fileOffset, oldestOffset); } - // TODO dchen put this in common code path for transactional and non-transactional private Map createStoreEngines(Set storeNames, JobContext jobContext, ContainerContext containerContext, Map> storageEngineFactories, Map> serdes, MetricsRegistry metricsRegistry, diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java index fa0beb099a..7eb4e5ddb1 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java @@ -84,7 +84,7 @@ public class BlobStoreRestoreManager implements TaskRestoreManager { private final File loggedBaseDir; private final File nonLoggedBaseDir; private final String taskName; - private final List storesToRestore; + private final Set storesToRestore; private final BlobStoreRestoreManagerMetrics metrics; private BlobStoreManager blobStoreManager; @@ -95,7 +95,7 @@ public class BlobStoreRestoreManager implements TaskRestoreManager { */ private Map> prevStoreSnapshotIndexes; - public BlobStoreRestoreManager(TaskModel taskModel, ExecutorService restoreExecutor, + public BlobStoreRestoreManager(TaskModel taskModel, ExecutorService restoreExecutor, Set storesToRestore, BlobStoreRestoreManagerMetrics metrics, Config config, File loggedBaseDir, File nonLoggedBaseDir, StorageManagerUtil storageManagerUtil, BlobStoreManager blobStoreManager) { this.taskModel = taskModel; @@ -113,9 +113,7 @@ public BlobStoreRestoreManager(TaskModel taskModel, ExecutorService restoreExecu this.loggedBaseDir = loggedBaseDir; this.nonLoggedBaseDir = nonLoggedBaseDir; this.taskName = taskModel.getTaskName().getTaskName(); - StorageConfig storageConfig = new StorageConfig(config); - this.storesToRestore = - storageConfig.getStoresWithRestoreFactory(BlobStoreStateBackendFactory.class.getName()); + this.storesToRestore = storesToRestore; this.metrics = metrics; } @@ -209,7 +207,7 @@ static void deleteUnusedStoresFromBlobStore(String jobName, String jobId, String * Restores all eligible stores in the task. */ @VisibleForTesting - static void restoreStores(String jobName, String jobId, TaskName taskName, List storesToRestore, + static void restoreStores(String jobName, String jobId, TaskName taskName, Set storesToRestore, Map> prevStoreSnapshotIndexes, File loggedBaseDir, StorageConfig storageConfig, BlobStoreRestoreManagerMetrics metrics, StorageManagerUtil storageManagerUtil, BlobStoreUtil blobStoreUtil, DirDiffUtil dirDiffUtil, diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreStateBackendFactory.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreStateBackendFactory.java index 23482c7b44..e2512b4890 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreStateBackendFactory.java +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreStateBackendFactory.java @@ -21,6 +21,7 @@ import com.google.common.base.Preconditions; import java.io.File; +import java.util.Set; import java.util.concurrent.ExecutorService; import org.apache.commons.lang3.StringUtils; import org.apache.samza.config.BlobStoreConfig; @@ -73,6 +74,7 @@ public TaskRestoreManager getRestoreManager( TaskModel taskModel, ExecutorService restoreExecutor, MetricsRegistry metricsRegistry, + Set storesToRestore, Config config, Clock clock, File loggedStoreBaseDir, @@ -84,7 +86,7 @@ public TaskRestoreManager getRestoreManager( BlobStoreManagerFactory factory = ReflectionUtil.getObj(blobStoreManagerFactory, BlobStoreManagerFactory.class); BlobStoreManager blobStoreManager = factory.getRestoreBlobStoreManager(config, restoreExecutor); BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(metricsRegistry); - return new BlobStoreRestoreManager(taskModel, restoreExecutor, metrics, config, loggedStoreBaseDir, + return new BlobStoreRestoreManager(taskModel, restoreExecutor, storesToRestore, metrics, config, loggedStoreBaseDir, nonLoggedStoreBaseDir, new StorageManagerUtil(), blobStoreManager); } diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index e6b0d7834c..fe275528de 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -23,14 +23,12 @@ import java.io.File import java.lang.management.ManagementFactory import java.net.{URL, UnknownHostException} import java.nio.file.Path -import java.time.Duration import java.util import java.util.concurrent._ import java.util.function.Consumer import java.util.{Base64, Optional} import com.google.common.annotations.VisibleForTesting import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.apache.commons.lang.builder.ReflectionToStringBuilder import org.apache.samza.SamzaException import org.apache.samza.checkpoint.{CheckpointListener, OffsetManager, OffsetManagerMetrics} import org.apache.samza.clustermanager.StandbyTaskUtil @@ -507,15 +505,17 @@ object SamzaContainer extends Logging { // Restore factories should be a subset of backup factories if (!backupFactoryNames.containsAll(restoreFactoryNames)) { - throw new SamzaException("Restore state backend factories is not a subset of backup state backend factories") + backupFactoryNames.removeAll(restoreFactoryNames) + throw new SamzaException("Restore state backend factories is not a subset of backup state backend factories, " + + "missing factories: " + backupFactoryNames.toString) } val stateStorageBackendBackupFactories = backupFactoryNames.asScala.map( ReflectionUtil.getObj(_, classOf[StateBackendFactory]) ) val stateStorageBackendRestoreFactories = restoreFactoryNames.asScala.map( - factoryName => factoryName -> ReflectionUtil.getObj(factoryName, classOf[StateBackendFactory]) - ).toMap.asJava + factoryName => (factoryName , ReflectionUtil.getObj(factoryName, classOf[StateBackendFactory]))) + .toMap.asJava val containerStorageManager = new ContainerStorageManager( checkpointManager, diff --git a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java index cd26d58504..2879f9181c 100644 --- a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java +++ b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java @@ -132,7 +132,7 @@ public class ContainerStorageManager { private final Map> serdes; // Map of Serde objects indexed by serde name (specified in config) private final SystemAdmins systemAdmins; private final Clock clock; - private final Map stateBackendFactories; + private final Map restoreStateBackendFactories; private final StreamMetadataCache streamMetadataCache; private final SamzaContainerMetrics samzaContainerMetrics; @@ -185,7 +185,7 @@ public ContainerStorageManager( SamzaContainerMetrics samzaContainerMetrics, JobContext jobContext, ContainerContext containerContext, - Map stateBackendFactories, + Map restoreStateBackendFactories, Map taskInstanceCollectors, File loggedStoreBaseDirectory, File nonLoggedStoreBaseDirectory, @@ -206,7 +206,7 @@ public ContainerStorageManager( LOG.info("Starting with changelogSystemStreams = {} taskSideInputStoreSSPs = {}", this.changelogSystemStreams, this.taskSideInputStoreSSPs); this.clock = clock; - this.stateBackendFactories = stateBackendFactories; + this.restoreStateBackendFactories = restoreStateBackendFactories; this.storageEngineFactories = storageEngineFactories; this.serdes = serdes; this.loggedStoreBaseDirectory = loggedStoreBaseDirectory; @@ -264,7 +264,7 @@ public ContainerStorageManager( JobConfig jobConfig = new JobConfig(config); int restoreThreadPoolSize = Math.min( - Math.max(containerModel.getTasks().size() * stateBackendFactories.size() + 1, + Math.max(containerModel.getTasks().size() * restoreStateBackendFactories.size() * 2, jobConfig.getRestoreThreadPoolSize()), jobConfig.getRestoreThreadPoolMaxSize() ); @@ -406,9 +406,9 @@ private Map createTaskRestoreManagers(Map createTaskRestoreManagers(Map> getBackendFactoryStoreNames(Checkpoint checkpoint, Set storeNames, + @VisibleForTesting + Map> getBackendFactoryStoreNames(Checkpoint checkpoint, Set storeNames, StorageConfig storageConfig) { Map> backendFactoryStoreNames = new HashMap<>(); // backendFactoryName -> set(storeNames) if (checkpoint != null && checkpoint.getVersion() == 1) { + // Only restore stores with changelog streams configured + Set changelogStores = storeNames.stream() + .filter(storeName -> storageConfig.getChangelogStream(storeName).isPresent()) + .collect(Collectors.toSet()); // Default to changelog backend factory when using checkpoint v1 for backwards compatibility - backendFactoryStoreNames.put(StorageConfig.KAFKA_STATE_BACKEND_FACTORY, storeNames); + backendFactoryStoreNames.put(StorageConfig.KAFKA_STATE_BACKEND_FACTORY, changelogStores); + if (storeNames.size() > changelogStores.size()) { + Set nonChangelogStores = storeNames.stream() + .filter(storeName -> !changelogStores.contains(storeName)) + .collect(Collectors.toSet()); + LOG.info("non-Side input stores: {}, do not have a configured store changelogs for checkpoint V1," + + "restore for the store will be skipped", + nonChangelogStores); + } } else if (checkpoint == null || checkpoint.getVersion() == 2) { // Extract the state checkpoint markers if checkpoint exists Map> stateCheckpointMarkers = checkpoint == null ? Collections.emptyMap() : @@ -435,31 +448,29 @@ private Map> getBackendFactoryStoreNames(Checkpoint checkpoi // Find stores associated to each state backend factory storeNames.forEach(storeName -> { List storeFactories = storageConfig.getStoreRestoreFactories(storeName); - boolean storeCheckpointFound = false; - - // Search the ordered list for the first matched state backend factory in the checkpoint - // If the checkpoint does not exist or state checkpoint markers does not exist, we match the first configured - // restore manager - for (String factoryName : storeFactories) { - if (stateCheckpointMarkers.containsKey(factoryName) && - stateCheckpointMarkers.get(factoryName).containsKey(storeName)) { - if (!backendFactoryStoreNames.containsKey(factoryName)) { - backendFactoryStoreNames.put(factoryName, new HashSet<>()); - } - backendFactoryStoreNames.get(factoryName).add(storeName); - storeCheckpointFound = true; - break; - } - } if (storeFactories.isEmpty()) { // If the restore factory is not configured for the store and the store does not have a changelog topic - LOG.warn("non-Side input store: {}, does not have a configured restore factories nor store changelogs," + LOG.info("non-Side input store: {}, does not have a configured restore factories nor store changelogs," + "restore for the store will be skipped", storeName); - } else if (!storeCheckpointFound) { // restore factories configured but no checkpoints found - // Use first configured restore factory - String factoryName = storeFactories.get(0); + } else { + // Search the ordered list for the first matched state backend factory in the checkpoint + // If the checkpoint does not exist or state checkpoint markers does not exist, we match the first configured + // restore manager + Optional factoryNameOpt = storeFactories.stream() + .filter(factoryName -> stateCheckpointMarkers.containsKey(factoryName) && + stateCheckpointMarkers.get(factoryName).containsKey(storeName)) + .findFirst(); + String factoryName; + if (factoryNameOpt.isPresent()) { + factoryName = factoryNameOpt.get(); + } else { // Restore factories configured but no checkpoints found + // Use first configured restore factory + factoryName = storeFactories.get(0); + LOG.warn("No matching checkpoints found for configured factories: {}, " + + "defaulting to using the first configured factory with no checkpoints", storeFactories); + } if (!backendFactoryStoreNames.containsKey(factoryName)) { backendFactoryStoreNames.put(factoryName, new HashSet<>()); } @@ -499,9 +510,9 @@ private Map> createTaskStores(Set s } for (String storeName : storesToCreate) { - List storeBackupManager = storageConfig.getStoreBackupFactory(storeName); + List storeBackupManagers = storageConfig.getStoreBackupFactories(storeName); // A store is considered durable if it is backed by a changelog or another backupManager factory - boolean isDurable = changelogSystemStreams.containsKey(storeName) || !storeBackupManager.isEmpty(); + boolean isDurable = changelogSystemStreams.containsKey(storeName) || !storeBackupManagers.isEmpty(); boolean isSideInput = this.sideInputStoreNames.contains(storeName); // Use the logged-store-base-directory for change logged stores and sideInput stores, and non-logged-store-base-dir // for non logged stores @@ -702,7 +713,7 @@ public void start() throws SamzaException, InterruptedException { private void restoreStores() throws InterruptedException { LOG.info("Store Restore started"); Set activeTasks = getTasks(containerModel, TaskMode.Active).keySet(); - // TODO dchen verify davinci lifecycle + // TODO HIGH dchen verify davinci lifecycle // Find all non-side input stores Set nonSideInputStoreNames = storageEngineFactories.keySet() .stream() @@ -722,7 +733,7 @@ private void restoreStores() throws InterruptedException { taskCheckpoints.put(taskName, taskCheckpoint); Map> backendFactoryStoreNames = getBackendFactoryStoreNames(taskCheckpoint, nonSideInputStoreNames, new StorageConfig(config)); - Map taskStoreRestoreManagers = createTaskRestoreManagers(stateBackendFactories, + Map taskStoreRestoreManagers = createTaskRestoreManagers(restoreStateBackendFactories, backendFactoryStoreNames, clock, samzaContainerMetrics, taskName, taskModel); taskRestoreManagers.put(taskName, taskStoreRestoreManagers); }); diff --git a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java index afbd9f8a13..9a60f12add 100644 --- a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java +++ b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java @@ -72,7 +72,7 @@ public void testGetStoreNames() { } /** - * Test verifies that the {@link StorageConfig#STORE_RESTORE_FACTORY} which matches pattern for store.%s.factory + * Test verifies that the {@link StorageConfig#STORE_RESTORE_FACTORIES} which matches pattern for store.%s.factory * is not picked up as in store names list */ @Test @@ -84,8 +84,7 @@ public void testGetStoreNamesIgnoreStateRestoreFactory() { // has stores StorageConfig storageConfig = new StorageConfig(new MapConfig( ImmutableMap.of(String.format(StorageConfig.FACTORY, STORE_NAME0), "store0.factory.class", - String.format(StorageConfig.FACTORY, STORE_NAME1), "store1.factory.class", - STORE_RESTORE_FACTORY, "org.apache.class"))); + String.format(StorageConfig.FACTORY, STORE_NAME1), "store1.factory.class", STORE_RESTORE_FACTORIES, "org.apache.class"))); List actual = storageConfig.getStoreNames(); // ordering shouldn't matter @@ -204,12 +203,12 @@ public void testGetBackupManagerFactories() { assertTrue(factories.contains(factory3)); assertTrue(factories.contains(KAFKA_STATE_BACKEND_FACTORY)); assertEquals(4, factories.size()); - assertEquals(ImmutableList.of(factory1, factory2), storageConfig.getStoreBackupFactory(STORE_NAME0)); - assertEquals(ImmutableList.of(factory1), storageConfig.getStoreBackupFactory(STORE_NAME1)); - assertEquals(ImmutableList.of(factory3), storageConfig.getStoreBackupFactory(STORE_NAME2)); - assertEquals(DEFAULT_BACKUP_FACTORIES, storageConfig.getStoreBackupFactory(STORE_NAME3)); - assertTrue(storageConfig.getStoreBackupFactory("emptyStore").isEmpty()); - assertTrue(storageConfig.getStoreBackupFactory("noFactoryStore").isEmpty()); + assertEquals(ImmutableList.of(factory1, factory2), storageConfig.getStoreBackupFactories(STORE_NAME0)); + assertEquals(ImmutableList.of(factory1), storageConfig.getStoreBackupFactories(STORE_NAME1)); + assertEquals(ImmutableList.of(factory3), storageConfig.getStoreBackupFactories(STORE_NAME2)); + assertEquals(DEFAULT_BACKUP_FACTORIES, storageConfig.getStoreBackupFactories(STORE_NAME3)); + assertTrue(storageConfig.getStoreBackupFactories("emptyStore").isEmpty()); + assertTrue(storageConfig.getStoreBackupFactories("noFactoryStore").isEmpty()); } @Test @@ -471,12 +470,108 @@ public void testGetRestoreManagers() { String storeRestoreFactory1 = "storeBackendRestoreFactory1"; String storeRestoreFactory2 = "storeBackendRestoreFactory2"; String storeRestoreFactoryOverride = storeRestoreFactory1 + "," + storeRestoreFactory2; - configMap.put(String.format(STORE_RESTORE_FACTORY, storeName), storeRestoreFactoryOverride); + configMap.put(String.format(STORE_RESTORE_FACTORIES, storeName), storeRestoreFactoryOverride); assertEquals(ImmutableSet.of(jobRestoreFactory1, jobRestoreFactory2, storeRestoreFactory1, storeRestoreFactory2), new StorageConfig(new MapConfig(configMap)).getRestoreFactories()); assertEquals(ImmutableList.of(storeRestoreFactory1, storeRestoreFactory2), new StorageConfig(new MapConfig(configMap)).getStoreRestoreFactories(storeName)); assertEquals(ImmutableList.of(jobRestoreFactory1, jobRestoreFactory2), new StorageConfig(new MapConfig(configMap)).getStoreRestoreFactories(storeName2)); + + String emptyBackupFactory = ""; + configMap.put(String.format(STORE_RESTORE_FACTORIES, storeName), emptyBackupFactory); + assertEquals(ImmutableSet.of(jobRestoreFactory1, jobRestoreFactory2), + new StorageConfig(new MapConfig(configMap)).getRestoreFactories()); + assertEquals(Collections.emptyList(), + new StorageConfig(new MapConfig(configMap)).getStoreRestoreFactories(storeName)); + assertEquals(ImmutableList.of(jobRestoreFactory1, jobRestoreFactory2), + new StorageConfig(new MapConfig(configMap)).getStoreRestoreFactories(storeName2)); + assertEquals(Collections.emptyList(), + new StorageConfig(new MapConfig(configMap)).getStoresWithRestoreFactory(KAFKA_STATE_BACKEND_FACTORY)); + assertEquals(Collections.emptyList(), + new StorageConfig(new MapConfig(configMap)).getStoresWithRestoreFactory(storeRestoreFactory1)); + assertEquals(Collections.emptyList(), + new StorageConfig(new MapConfig(configMap)).getStoresWithRestoreFactory(storeRestoreFactory2)); + } + + @Test + public void testGetBackupManagers() { + String storeName = "store1"; + String storeName2 = "store2"; + Map configMap = new HashMap<>(); + configMap.put(String.format(FACTORY, storeName), "store1.factory.class"); + configMap.put(String.format(FACTORY, storeName2), "store2.factory.class"); + + // empty config, return no restore managers + assertEquals(Collections.emptySet(), new StorageConfig(new MapConfig(configMap)).getBackupFactories()); + assertEquals(Collections.emptyList(), new StorageConfig(new MapConfig(configMap)).getStoreBackupFactories(storeName)); + assertEquals(Collections.emptyList(), new StorageConfig(new MapConfig(configMap)).getStoreBackupFactories(storeName2)); + + // changelog set, should default to kafka state backend restore + String changelogStreamOverride = "changelogStream"; + configMap.put(String.format(CHANGELOG_STREAM, storeName), changelogStreamOverride); + configMap.put(String.format(CHANGELOG_STREAM, storeName2), changelogStreamOverride); + configMap.put(StorageConfig.CHANGELOG_SYSTEM, "changelog-system"); + configMap.put(String.format(StorageConfig.CHANGELOG_STREAM, storeName), "changelog-stream0"); + assertEquals(ImmutableSet.of(KAFKA_STATE_BACKEND_FACTORY), new StorageConfig(new MapConfig(configMap)).getBackupFactories()); + assertEquals(DEFAULT_BACKUP_FACTORIES, new StorageConfig(new MapConfig(configMap)).getStoreBackupFactories(storeName)); + assertEquals(DEFAULT_BACKUP_FACTORIES, new StorageConfig(new MapConfig(configMap)).getStoreBackupFactories(storeName2)); + assertEquals(ImmutableList.of(storeName2, storeName), + new StorageConfig(new MapConfig(configMap)).getStoresWithBackupFactory(KAFKA_STATE_BACKEND_FACTORY)); + + // job restore manager config set should override to job backend factory + String jobBackupFactory1 = "jobBackendBackupFactory1"; + String jobBackupFactory2 = "jobBackendBackupFactory2"; + String jobBackupFactoryOverride = jobBackupFactory1 + "," + jobBackupFactory2; + configMap.put(JOB_BACKUP_FACTORIES, jobBackupFactoryOverride); + assertEquals(ImmutableSet.of(jobBackupFactory1, jobBackupFactory2), + new StorageConfig(new MapConfig(configMap)).getBackupFactories()); + assertEquals(ImmutableList.of(jobBackupFactory1, jobBackupFactory2), + new StorageConfig(new MapConfig(configMap)).getStoreBackupFactories(storeName)); + assertEquals(ImmutableList.of(jobBackupFactory1, jobBackupFactory2), + new StorageConfig(new MapConfig(configMap)).getStoreBackupFactories(storeName2)); + assertEquals(Collections.emptyList(), + new StorageConfig(new MapConfig(configMap)).getStoresWithBackupFactory(KAFKA_STATE_BACKEND_FACTORY)); + assertEquals(ImmutableList.of(storeName2, storeName), + new StorageConfig(new MapConfig(configMap)).getStoresWithBackupFactory(jobBackupFactory1)); + assertEquals(ImmutableList.of(storeName2, storeName), + new StorageConfig(new MapConfig(configMap)).getStoresWithBackupFactory(jobBackupFactory2)); + + // store specific restore managers set + String storeBackupFactory1 = "storeBackendBackupFactory1"; + String storeBackupFactory2 = "storeBackendBackupFactory2"; + String storeBackupFactoryOverride = storeBackupFactory1 + "," + storeBackupFactory2; + configMap.put(String.format(STORE_BACKUP_FACTORIES, storeName), storeBackupFactoryOverride); + assertEquals(ImmutableSet.of(jobBackupFactory1, jobBackupFactory2, storeBackupFactory1, storeBackupFactory2), + new StorageConfig(new MapConfig(configMap)).getBackupFactories()); + assertEquals(ImmutableList.of(storeBackupFactory1, storeBackupFactory2), + new StorageConfig(new MapConfig(configMap)).getStoreBackupFactories(storeName)); + assertEquals(ImmutableList.of(jobBackupFactory1, jobBackupFactory2), + new StorageConfig(new MapConfig(configMap)).getStoreBackupFactories(storeName2)); + assertEquals(Collections.emptyList(), + new StorageConfig(new MapConfig(configMap)).getStoresWithBackupFactory(KAFKA_STATE_BACKEND_FACTORY)); + assertEquals(ImmutableList.of(storeName2), + new StorageConfig(new MapConfig(configMap)).getStoresWithBackupFactory(jobBackupFactory1)); + assertEquals(ImmutableList.of(storeName2), + new StorageConfig(new MapConfig(configMap)).getStoresWithBackupFactory(jobBackupFactory2)); + assertEquals(ImmutableList.of(storeName), + new StorageConfig(new MapConfig(configMap)).getStoresWithBackupFactory(storeBackupFactory1)); + assertEquals(ImmutableList.of(storeName), + new StorageConfig(new MapConfig(configMap)).getStoresWithBackupFactory(storeBackupFactory2)); + + String emptyBackupFactory = ""; + configMap.put(String.format(STORE_BACKUP_FACTORIES, storeName), emptyBackupFactory); + assertEquals(ImmutableSet.of(jobBackupFactory1, jobBackupFactory2), + new StorageConfig(new MapConfig(configMap)).getBackupFactories()); + assertEquals(Collections.emptyList(), + new StorageConfig(new MapConfig(configMap)).getStoreBackupFactories(storeName)); + assertEquals(ImmutableList.of(jobBackupFactory1, jobBackupFactory2), + new StorageConfig(new MapConfig(configMap)).getStoreBackupFactories(storeName2)); + assertEquals(Collections.emptyList(), + new StorageConfig(new MapConfig(configMap)).getStoresWithBackupFactory(KAFKA_STATE_BACKEND_FACTORY)); + assertEquals(Collections.emptyList(), + new StorageConfig(new MapConfig(configMap)).getStoresWithBackupFactory(storeBackupFactory1)); + assertEquals(Collections.emptyList(), + new StorageConfig(new MapConfig(configMap)).getStoresWithBackupFactory(storeBackupFactory2)); } } diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java index 7e0da6d839..e45a03714f 100644 --- a/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java +++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.MoreExecutors; import java.io.File; @@ -30,6 +31,7 @@ import java.nio.file.Paths; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import org.apache.commons.lang3.tuple.Pair; @@ -176,7 +178,7 @@ public void testRestoreDeletesStoreDir() throws IOException { TaskName taskName = mock(TaskName.class); BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(new MetricsRegistryMap()); metrics.initStoreMetrics(ImmutableList.of("storeName")); - List storesToRestore = ImmutableList.of("storeName"); + Set storesToRestore = ImmutableSet.of("storeName"); SnapshotIndex snapshotIndex = mock(SnapshotIndex.class); Map> prevStoreSnapshotIndexes = ImmutableMap.of("storeName", Pair.of("blobId", snapshotIndex)); @@ -222,7 +224,7 @@ public void testRestoreDeletesCheckpointDirsIfRestoring() throws IOException { TaskName taskName = mock(TaskName.class); BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(new MetricsRegistryMap()); metrics.initStoreMetrics(ImmutableList.of("storeName")); - List storesToRestore = ImmutableList.of("storeName"); + Set storesToRestore = ImmutableSet.of("storeName"); SnapshotIndex snapshotIndex = mock(SnapshotIndex.class); Map> prevStoreSnapshotIndexes = ImmutableMap.of("storeName", Pair.of("blobId", snapshotIndex)); @@ -274,7 +276,7 @@ public void testRestoreRetainsCheckpointDirsIfValid() throws IOException { TaskName taskName = mock(TaskName.class); BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(new MetricsRegistryMap()); metrics.initStoreMetrics(ImmutableList.of("storeName")); - List storesToRestore = ImmutableList.of("storeName"); + Set storesToRestore = ImmutableSet.of("storeName"); SnapshotIndex snapshotIndex = mock(SnapshotIndex.class); Map> prevStoreSnapshotIndexes = ImmutableMap.of("storeName", Pair.of("blobId", snapshotIndex)); @@ -332,7 +334,7 @@ public void testRestoreSkipsStoresWithMissingCheckpointSCM() { TaskName taskName = mock(TaskName.class); BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(new MetricsRegistryMap()); metrics.initStoreMetrics(ImmutableList.of("newStoreName")); - List storesToRestore = ImmutableList.of("newStoreName"); // new store in config + Set storesToRestore = ImmutableSet.of("newStoreName"); // new store in config SnapshotIndex snapshotIndex = mock(SnapshotIndex.class); Map> prevStoreSnapshotIndexes = mock(Map.class); when(prevStoreSnapshotIndexes.containsKey("newStoreName")).thenReturn(false); diff --git a/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java b/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java index e0e5b476cb..535761f01c 100644 --- a/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java +++ b/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java @@ -18,15 +18,21 @@ */ package org.apache.samza.storage; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import java.io.File; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.Optional; +import java.util.Set; import org.apache.samza.Partition; import org.apache.samza.checkpoint.CheckpointManager; import org.apache.samza.checkpoint.CheckpointV1; +import org.apache.samza.checkpoint.CheckpointV2; import org.apache.samza.config.Config; import org.apache.samza.config.MapConfig; import org.apache.samza.config.StorageConfig; @@ -225,7 +231,7 @@ public Void answer(InvocationOnMock invocation) { StateBackendFactory backendFactory = mock(StateBackendFactory.class); TaskRestoreManager restoreManager = mock(TaskRestoreManager.class); - when(backendFactory.getRestoreManager(any(), any(), any(), any(), any(), any(), any(), any(), any(), any())) + when(backendFactory.getRestoreManager(any(), any(), any(), any(), any(), any(), any(), any(), any(), any(), any())) .thenReturn(restoreManager); doAnswer(invocation -> { storeRestoreCallCount++; @@ -272,4 +278,141 @@ public void testParallelismAndMetrics() throws InterruptedException { Assert.assertEquals("systemConsumerStopCount count should be 1", 1, this.systemConsumerStopCount); Assert.assertEquals("systemConsumerStartCount count should be 1", 1, this.systemConsumerStartCount); } + + @Test + public void testCheckpointBasedRestoreFactoryCreation() { + Set storeNames = ImmutableSet.of("storeName0", "storeName1", "storeName2"); + + StorageConfig mockConfig = mock(StorageConfig.class); + when(mockConfig.getStoreRestoreFactories("storeName0")) + .thenReturn(ImmutableList.of("factory0", "factory1", "factory2")); + when(mockConfig.getStoreRestoreFactories("storeName1")) + .thenReturn(ImmutableList.of("factory2", "factory1")); + when(mockConfig.getStoreRestoreFactories("storeName2")) + .thenReturn(Collections.emptyList()); + + when(mockConfig.getChangelogStream("storeName0")) + .thenReturn(Optional.empty()); + when(mockConfig.getChangelogStream("storeName1")) + .thenReturn(Optional.of("changelog")); + when(mockConfig.getChangelogStream("storeName2")) + .thenReturn(Optional.of("changelog")); + + CheckpointV1 checkpointV1 = mock(CheckpointV1.class); + when(checkpointV1.getVersion()).thenReturn((short) 1); + Map> factoriesToStores = this.containerStorageManager + .getBackendFactoryStoreNames(checkpointV1, storeNames, mockConfig); + + Assert.assertEquals(1, factoriesToStores.size()); + Assert.assertEquals(ImmutableSet.of("storeName1", "storeName2"), + factoriesToStores.get(StorageConfig.KAFKA_STATE_BACKEND_FACTORY)); + + factoriesToStores = this.containerStorageManager + .getBackendFactoryStoreNames(null, storeNames, mockConfig); + + Assert.assertEquals(2, factoriesToStores.size()); + Assert.assertEquals(ImmutableSet.of("storeName0"), + factoriesToStores.get("factory0")); + Assert.assertEquals(ImmutableSet.of("storeName1"), + factoriesToStores.get("factory2")); + } + + @Test + public void testCheckpointV2BasedRestoreFactoryCreation() { + Set storeNames = ImmutableSet.of("storeName0", "storeName1", "storeName2"); + + StorageConfig mockConfig = mock(StorageConfig.class); + when(mockConfig.getStoreRestoreFactories("storeName0")) + .thenReturn(ImmutableList.of("factory0", "factory1", "factory2")); + when(mockConfig.getStoreRestoreFactories("storeName1")) + .thenReturn(ImmutableList.of("factory2", "factory1")); + when(mockConfig.getStoreRestoreFactories("storeName2")) + .thenReturn(Collections.emptyList()); + + when(mockConfig.getChangelogStream("storeName0")) + .thenReturn(Optional.empty()); + when(mockConfig.getChangelogStream("storeName1")) + .thenReturn(Optional.of("changelog")); + when(mockConfig.getChangelogStream("storeName2")) + .thenReturn(Optional.of("changelog")); + + CheckpointV2 checkpointV2 = mock(CheckpointV2.class); + when(checkpointV2.getVersion()).thenReturn((short) 2); + when(checkpointV2.getStateCheckpointMarkers()) + .thenReturn(ImmutableMap.of( + "factory0", ImmutableMap.of("storeName0", "", "storeName1", "", "storeName2", ""), + "factory1", ImmutableMap.of("storeName0", "", "storeName1", "", "storeName2", ""), + "factory2", ImmutableMap.of("storeName0", "", "storeName1", "", "storeName2", ""))); + + Map> factoriesToStores = this.containerStorageManager + .getBackendFactoryStoreNames(checkpointV2, storeNames, mockConfig); + Assert.assertEquals(2, factoriesToStores.size()); + Assert.assertEquals(ImmutableSet.of("storeName0"), + factoriesToStores.get("factory0")); + Assert.assertEquals(ImmutableSet.of("storeName1"), + factoriesToStores.get("factory2")); + + when(checkpointV2.getStateCheckpointMarkers()) + .thenReturn(ImmutableMap.of( + "factory2", ImmutableMap.of("storeName0", "", "storeName1", "", "storeName2", ""))); + factoriesToStores = this.containerStorageManager + .getBackendFactoryStoreNames(checkpointV2, storeNames, mockConfig); + Assert.assertEquals(1, factoriesToStores.size()); + Assert.assertEquals(ImmutableSet.of("storeName1", "storeName0"), + factoriesToStores.get("factory2")); + + when(checkpointV2.getStateCheckpointMarkers()) + .thenReturn(ImmutableMap.of( + "factory1", ImmutableMap.of("storeName0", "", "storeName1", "", "storeName2", ""), + "factory2", ImmutableMap.of("storeName0", "", "storeName1", "", "storeName2", ""))); + factoriesToStores = this.containerStorageManager + .getBackendFactoryStoreNames(checkpointV2, storeNames, mockConfig); + Assert.assertEquals(2, factoriesToStores.size()); + Assert.assertEquals(ImmutableSet.of("storeName0"), + factoriesToStores.get("factory1")); + Assert.assertEquals(ImmutableSet.of("storeName1"), + factoriesToStores.get("factory2")); + + when(checkpointV2.getStateCheckpointMarkers()) + .thenReturn(ImmutableMap.of( + "factory1", ImmutableMap.of("storeName0", "", "storeName1", "", "storeName2", ""), + "factory2", ImmutableMap.of("storeName0", "", "storeName2", ""))); + factoriesToStores = this.containerStorageManager + .getBackendFactoryStoreNames(checkpointV2, storeNames, mockConfig); + Assert.assertEquals(1, factoriesToStores.size()); + Assert.assertEquals(ImmutableSet.of("storeName0", "storeName1"), + factoriesToStores.get("factory1")); + + when(checkpointV2.getStateCheckpointMarkers()) + .thenReturn(ImmutableMap.of( + "factory1", ImmutableMap.of("storeName0", "", "storeName1", "", "storeName2", ""))); + factoriesToStores = this.containerStorageManager + .getBackendFactoryStoreNames(checkpointV2, storeNames, mockConfig); + Assert.assertEquals(1, factoriesToStores.size()); + Assert.assertEquals(ImmutableSet.of("storeName0", "storeName1"), + factoriesToStores.get("factory1")); + + when(checkpointV2.getStateCheckpointMarkers()) + .thenReturn(Collections.emptyMap()); + factoriesToStores = this.containerStorageManager + .getBackendFactoryStoreNames(checkpointV2, storeNames, mockConfig); + Assert.assertEquals(2, factoriesToStores.size()); + Assert.assertEquals(ImmutableSet.of("storeName0"), + factoriesToStores.get("factory0")); + Assert.assertEquals(ImmutableSet.of("storeName1"), + factoriesToStores.get("factory2")); + + when(checkpointV2.getStateCheckpointMarkers()) + .thenReturn(ImmutableMap.of( + "factory0", ImmutableMap.of("storeName1", "", "storeName2", ""), + "factory1", ImmutableMap.of("storeName1", "", "storeName2", ""), + "factory2", ImmutableMap.of("storeName0", "", "storeName2", ""))); + factoriesToStores = this.containerStorageManager + .getBackendFactoryStoreNames(checkpointV2, storeNames, mockConfig); + Assert.assertEquals(2, factoriesToStores.size()); + Assert.assertEquals(ImmutableSet.of("storeName1"), + factoriesToStores.get("factory1")); + Assert.assertEquals(ImmutableSet.of("storeName0"), + factoriesToStores.get("factory2")); + } } diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java index d5059b341c..34d407f13a 100644 --- a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java @@ -106,8 +106,8 @@ public StorageEngine getStorageEngine(String storeName, storePropertiesBuilder.setPersistedToDisk(true); } // The store is durable iff it is backed by the task backup manager - List storeBackupManager = storageConfig.getStoreBackupFactory(storeName); - storePropertiesBuilder.setIsDurable(!storeBackupManager.isEmpty()); + List storeBackupManagers = storageConfig.getStoreBackupFactories(storeName); + storePropertiesBuilder.setIsDurable(!storeBackupManagers.isEmpty()); int batchSize = storageConfigSubset.getInt(WRITE_BATCH_SIZE, DEFAULT_WRITE_BATCH_SIZE); int cacheSize = storageConfigSubset.getInt(OBJECT_CACHE_SIZE, Math.max(batchSize, DEFAULT_OBJECT_CACHE_SIZE)); From 06974575051b5db971e23bb5cb323de54a7603d4 Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Wed, 28 Jul 2021 12:21:10 -0700 Subject: [PATCH 15/20] Fixed case where there are no stores to backup for CSM --- .../src/main/java/org/apache/samza/config/StorageConfig.java | 4 ++-- .../org/apache/samza/storage/ContainerStorageManager.java | 4 +++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java index 35d6003cfa..a915531f9a 100644 --- a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java @@ -299,8 +299,8 @@ private List getJobStoreBackupFactories() { * Backup state backend factory follows the precedence: * * 1. If stores.store-name.backup.factories config key exists the store-name, that value is used - * 2. If stores.store-name.changelog is set for store-name, the default Kafka changelog state backend factory - * 3. If stores.backup.factories is set for the job, that value is used + * 2. If stores.backup.factories is set for the job, that value is used + * 3. If stores.store-name.changelog is set for store-name, the default Kafka changelog state backend factory * 4. Otherwise no backup factories will be configured for the store * * Note: that 2 takes precedence over 3 enables job based migration off of Changelog restores diff --git a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java index 2879f9181c..6c11664806 100644 --- a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java +++ b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java @@ -431,7 +431,9 @@ Map> getBackendFactoryStoreNames(Checkpoint checkpoint, Set< .filter(storeName -> storageConfig.getChangelogStream(storeName).isPresent()) .collect(Collectors.toSet()); // Default to changelog backend factory when using checkpoint v1 for backwards compatibility - backendFactoryStoreNames.put(StorageConfig.KAFKA_STATE_BACKEND_FACTORY, changelogStores); + if (!changelogStores.isEmpty()) { + backendFactoryStoreNames.put(StorageConfig.KAFKA_STATE_BACKEND_FACTORY, changelogStores); + } if (storeNames.size() > changelogStores.size()) { Set nonChangelogStores = storeNames.stream() .filter(storeName -> !changelogStores.contains(storeName)) From e74c2415cee42ec6457947698c4827d04e821df5 Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Wed, 28 Jul 2021 17:18:59 -0700 Subject: [PATCH 16/20] Added tests for CSM for non configured stores RB=2721143 G=samza-reviewers R=shesharm A= --- .../storage/TestContainerStorageManager.java | 87 ++++++++++++++++++- .../storage/TestTaskStorageManager.scala | 4 + 2 files changed, 90 insertions(+), 1 deletion(-) diff --git a/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java b/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java index 535761f01c..b75f50405e 100644 --- a/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java +++ b/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java @@ -178,6 +178,8 @@ public void setUp() throws InterruptedException { Map configMap = new HashMap<>(); configMap.put("stores." + STORE_NAME + ".key.serde", "stringserde"); configMap.put("stores." + STORE_NAME + ".msg.serde", "stringserde"); + configMap.put("stores." + STORE_NAME + ".factory", mockStorageEngineFactory.getClass().getName()); + configMap.put("stores." + STORE_NAME + ".changelog", SYSTEM_NAME + "." + STREAM_NAME); configMap.put("serializers.registry.stringserde.class", StringSerdeFactory.class.getName()); configMap.put(TaskConfig.TRANSACTIONAL_STATE_RETAIN_EXISTING_STATE, "true"); Config config = new MapConfig(configMap); @@ -223,7 +225,7 @@ public Void answer(InvocationOnMock invocation) { ContainerModel mockContainerModel = new ContainerModel("samza-container-test", tasks); when(mockContainerContext.getContainerModel()).thenReturn(mockContainerModel); - // Reset the expected number of sysConsumer create, start and stop calls, and store.restore() calls + // Reset the expected number of sysConsumer create, start and stop calls, and store.restore() calls this.systemConsumerCreationCount = 0; this.systemConsumerStartCount = 0; this.systemConsumerStopCount = 0; @@ -279,6 +281,89 @@ public void testParallelismAndMetrics() throws InterruptedException { Assert.assertEquals("systemConsumerStartCount count should be 1", 1, this.systemConsumerStartCount); } + @Test + public void testNoConfiguredDurableStores() throws InterruptedException{ + taskRestoreMetricGauges = new HashMap<>(); + this.tasks = new HashMap<>(); + this.taskInstanceMetrics = new HashMap<>(); + + // Add two mocked tasks + addMockedTask("task 0", 0); + addMockedTask("task 1", 1); + + // Mock container metrics + samzaContainerMetrics = mock(SamzaContainerMetrics.class); + when(samzaContainerMetrics.taskStoreRestorationMetrics()).thenReturn(taskRestoreMetricGauges); + + // Create mocked configs for specifying serdes + Map configMap = new HashMap<>(); + configMap.put("serializers.registry.stringserde.class", StringSerdeFactory.class.getName()); + configMap.put(TaskConfig.TRANSACTIONAL_STATE_RETAIN_EXISTING_STATE, "true"); + Config config = new MapConfig(configMap); + + Map> serdes = new HashMap<>(); + serdes.put("stringserde", mock(Serde.class)); + + CheckpointManager checkpointManager = mock(CheckpointManager.class); + when(checkpointManager.readLastCheckpoint(any(TaskName.class))).thenReturn(new CheckpointV1(new HashMap<>())); + + ContainerContext mockContainerContext = mock(ContainerContext.class); + ContainerModel mockContainerModel = new ContainerModel("samza-container-test", tasks); + when(mockContainerContext.getContainerModel()).thenReturn(mockContainerModel); + + // Reset the expected number of sysConsumer create, start and stop calls, and store.restore() calls + this.systemConsumerCreationCount = 0; + this.systemConsumerStartCount = 0; + this.systemConsumerStopCount = 0; + this.storeRestoreCallCount = 0; + + StateBackendFactory backendFactory = mock(StateBackendFactory.class); + TaskRestoreManager restoreManager = mock(TaskRestoreManager.class); + when(backendFactory.getRestoreManager(any(), any(), any(), any(), any(), any(), any(), any(), any(), any(), any())) + .thenReturn(restoreManager); + doAnswer(invocation -> { + storeRestoreCallCount++; + return null; + }).when(restoreManager).restore(); + + // Create the container storage manager + ContainerStorageManager containerStorageManager = new ContainerStorageManager( + checkpointManager, + mockContainerModel, + mock(StreamMetadataCache.class), + mock(SystemAdmins.class), + new HashMap<>(), + new HashMap<>(), + new HashMap<>(), + new HashMap<>(), + serdes, + config, + taskInstanceMetrics, + samzaContainerMetrics, + mock(JobContext.class), + mockContainerContext, + new HashMap<>(), + mock(Map.class), + DEFAULT_LOGGED_STORE_BASE_DIR, + DEFAULT_STORE_BASE_DIR, + null, + new SystemClock()); + + containerStorageManager.start(); + containerStorageManager.shutdown(); + + for (Gauge gauge : taskRestoreMetricGauges.values()) { + Assert.assertTrue("Restoration time gauge value should never be invoked", + mockingDetails(gauge).getInvocations().size() == 0); + } + + Assert.assertEquals("Store restore count should be 2 because there are 0 stores", 0, this.storeRestoreCallCount); + Assert.assertEquals(0, + this.systemConsumerCreationCount); + Assert.assertEquals(0, this.systemConsumerStopCount); + Assert.assertEquals(0, this.systemConsumerStartCount); + } + @Test public void testCheckpointBasedRestoreFactoryCreation() { Set storeNames = ImmutableSet.of("storeName0", "storeName1", "storeName2"); diff --git a/samza-kafka/src/test/java/org/apache/samza/storage/TestTaskStorageManager.scala b/samza-kafka/src/test/java/org/apache/samza/storage/TestTaskStorageManager.scala index b4a2c3581b..4c8d8e489c 100644 --- a/samza-kafka/src/test/java/org/apache/samza/storage/TestTaskStorageManager.scala +++ b/samza-kafka/src/test/java/org/apache/samza/storage/TestTaskStorageManager.scala @@ -767,8 +767,12 @@ class TaskStorageManagerBuilder extends MockitoSugar { "stores.loggedStore1.clean.on.container.start" -> cleanStoreDirsOnStart.toString, "stores.store1.key.serde" -> classOf[StringSerdeFactory].getCanonicalName, "stores.store1.msg.serde" -> classOf[StringSerdeFactory].getCanonicalName, + "stores.store1.factory" -> mockStorageEngineFactory.getClass.getName, + "stores.store1.changelog" -> "system.stream", + "stores.loggedStore1.factory" -> mockStorageEngineFactory.getClass.getName, "stores.loggedStore1.key.serde" -> classOf[StringSerdeFactory].getCanonicalName, "stores.loggedStore1.msg.serde" -> classOf[StringSerdeFactory].getCanonicalName, + "stores.loggedStore1.changelog" -> "system.stream", TaskConfig.TRANSACTIONAL_STATE_RESTORE_ENABLED -> "false").asJava) var mockSerdes: Map[String, Serde[AnyRef]] = HashMap[String, Serde[AnyRef]]((classOf[StringSerdeFactory].getCanonicalName, Mockito.mock(classOf[Serde[AnyRef]]))) From ad408d5e8557255582d84dce522ed3da133eba23 Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Mon, 2 Aug 2021 15:31:34 -0700 Subject: [PATCH 17/20] Cleanup of TODOs --- .../org/apache/samza/storage/StorageManagerUtil.java | 9 +++++---- .../samza/storage/blobstore/BlobStoreRestoreManager.java | 4 ++-- .../apache/samza/storage/ContainerStorageManager.java | 5 ++++- .../apache/samza/system/kafka/KafkaConsumerProxy.java | 7 ++----- 4 files changed, 13 insertions(+), 12 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java b/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java index 057b248ae9..badeb28c00 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java +++ b/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java @@ -113,7 +113,6 @@ public String getStartingOffset( * @param isSideInput true if store is a side-input store, false if it is a regular store * @return true if the store is stale, false otherwise */ - // TODO BLOCKER dchen do these methods need to be updated to also read the new checkpoint file? public boolean isStaleStore(File storeDir, long storeDeleteRetentionInMs, long currentTimeMs, boolean isSideInput) { long offsetFileLastModifiedTime; boolean isStaleStore = false; @@ -127,8 +126,11 @@ public boolean isStaleStore(File storeDir, long storeDeleteRetentionInMs, long c File offsetFileRefNew = new File(storeDir, OFFSET_FILE_NAME_NEW); File offsetFileRefLegacy = new File(storeDir, OFFSET_FILE_NAME_LEGACY); File sideInputOffsetFileRefLegacy = new File(storeDir, SIDE_INPUT_OFFSET_FILE_NAME_LEGACY); + File checkpointV2File = new File(storeDir, CHECKPOINT_FILE_NAME); - if (offsetFileRefNew.exists()) { + if (checkpointV2File.exists()) { + offsetFileLastModifiedTime = checkpointV2File.lastModified(); + } else if (offsetFileRefNew.exists()) { offsetFileLastModifiedTime = offsetFileRefNew.lastModified(); } else if (!isSideInput && offsetFileRefLegacy.exists()) { offsetFileLastModifiedTime = offsetFileRefLegacy.lastModified(); @@ -184,7 +186,6 @@ && isOffsetFileValid(loggedStoreDir, Collections.singleton(changelogSSP), false) * @param isSideInput true if store is a side-input store, false if it is a regular store * @return true if the offset file is valid. false otherwise. */ - // TODO BLOCKER dchen do these methods need to be updated to also read the new checkpoint file? public boolean isOffsetFileValid(File storeDir, Set storeSSPs, boolean isSideInput) { boolean hasValidOffsetFile = false; if (storeDir.exists()) { @@ -305,11 +306,11 @@ public Map readOffsetFile(File storagePartitionDi /** * Read and return the {@link CheckpointV2} from the directory's {@link #CHECKPOINT_FILE_NAME} file. * If the file does not exist, returns null. - * // TODO HIGH dchen add tests at all call sites for handling null value. * * @param storagePartitionDir store directory to read the checkpoint file from * @return the {@link CheckpointV2} object retrieved from the checkpoint file if found, otherwise return null */ + // TODO dchen use checkpoint v2 file before migrating off of dual checkpoints public CheckpointV2 readCheckpointV2File(File storagePartitionDir) { File checkpointFile = new File(storagePartitionDir, CHECKPOINT_FILE_NAME); if (checkpointFile.exists()) { diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java index 7eb4e5ddb1..64bc74e873 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java @@ -101,7 +101,7 @@ public BlobStoreRestoreManager(TaskModel taskModel, ExecutorService restoreExecu this.taskModel = taskModel; this.jobName = new JobConfig(config).getName().get(); this.jobId = new JobConfig(config).getJobId(); - this.executor = restoreExecutor; // TODO BLOCKER dchen1 dont block on restore executor + this.executor = restoreExecutor; this.config = config; this.storageConfig = new StorageConfig(config); this.blobStoreConfig = new BlobStoreConfig(config); @@ -282,7 +282,7 @@ static void restoreStores(String jobName, String jobId, TaskName taskName, Set { LOG.info("Restore completed for task: {} stores", taskName); metrics.restoreNs.set(System.nanoTime() - restoreStartTime); - }).join(); // TODO BLOCKER dchen1 make non-blocking. + }).join(); // TODO dchen make non-blocking for the restore executor } /** diff --git a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java index 6c11664806..f804a5e7c9 100644 --- a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java +++ b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java @@ -747,7 +747,10 @@ private void restoreStores() throws InterruptedException { ) ); - // Start each store consumer once + // Start each store consumer once. + // Note: These consumers are per system and only changelog system store consumers will be started. + // Some TaskRestoreManagers may not require the consumer to to be started, but due to the agnostic nature of + // ContainerStorageManager we always start the changelog consumer here in case it is required this.storeConsumers.values().stream().distinct().forEach(SystemConsumer::start); List taskRestoreFutures = new ArrayList<>(); diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaConsumerProxy.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaConsumerProxy.java index 9cf8eae3c0..5c061aec3f 100644 --- a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaConsumerProxy.java +++ b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaConsumerProxy.java @@ -152,11 +152,8 @@ public void start() { } if (topicPartitionToSSP.size() == 0) { - String msg = String.format("Cannot start KafkaConsumerProxy without any registered TopicPartitions for %s", systemName); - LOG.error(msg); - // TODO HIGH dchen the consumer init for blob stores only goes through this path as well if it has changelogs for backup - // need to isolate the changes - //throw new SamzaException(msg); + String msg = String.format("Started KafkaConsumerProxy without any registered TopicPartitions for %s", systemName); + LOG.warn(msg); } } From 785d5060410ba8342509e5dae2303e20fc62ffa3 Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Tue, 3 Aug 2021 12:18:25 -0700 Subject: [PATCH 18/20] checkstype cleanup --- .../src/main/java/org/apache/samza/config/StorageConfig.java | 2 +- .../src/main/java/org/apache/samza/storage/StorageRecovery.java | 2 +- .../test/java/org/apache/samza/config/TestStorageConfig.java | 2 +- .../samza/storage/blobstore/TestBlobStoreRestoreManager.java | 1 - .../org/apache/samza/storage/TestContainerStorageManager.java | 2 +- 5 files changed, 4 insertions(+), 5 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java index a915531f9a..6466dd751a 100644 --- a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java @@ -364,7 +364,7 @@ public List getStoreRestoreFactories(String storeName) { public Set getRestoreFactories() { return getStoreNames().stream() - .flatMap(((storesName) -> getStoreRestoreFactories(storesName).stream())) + .flatMap((storesName) -> getStoreRestoreFactories(storesName).stream()) .collect(Collectors.toSet()); } diff --git a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java index f1d7ce2d29..6b657d59d9 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java +++ b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java @@ -220,7 +220,7 @@ private void getContainerStorageManagers() { Set factoryClasses = new StorageConfig(jobConfig).getRestoreFactories(); Map stateBackendFactories = factoryClasses.stream().collect( Collectors.toMap(factoryClass -> factoryClass, - factoryClass -> ReflectionUtil.getObj(factoryClass, StateBackendFactory.class))); + factoryClass -> ReflectionUtil.getObj(factoryClass, StateBackendFactory.class))); Clock clock = SystemClock.instance(); StreamMetadataCache streamMetadataCache = new StreamMetadataCache(systemAdmins, 5000, clock); // don't worry about prefetching for this; looks like the tool doesn't flush to offset files anyways diff --git a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java index 9a60f12add..5dba698899 100644 --- a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java +++ b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java @@ -113,7 +113,7 @@ public void testGetChangelogStream() { // store has changelog stream defined, but system comes from job.changelog.system storageConfig = new StorageConfig(new MapConfig( ImmutableMap.of(String.format(StorageConfig.CHANGELOG_STREAM, STORE_NAME0), "changelog-stream0", - String.format(FACTORY, STORE_NAME0), "store0.factory.class" , + String.format(FACTORY, STORE_NAME0), "store0.factory.class", StorageConfig.CHANGELOG_SYSTEM, "changelog-system"))); assertEquals(Optional.of("changelog-system.changelog-stream0"), storageConfig.getChangelogStream(STORE_NAME0)); assertEquals(ImmutableMap.of(STORE_NAME0, new SystemStream("changelog-system", "changelog-stream0")), storageConfig.getStoreChangelogs()); diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java index e45a03714f..ddc0c8e19a 100644 --- a/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java +++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java @@ -29,7 +29,6 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; diff --git a/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java b/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java index b75f50405e..8645e4a21c 100644 --- a/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java +++ b/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java @@ -282,7 +282,7 @@ public void testParallelismAndMetrics() throws InterruptedException { } @Test - public void testNoConfiguredDurableStores() throws InterruptedException{ + public void testNoConfiguredDurableStores() throws InterruptedException { taskRestoreMetricGauges = new HashMap<>(); this.tasks = new HashMap<>(); this.taskInstanceMetrics = new HashMap<>(); From 3c8a4623deda61f5ceae8fd54090119d87ad4899 Mon Sep 17 00:00:00 2001 From: Daniel Chen Date: Tue, 3 Aug 2021 13:09:50 -0700 Subject: [PATCH 19/20] checkstyle for samza-api --- .../org/apache/samza/storage/KafkaChangelogRestoreParams.java | 1 - 1 file changed, 1 deletion(-) diff --git a/samza-api/src/main/java/org/apache/samza/storage/KafkaChangelogRestoreParams.java b/samza-api/src/main/java/org/apache/samza/storage/KafkaChangelogRestoreParams.java index 728b10f2f1..ed4601a5f5 100644 --- a/samza-api/src/main/java/org/apache/samza/storage/KafkaChangelogRestoreParams.java +++ b/samza-api/src/main/java/org/apache/samza/storage/KafkaChangelogRestoreParams.java @@ -20,7 +20,6 @@ package org.apache.samza.storage; import java.util.Map; -import java.util.Set; import org.apache.samza.serializers.Serde; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemConsumer; From 65b5df6824acaaaad7cb9e504b831b62eae2a57f Mon Sep 17 00:00:00 2001 From: shekhars-li Date: Tue, 3 Aug 2021 12:05:26 -0700 Subject: [PATCH 20/20] Test fix for failing job: only backup persistent stores --- .../java/org/apache/samza/config/StorageConfig.java | 12 ++++++++++++ .../storage/blobstore/BlobStoreBackupManager.java | 2 +- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java index 6466dd751a..4367244ef0 100644 --- a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java @@ -333,6 +333,18 @@ public List getStoresWithBackupFactory(String backendFactoryName) { .collect(Collectors.toList()); } + public List getPersistentStoresWithBackupFactory(String backendFactoryName) { + return getStoreNames().stream() + .filter(storeName -> { + Optional storeFactory = getStorageFactoryClassName(storeName); + return storeFactory.isPresent() && + !storeFactory.get().equals(StorageConfig.INMEMORY_KV_STORAGE_ENGINE_FACTORY); + }) + .filter((storeName) -> getStoreBackupFactories(storeName) + .contains(backendFactoryName)) + .collect(Collectors.toList()); + } + private List getJobStoreRestoreFactories() { return getList(JOB_RESTORE_FACTORIES, new ArrayList<>()); } diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java index 15cc87b6d8..fb04ce5d67 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java @@ -114,7 +114,7 @@ public BlobStoreBackupManager(JobModel jobModel, ContainerModel containerModel, this.storageManagerUtil = storageManagerUtil; StorageConfig storageConfig = new StorageConfig(config); this.storesToBackup = - storageConfig.getStoresWithBackupFactory(BlobStoreStateBackendFactory.class.getName()); + storageConfig.getPersistentStoresWithBackupFactory(BlobStoreStateBackendFactory.class.getName()); this.loggedStoreBaseDir = loggedStoreBaseDir; this.blobStoreManager = blobStoreManager; this.blobStoreUtil = createBlobStoreUtil(blobStoreManager, executor, blobStoreTaskBackupMetrics);