You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2018/10/15 16:28:14 UTC

[GitHub] kent2171 closed pull request #6824: [FLINK-9592][flink-connector-filesystem] added ability to hook file state changing

kent2171 closed pull request #6824: [FLINK-9592][flink-connector-filesystem] added ability to hook file state changing
URL: https://github.com/apache/flink/pull/6824
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
index 4f85e3cf8d5..67500ed549e 100644
--- a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
+++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java
@@ -58,6 +58,7 @@
 import java.lang.reflect.Method;
 import java.net.URI;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -113,6 +114,9 @@
  * a part file is closed for writing it becomes {@code pending}. When a checkpoint is successful the currently
  * pending files will be moved to {@code finished}.
  *
+ * <p>If it's necessary to perform any additional actions, when state of the file is changed, you need to register
+ * the list of callbacks using {@link #registerFileStateChangedCallback(FileStateChangedCallback...)} method.
+ * All of them will be called in accordance with the specified order.
  *
  * <p>If case of a failure, and in order to guarantee exactly-once semantics, the sink should roll back to the state it
  * had when that last successful checkpoint occurred. To this end, when restoring, the restored files in {@code pending}
@@ -323,6 +327,11 @@
 
 	private transient ProcessingTimeService processingTimeService;
 
+	/**
+	 * The list of callbacks, that should be called, when state of the file is changed.
+	 */
+	private List<FileStateChangedCallback> fileStateChangedCallbacks = new ArrayList<>();
+
 	/**
 	 * Creates a new {@code BucketingSink} that writes files to the given base directory.
 	 *
@@ -368,6 +377,11 @@ public void setInputType(TypeInformation<?> type, ExecutionConfig executionConfi
 		}
 	}
 
+	public BucketingSink<T> registerFileStateChangedCallback(FileStateChangedCallback... callbacks) {
+		fileStateChangedCallbacks.addAll(Arrays.asList(callbacks));
+		return this;
+	}
+
 	@Override
 	public void initializeState(FunctionInitializationContext context) throws Exception {
 		Preconditions.checkArgument(this.restoredBucketStates == null, "The operator has already been initialized.");
@@ -598,10 +612,14 @@ private void closeCurrentPartFile(BucketState<T> bucketState) throws Exception {
 			Path inProgressPath = getInProgressPathFor(currentPartPath);
 			Path pendingPath = getPendingPathFor(currentPartPath);
 
+			LOG.debug("Moving in-progress bucket {} to pending file {}", inProgressPath, pendingPath);
 			fs.rename(inProgressPath, pendingPath);
-			LOG.debug("Moving in-progress bucket {} to pending file {}",
-				inProgressPath,
-				pendingPath);
+			for (FileStateChangedCallback callback : fileStateChangedCallbacks) {
+				callback.onInProgressToPending(fs, pendingPath);
+			}
+
+			LOG.debug("In-progress bucket {} successfully moved to pending file {}", inProgressPath, pendingPath);
+
 			bucketState.pendingFiles.add(currentPartPath.toString());
 			bucketState.currentFile = null;
 		}
@@ -702,11 +720,18 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception {
 								Path finalPath = new Path(filename);
 								Path pendingPath = getPendingPathFor(finalPath);
 
-								fs.rename(pendingPath, finalPath);
 								LOG.debug(
 									"Moving pending file {} to final location having completed checkpoint {}.",
-									pendingPath,
-									pastCheckpointId);
+									pendingPath, pastCheckpointId
+								);
+								fs.rename(pendingPath, finalPath);
+								for (FileStateChangedCallback callback : fileStateChangedCallbacks) {
+									callback.onPendingToFinal(fs, finalPath);
+								}
+								LOG.debug(
+									"Pending file {} is moved into final location",
+									pendingPath, pastCheckpointId
+								);
 							}
 							pendingCheckpointsIt.remove();
 						}
diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/FileStateChangedCallback.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/FileStateChangedCallback.java
new file mode 100644
index 00000000000..cf81ec9cf55
--- /dev/null
+++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/FileStateChangedCallback.java
@@ -0,0 +1,35 @@
+/*
+ * 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.streaming.connectors.fs.bucketing;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+
+/**
+ * The {@code FileStateChangedCallback} is used to perform any additional operations, when {@link BucketingSink}
+ * moves file from one state to another.
+ */
+public interface FileStateChangedCallback {
+
+	void onInProgressToPending(FileSystem fs, Path path) throws IOException;
+
+	void onPendingToFinal(FileSystem fs, Path path) throws IOException;
+}
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java
index ee425ff811e..bffae3c3e2b 100644
--- a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java
+++ b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java
@@ -42,6 +42,7 @@
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.specific.SpecificDatumReader;
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -63,7 +64,10 @@
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.HashMap;
@@ -71,11 +75,13 @@
 
 import static org.apache.flink.streaming.connectors.fs.bucketing.BucketingSinkTestUtils.IN_PROGRESS_SUFFIX;
 import static org.apache.flink.streaming.connectors.fs.bucketing.BucketingSinkTestUtils.PART_PREFIX;
+import static org.apache.flink.streaming.connectors.fs.bucketing.BucketingSinkTestUtils.PART_SUFFIX;
 import static org.apache.flink.streaming.connectors.fs.bucketing.BucketingSinkTestUtils.PENDING_SUFFIX;
 import static org.apache.flink.streaming.connectors.fs.bucketing.BucketingSinkTestUtils.VALID_LENGTH_SUFFIX;
 import static org.apache.flink.streaming.connectors.fs.bucketing.BucketingSinkTestUtils.checkLocalFs;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
 
 /**
  * Tests for the {@link BucketingSink}.
@@ -939,6 +945,170 @@ public void testThatPartIndexIsIncrementedWhenPartSuffixIsNotSpecifiedAndPreviou
 		testThatPartIndexIsIncremented(null, "part-0-0");
 	}
 
+	@Test
+	public void testThatOnInProgressToPendingCallbackIsFiredWhenFilesAreMovedToPendingStateByTimeout()
+		throws Exception {
+
+		File outDir = tempFolder.newFolder();
+		long inactivityInterval = 100;
+
+		String basePath = outDir.getAbsolutePath();
+		BucketingSink<String> sink = new BucketingSink<String>(basePath)
+			.setBucketer(new NoopBucketer())
+			.setWriter(new StringWriter<>())
+			.setInactiveBucketCheckInterval(inactivityInterval)
+			.setInactiveBucketThreshold(inactivityInterval)
+			.setPartPrefix(PART_PREFIX)
+			.setInProgressPrefix("")
+			.setPendingPrefix("")
+			.setValidLengthPrefix("")
+			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
+			.setPendingSuffix(PENDING_SUFFIX)
+			.setValidLengthSuffix(VALID_LENGTH_SUFFIX)
+			.registerFileStateChangedCallback(new FileStateChangedCallbackImpl(basePath));
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness = createTestSink(sink, 1, 0);
+		testHarness.setup();
+		testHarness.open();
+
+		testHarness.setProcessingTime(0L);
+
+		testHarness.processElement(new StreamRecord<>("test1", 1L));
+		testHarness.processElement(new StreamRecord<>("test2", 1L));
+
+		java.nio.file.Path filesInPendingState = outDir.toPath().resolve("report-inProgressToPending.txt");
+		assertThat(Files.exists(filesInPendingState), is(false));
+
+		testHarness.setProcessingTime(101L);
+
+		assertThat(
+			Files.readAllLines(filesInPendingState),
+			containsInAnyOrder("/test1/part-0-0.pending", "/test2/part-0-0.pending")
+		);
+	}
+
+	@Test
+	public void testThatOnInProgressToPendingCallbackIsFiredWhenFilesAreMovedToPendingStateBySize() throws Exception {
+		File outDir = tempFolder.newFolder();
+		long inactivityInterval = 100;
+
+		String basePath = outDir.getAbsolutePath();
+		BucketingSink<String> sink = new BucketingSink<String>(basePath)
+			.setBucketer(new NoopBucketer())
+			.setInactiveBucketCheckInterval(inactivityInterval)
+			.setInactiveBucketThreshold(inactivityInterval)
+			.setPartPrefix(PART_PREFIX)
+			.setInProgressPrefix("")
+			.setPendingPrefix("")
+			.setValidLengthPrefix("")
+			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
+			.setPendingSuffix(PENDING_SUFFIX)
+			.setValidLengthSuffix(VALID_LENGTH_SUFFIX)
+			.setBatchSize(0)
+			.registerFileStateChangedCallback(new FileStateChangedCallbackImpl(basePath));
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness = createTestSink(sink, 1, 0);
+		testHarness.setup();
+		testHarness.open();
+
+		testHarness.setProcessingTime(0L);
+
+		testHarness.processElement(new StreamRecord<>("test1", 1L));
+		testHarness.processElement(new StreamRecord<>("test2", 1L));
+
+		java.nio.file.Path filesInPendingState = outDir.toPath().resolve("report-inProgressToPending.txt");
+		assertThat(Files.exists(filesInPendingState), is(false));
+
+		testHarness.processElement(new StreamRecord<>("test1", 1L));
+		testHarness.processElement(new StreamRecord<>("test2", 1L));
+
+		assertThat(
+			Files.readAllLines(filesInPendingState),
+			containsInAnyOrder("/test1/part-0-0.pending", "/test2/part-0-0.pending")
+		);
+	}
+
+	@Test
+	public void testThatOnInProgressToPendingCallbackIsFiredWhenFunctionIsClosed() throws Exception {
+		File outDir = tempFolder.newFolder();
+		long inactivityInterval = 100;
+
+		String basePath = outDir.getAbsolutePath();
+		@SuppressWarnings("unchecked")
+		BucketingSink<String> sink = new BucketingSink<String>(basePath)
+			.setBucketer(new NoopBucketer())
+			.setInactiveBucketCheckInterval(inactivityInterval)
+			.setInactiveBucketThreshold(inactivityInterval)
+			.setPartPrefix(PART_PREFIX)
+			.setInProgressPrefix("")
+			.setPendingPrefix("")
+			.setValidLengthPrefix("")
+			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
+			.setPendingSuffix(PENDING_SUFFIX)
+			.setValidLengthSuffix(VALID_LENGTH_SUFFIX)
+			.setPartSuffix(PART_SUFFIX)
+			.setBatchSize(0)
+			.registerFileStateChangedCallback(new FileStateChangedCallbackImpl(basePath));
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness = createTestSink(sink, 1, 0);
+		testHarness.setup();
+		testHarness.open();
+
+		testHarness.setProcessingTime(0L);
+
+		testHarness.processElement(new StreamRecord<>("test1", 1L));
+		testHarness.processElement(new StreamRecord<>("test2", 1L));
+
+		java.nio.file.Path filesInPendingState = outDir.toPath().resolve("report-inProgressToPending.txt");
+		assertThat(Files.exists(filesInPendingState), is(false));
+
+		sink.close();
+
+		assertThat(
+			Files.readAllLines(filesInPendingState),
+			containsInAnyOrder("/test1/part-0-0.my.pending", "/test2/part-0-0.my.pending")
+		);
+	}
+
+	@Test
+	public void testThatOnPendingToFinalCallbackIsFiredWhenCheckpointingIsCompleted() throws Exception {
+		File outDir = tempFolder.newFolder();
+		long inactivityInterval = 100;
+
+		String basePath = outDir.getAbsolutePath();
+		BucketingSink<String> sink = new BucketingSink<String>(basePath)
+			.setBucketer(new NoopBucketer())
+			.setWriter(new StringWriter<>())
+			.setInactiveBucketCheckInterval(inactivityInterval)
+			.setInactiveBucketThreshold(inactivityInterval)
+			.setPartPrefix(PART_PREFIX)
+			.setInProgressPrefix("")
+			.setPendingPrefix("")
+			.setValidLengthPrefix("")
+			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
+			.setPendingSuffix(PENDING_SUFFIX)
+			.setValidLengthSuffix(VALID_LENGTH_SUFFIX)
+			.registerFileStateChangedCallback(new FileStateChangedCallbackImpl(basePath));
+
+		OneInputStreamOperatorTestHarness<String, Object> testHarness = createTestSink(sink, 1, 0);
+		testHarness.setup();
+		testHarness.open();
+
+		testHarness.setProcessingTime(0L);
+
+		testHarness.processElement(new StreamRecord<>("test1", 1L));
+		testHarness.processElement(new StreamRecord<>("test2", 1L));
+
+		testHarness.setProcessingTime(101L);
+
+		testHarness.snapshot(0, 0);
+		java.nio.file.Path filesInFinalState = outDir.toPath().resolve("report-pendingToFinal.txt");
+		assertThat(Files.exists(filesInFinalState), is(false));
+
+		testHarness.notifyOfCompletedCheckpoint(0);
+		assertThat(Files.readAllLines(filesInFinalState), containsInAnyOrder("/test1/part-0-0", "/test2/part-0-0"));
+	}
+
 	private void testThatPartIndexIsIncremented(String partSuffix, String existingPartFile) throws Exception {
 		File outDir = tempFolder.newFolder();
 		long inactivityInterval = 100;
@@ -1001,4 +1171,50 @@ public void open(FileSystem fs, Path path) throws IOException {
 		}
 	}
 
+	private static final class NoopBucketer implements Bucketer<String> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Path getBucketPath(Clock clock, Path basePath, String element) {
+			return new Path(basePath, element);
+		}
+	}
+
+	private static final class FileStateChangedCallbackImpl implements FileStateChangedCallback {
+
+		private static final long serialVersionUID = 4806217568474143413L;
+
+		private String basePath;
+
+		public FileStateChangedCallbackImpl(String basePath) {
+			this.basePath = basePath;
+		}
+
+		@Override
+		public void onInProgressToPending(FileSystem fs, Path path) throws IOException {
+			log(fs, path, "inProgressToPending");
+		}
+
+		@Override
+		public void onPendingToFinal(FileSystem fs, Path path) throws IOException {
+			log(fs, path, "pendingToFinal");
+		}
+
+		private void log(FileSystem fs, Path path, String state) throws IOException {
+			StringBuilder content = new StringBuilder();
+			Path result = new Path(new Path(basePath), String.format("report-%s.txt", state));
+			if (fs.exists(result)) {
+				try (InputStream stream = fs.open(result)) {
+					content.append(IOUtils.toString(stream, StandardCharsets.UTF_8));
+					content.append("\n");
+				}
+			}
+			content.append(path.toUri().getPath().replace(basePath, ""));
+
+			try (OutputStream output = fs.create(result, true)) {
+				output.write(content.toString().getBytes(StandardCharsets.UTF_8));
+			}
+		}
+	}
 }
diff --git a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTestUtils.java b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTestUtils.java
index de557b71ff1..a74fd43ef62 100644
--- a/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTestUtils.java
+++ b/flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTestUtils.java
@@ -34,6 +34,7 @@
 public class BucketingSinkTestUtils {
 
 	public static final String PART_PREFIX = "part";
+	public static final String PART_SUFFIX = ".my";
 	public static final String PENDING_SUFFIX = ".pending";
 	public static final String IN_PROGRESS_SUFFIX = ".in-progress";
 	public static final String VALID_LENGTH_SUFFIX = ".valid";


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services