Skip to content

Commit

Permalink
[FLINK-28172][changelog] Scatter dstl files into separate directories…
Browse files Browse the repository at this point in the history
… by job id
  • Loading branch information
zoltar9264 authored and rkhachatryan committed Jul 8, 2022
1 parent 231e96b commit 1351303
Show file tree
Hide file tree
Showing 14 changed files with 121 additions and 16 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import org.apache.flink.annotation.Experimental;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.operators.MailboxExecutor;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
Expand Down Expand Up @@ -61,24 +62,31 @@ public class FsStateChangelogStorage extends FsStateChangelogStorageForRecovery

private final TaskChangelogRegistry changelogRegistry;

public FsStateChangelogStorage(Configuration config, TaskManagerJobMetricGroup metricGroup)
public FsStateChangelogStorage(
JobID jobID, Configuration config, TaskManagerJobMetricGroup metricGroup)
throws IOException {
this(config, metricGroup, defaultChangelogRegistry(config.get(NUM_DISCARD_THREADS)));
this(jobID, config, metricGroup, defaultChangelogRegistry(config.get(NUM_DISCARD_THREADS)));
}

public FsStateChangelogStorage(
JobID jobID,
Configuration config,
TaskManagerJobMetricGroup metricGroup,
TaskChangelogRegistry changelogRegistry)
throws IOException {
this(
fromConfig(config, new ChangelogStorageMetricGroup(metricGroup), changelogRegistry),
fromConfig(
jobID,
config,
new ChangelogStorageMetricGroup(metricGroup),
changelogRegistry),
config.get(PREEMPTIVE_PERSIST_THRESHOLD).getBytes(),
changelogRegistry);
}

