You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by kk...@apache.org on 2019/01/09 09:19:57 UTC
[flink] branch master updated: [FLINK-11140][fs-connector] Fix
empty child path check in Buckets.
This is an automated email from the ASF dual-hosted git repository.
kkloudas pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new 3702029 [FLINK-11140][fs-connector] Fix empty child path check in Buckets.
3702029 is described below
commit 3702029f45b7034b767e2b7eb01601c7f76ab35e
Author: Matrix42 <93...@qq.com>
AuthorDate: Tue Jan 8 14:43:19 2019 +0100
[FLINK-11140][fs-connector] Fix empty child path check in Buckets.
This closes #7287.
---
.../api/functions/sink/filesystem/Buckets.java | 9 ++-
.../sink/filesystem/BucketAssignerITCases.java | 66 ++++++++++++++++++++++
.../api/functions/sink/filesystem/BucketsTest.java | 10 ++--
3 files changed, 78 insertions(+), 7 deletions(-)
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java
index d08bc2a..7b35e50 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java
@@ -254,7 +254,7 @@ public class Buckets<IN, BucketID> {
}
}
- void onElement(final IN value, final SinkFunction.Context context) throws Exception {
+ Bucket<IN, BucketID> onElement(final IN value, final SinkFunction.Context context) throws Exception {
final long currentProcessingTime = context.currentProcessingTime();
// setting the values in the bucketer context
@@ -272,6 +272,7 @@ public class Buckets<IN, BucketID> {
// another part file for the bucket, if we start from 0 we may overwrite previous parts.
this.maxPartCounter = Math.max(maxPartCounter, bucket.getPartCounter());
+ return bucket;
}
private Bucket<IN, BucketID> getOrCreateBucketForBucketId(final BucketID bucketId) throws IOException {
@@ -304,7 +305,11 @@ public class Buckets<IN, BucketID> {
}
private Path assembleBucketPath(BucketID bucketId) {
- return new Path(basePath, bucketId.toString());
+ final String child = bucketId.toString();
+ if ("".equals(child)) {
+ return basePath;
+ }
+ return new Path(basePath, child);
}
/**
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java
new file mode 100644
index 0000000..5ae57ce
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java
@@ -0,0 +1,66 @@
+/*
+ * 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.api.functions.sink.filesystem;
+
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.BasePathBucketAssigner;
+import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy;
+
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+
+/**
+ * Integration tests for {@link BucketAssigner bucket assigners}.
+ */
+public class BucketAssignerITCases {
+
+ @ClassRule
+ public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder();
+
+ @Test
+ public void testAssembleBucketPath() throws Exception {
+ final File outDir = TEMP_FOLDER.newFolder();
+ final Path basePath = new Path(outDir.toURI());
+ final long time = 1000L;
+
+ final RollingPolicy<String, String> rollingPolicy =
+ DefaultRollingPolicy
+ .create()
+ .withMaxPartSize(7L)
+ .build();
+
+ final Buckets<String, String> buckets = new Buckets<>(
+ basePath,
+ new BasePathBucketAssigner<>(),
+ new DefaultBucketFactoryImpl<>(),
+ new RowWisePartWriter.Factory<>(new SimpleStringEncoder<>()),
+ rollingPolicy,
+ 0
+ );
+
+ Bucket<String, String> bucket =
+ buckets.onElement("abc", new TestUtils.MockSinkContext(time, time, time));
+ Assert.assertEquals(new Path(basePath.toUri()), bucket.getBucketPath());
+ }
+}
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java
index aee3621..8369b90 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java
@@ -188,7 +188,7 @@ public class BucketsTest {
// it takes the current processing time of the context for the creation time,
// and for the last modification time.
- buckets.onElement("test", new TestUtils.MockSinkContext(1L, 2L , 3L));
+ buckets.onElement("test", new TestUtils.MockSinkContext(1L, 2L, 3L));
// now it should roll
buckets.onProcessingTime(7L);
@@ -214,13 +214,13 @@ public class BucketsTest {
final Path path = new Path(outDir.toURI());
final OnProcessingTimePolicy<String, String> rollOnProcessingTimeCountingPolicy =
- new OnProcessingTimePolicy<>(2L);
+ new OnProcessingTimePolicy<>(2L);
final Buckets<String, String> buckets =
createBuckets(path, rollOnProcessingTimeCountingPolicy, 0);
// it takes the current processing time of the context for the creation time, and for the last modification time.
- buckets.onElement("test", new TestUtils.MockSinkContext(1L, 2L , 3L));
+ buckets.onElement("test", new TestUtils.MockSinkContext(1L, 2L, 3L));
// now it should roll
buckets.onProcessingTime(7L);
@@ -244,7 +244,7 @@ public class BucketsTest {
createBuckets(path, rollOnProcessingTimeCountingPolicy, 0);
// it takes the current processing time of the context for the creation time, and for the last modification time.
- buckets.onElement("test", new TestUtils.MockSinkContext(1L, 2L , 3L));
+ buckets.onElement("test", new TestUtils.MockSinkContext(1L, 2L, 3L));
Assert.assertEquals(1L, buckets.getActiveBuckets().get("test").getPartCounter());
// now it should roll
@@ -257,7 +257,7 @@ public class BucketsTest {
Assert.assertTrue(buckets.getActiveBuckets().isEmpty());
- buckets.onElement("test", new TestUtils.MockSinkContext(2L, 3L , 4L));
+ buckets.onElement("test", new TestUtils.MockSinkContext(2L, 3L, 4L));
Assert.assertEquals(2L, buckets.getActiveBuckets().get("test").getPartCounter());
}