@VisibleForTesting
public FsStateChangelogStorage(
JobID jobID,
Path basePath,
boolean compression,
int bufferSize,
Expand All @@ -88,6 +96,7 @@ public FsStateChangelogStorage(
this(
directScheduler(
new StateChangeFsUploader(
jobID,
basePath,
basePath.getFileSystem(),
compression,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.flink.changelog.fs;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.metrics.groups.TaskManagerJobMetricGroup;
import org.apache.flink.runtime.state.changelog.StateChangelogStorage;
Expand Down Expand Up @@ -46,8 +47,9 @@ public String getIdentifier() {

@Override
public StateChangelogStorage<?> createStorage(
Configuration configuration, TaskManagerJobMetricGroup metricGroup) throws IOException {
return new FsStateChangelogStorage(configuration, metricGroup);
JobID jobID, Configuration configuration, TaskManagerJobMetricGroup metricGroup)
throws IOException {
return new FsStateChangelogStorage(jobID, configuration, metricGroup);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.flink.changelog.fs;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.JobID;
import org.apache.flink.changelog.fs.StateChangeUploadScheduler.UploadTask;
import org.apache.flink.core.fs.FSDataOutputStream;
import org.apache.flink.core.fs.FileSystem;
Expand Down Expand Up @@ -54,6 +55,8 @@
public class StateChangeFsUploader implements StateChangeUploader {
private static final Logger LOG = LoggerFactory.getLogger(StateChangeFsUploader.class);

@VisibleForTesting public static final String PATH_SUB_DIR = "dstl";

private final Path basePath;
private final FileSystem fileSystem;
private final StateChangeFormat format;
Expand All @@ -66,13 +69,15 @@ public class StateChangeFsUploader implements StateChangeUploader {

@VisibleForTesting
public StateChangeFsUploader(
JobID jobID,
Path basePath,
FileSystem fileSystem,
boolean compression,
int bufferSize,
ChangelogStorageMetricGroup metrics,
TaskChangelogRegistry changelogRegistry) {
this(
jobID,
basePath,
fileSystem,
compression,
Expand All @@ -83,14 +88,16 @@ public StateChangeFsUploader(
}

public StateChangeFsUploader(
JobID jobID,
Path basePath,
FileSystem fileSystem,
boolean compression,
int bufferSize,
ChangelogStorageMetricGroup metrics,
TaskChangelogRegistry changelogRegistry,
BiFunction<Path, Long, StreamStateHandle> handleFactory) {
this.basePath = basePath;
this.basePath =
new Path(basePath, String.format("%s/%s", jobID.toHexString(), PATH_SUB_DIR));
this.fileSystem = fileSystem;
this.format = new StateChangeFormat();
this.compression = compression;
Expand All @@ -101,6 +108,11 @@ public StateChangeFsUploader(
this.handleFactory = handleFactory;
}

@VisibleForTesting
public Path getBasePath() {
return this.basePath;
}

public UploadTasksResult upload(Collection<UploadTask> tasks) throws IOException {
final String fileName = generateFileName();
LOG.debug("upload {} tasks to {}", tasks.size(), fileName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.flink.changelog.fs;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.io.AvailabilityProvider;
Expand Down Expand Up @@ -84,6 +85,7 @@ public void close() throws Exception {
}

static StateChangeUploadScheduler fromConfig(
JobID jobID,
ReadableConfig config,
ChangelogStorageMetricGroup metricGroup,
TaskChangelogRegistry changelogRegistry)
Expand All @@ -94,6 +96,7 @@ static StateChangeUploadScheduler fromConfig(
int bufferSize = (int) bytes;
StateChangeFsUploader store =
new StateChangeFsUploader(
jobID,
basePath,
basePath.getFileSystem(),
config.get(COMPRESSION_ENABLED),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@ void testUploadsCounter() throws Exception {

try (FsStateChangelogStorage storage =
new FsStateChangelogStorage(
JobID.generate(),
Path.fromLocalFile(tempFolder.toFile()),
false,
100,
Expand All @@ -88,6 +89,7 @@ void testUploadSizes() throws Exception {

try (FsStateChangelogStorage storage =
new FsStateChangelogStorage(
JobID.generate(),
Path.fromLocalFile(tempFolder.toFile()),
false,
100,
Expand Down Expand Up @@ -120,6 +122,7 @@ void testUploadFailuresCounter() throws Exception {
new ChangelogStorageMetricGroup(createUnregisteredTaskManagerJobMetricGroup());
try (FsStateChangelogStorage storage =
new FsStateChangelogStorage(
JobID.generate(),
Path.fromLocalFile(file),
false,
100,
Expand Down Expand Up @@ -150,6 +153,7 @@ void testUploadBatchSizes() throws Exception {
Path basePath = Path.fromLocalFile(tempFolder.toFile());
StateChangeFsUploader uploader =
new StateChangeFsUploader(
JobID.generate(),
basePath,
basePath.getFileSystem(),
false,
Expand Down Expand Up @@ -287,6 +291,7 @@ void testTotalAttemptsPerUpload() throws Exception {

@Test
void testQueueSize() throws Exception {
JobID jobID = JobID.generate();
AtomicReference<Gauge<Integer>> queueSizeGauge = new AtomicReference<>();
ChangelogStorageMetricGroup metrics =
new ChangelogStorageMetricGroup(
Expand All @@ -301,12 +306,13 @@ void testQueueSize() throws Exception {
})
.build(),
createUnregisteredTaskManagerMetricGroup(),
new JobID(),
jobID,
"test"));

Path path = Path.fromLocalFile(tempFolder.toFile());
StateChangeFsUploader delegate =
new StateChangeFsUploader(
jobID,
path,
path.getFileSystem(),
false,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.flink.changelog.fs;

import org.apache.flink.api.common.JobID;
import org.apache.flink.changelog.fs.BatchingStateChangeUploadSchedulerTest.BlockingUploader;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.jobgraph.OperatorID;
Expand Down Expand Up @@ -51,6 +52,7 @@ public static Stream<Boolean> parameters() {
protected StateChangelogStorage<ChangelogStateHandleStreamImpl> getFactory(
boolean compression, File temporaryFolder) throws IOException {
return new FsStateChangelogStorage(
JobID.generate(),
Path.fromLocalFile(temporaryFolder),
compression,
1024 * 1024 * 10,
Expand Down
Original file line number Diff line number Diff line change
@@ -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.flink.changelog.fs;

import org.apache.flink.api.common.JobID;
import org.apache.flink.core.fs.Path;

import org.junit.jupiter.api.Test;

import java.io.IOException;

import static org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups.createUnregisteredTaskManagerJobMetricGroup;
import static org.junit.jupiter.api.Assertions.assertEquals;

/** {@link StateChangeFsUploader} test. */
class StateChangeFsUploaderTest {

@Test
void testBasePath() throws IOException {
JobID jobID = JobID.generate();
String rootPath = "/dstl-root-path";
Path oriBasePath = new Path(rootPath);

ChangelogStorageMetricGroup metrics =
new ChangelogStorageMetricGroup(createUnregisteredTaskManagerJobMetricGroup());

StateChangeFsUploader uploader =
new StateChangeFsUploader(
jobID,
oriBasePath,
oriBasePath.getFileSystem(),
false,
4096,
metrics,
TaskChangelogRegistry.NO_OP);

assertEquals(
uploader.getBasePath().getPath(),
String.format(
"%s/%s/%s",
rootPath, jobID.toHexString(), StateChangeFsUploader.PATH_SUB_DIR));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ public StateChangelogStorage<?> stateChangelogStorageForJob(

if (stateChangelogStorage == null) {
StateChangelogStorage<?> loaded =
StateChangelogStorageLoader.load(configuration, metricGroup);
StateChangelogStorageLoader.load(jobId, configuration, metricGroup);
stateChangelogStorage = Optional.ofNullable(loaded);
changelogStoragesByJobId.put(jobId, stateChangelogStorage);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.runtime.state.changelog;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.metrics.groups.TaskManagerJobMetricGroup;

Expand All @@ -35,7 +36,8 @@ public interface StateChangelogStorageFactory {

/** Create the storage based on a configuration. */
StateChangelogStorage<?> createStorage(
Configuration configuration, TaskManagerJobMetricGroup metricGroup) throws IOException;
JobID jobID, Configuration configuration, TaskManagerJobMetricGroup metricGroup)
throws IOException;

/** Create the storage for recovery. */
StateChangelogStorageView<?> createStorageView() throws IOException;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.flink.runtime.state.changelog;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.StateChangelogOptions;
import org.apache.flink.core.plugin.PluginManager;
Expand Down Expand Up @@ -86,7 +87,8 @@ public static void initialize(PluginManager pluginManager) {

@Nullable
public static StateChangelogStorage<?> load(
Configuration configuration, TaskManagerJobMetricGroup metricGroup) throws IOException {
JobID jobID, Configuration configuration, TaskManagerJobMetricGroup metricGroup)
throws IOException {
final String identifier =
configuration
.getString(StateChangelogOptions.STATE_CHANGE_LOG_STORAGE)
Expand All @@ -98,7 +100,7 @@ public static StateChangelogStorage<?> load(
return null;
} else {
LOG.info("Creating a changelog storage with name '{}'.", identifier);
return factory.createStorage(configuration, metricGroup);
return factory.createStorage(jobID, configuration, metricGroup);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.flink.runtime.state.changelog.inmemory;

import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.metrics.groups.TaskManagerJobMetricGroup;
import org.apache.flink.runtime.state.changelog.StateChangelogStorage;
Expand All @@ -35,7 +36,7 @@ public String getIdentifier() {

@Override
public StateChangelogStorage<?> createStorage(
Configuration configuration, TaskManagerJobMetricGroup metricGroup) {
JobID jobID, Configuration configuration, TaskManagerJobMetricGroup metricGroup) {
return new InMemoryStateChangelogStorage();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -207,7 +207,7 @@ public String getIdentifier() {

@Override
public StateChangelogStorage<?> createStorage(
Configuration configuration, TaskManagerJobMetricGroup metricGroup) {
JobID jobID, Configuration configuration, TaskManagerJobMetricGroup metricGroup) {
return new TestStateChangelogStorage();
}

Expand Down
Loading

0 comments on commit 1351303

Please sign in to comment.