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 2022/11/16 12:21:32 UTC

[GitHub] [flink] Aitozi commented on a diff in pull request #21257: [FLINK-29879][filesystem] introduce operators for merging files in batch mode.

Aitozi commented on code in PR #21257:
URL: https://github.com/apache/flink/pull/21257#discussion_r1023835378


##########
flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/CompactFileUtils.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.connector.file.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.stream.compact.CompactContext;
+import org.apache.flink.connector.file.table.stream.compact.CompactReader;
+import org.apache.flink.connector.file.table.stream.compact.CompactWriter;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Utils for compacting files. */
+public class CompactFileUtils {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CompactFileUtils.class);
+
+    public static final String UNCOMPACTED_PREFIX = ".uncompacted-";
+
+    public static final String COMPACTED_PREFIX = "compacted-";
+
+    /** The function interface for move single file. */
+    @FunctionalInterface
+    public interface SingleFileMvFunction<T, U, R> {

Review Comment:
   Can we can use the `BiFunctionWithException` here ?



##########
flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/CompactFileUtils.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.connector.file.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.stream.compact.CompactContext;
+import org.apache.flink.connector.file.table.stream.compact.CompactReader;
+import org.apache.flink.connector.file.table.stream.compact.CompactWriter;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Utils for compacting files. */
+public class CompactFileUtils {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CompactFileUtils.class);
+
+    public static final String UNCOMPACTED_PREFIX = ".uncompacted-";
+
+    public static final String COMPACTED_PREFIX = "compacted-";
+
+    /** The function interface for move single file. */
+    @FunctionalInterface
+    public interface SingleFileMvFunction<T, U, R> {
+        R apply(T t, U u) throws IOException;
+    }
+
+    /**
+     * Do Compaction: - Target file exists, do nothing. - Can do compaction: - Single file, do
+     * atomic renaming, there are optimizations for FileSystem. - Multiple file, do reading and
+     * writing.
+     */
+    public static <T> Path doCompact(
+            FileSystem fileSystem,
+            String partition,
+            List<Path> paths,
+            Configuration config,
+            SingleFileMvFunction<Path, Path, Boolean> singleFileMvFunc,
+            CompactReader.Factory<T> readerFactory,
+            CompactWriter.Factory<T> writerFactory)
+            throws IOException {
+        if (paths.size() == 0) {
+            return null;
+        }
+
+        Map<Path, Long> inputMap = new HashMap<>();
+        for (Path path : paths) {
+            inputMap.put(path, fileSystem.getFileStatus(path).getLen());
+        }
+
+        Path target = createCompactedFile(paths);
+        if (fileSystem.exists(target)) {
+            return target;
+        }
+
+        checkExist(fileSystem, paths);
+
+        long startMillis = System.currentTimeMillis();
+
+        boolean success = false;
+        if (paths.size() == 1) {
+            // optimizer for single file
+            success = singleFileMvFunc.apply(paths.get(0), target);
+        }
+
+        if (!success) {

Review Comment:
   The condition is a bit strange, what about 
   
   ```
           if (paths.size() == 1) {
               // optimizer for single file
               singleFileMvFunc.apply(paths.get(0), target);
           } else {
               doMultiFilesCompact(
                       partition, paths, target, config, fileSystem, readerFactory, writerFactory);
           }
   ```



##########
flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/batch/compact/BatchFileWriter.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.flink.connector.file.table.batch.compact;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.OutputFormatFactory;
+import org.apache.flink.connector.file.table.PartitionComputer;
+import org.apache.flink.connector.file.table.PartitionTempFileManager;
+import org.apache.flink.connector.file.table.PartitionWriter;
+import org.apache.flink.connector.file.table.PartitionWriterFactory;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CoordinatorInput;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.InputFile;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.functions.sink.filesystem.OutputFileConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.TableException;
+
+import java.util.LinkedHashMap;
+
+/**
+ * An operator for writing files in batch mode. Once creating a new file to write, the writing
+ * operator will emit the written file to upstream.
+ */
+public class BatchFileWriter<T> extends AbstractStreamOperator<CoordinatorInput>
+        implements OneInputStreamOperator<T, CoordinatorInput>, BoundedOneInput {

Review Comment:
   IDE complains "does not define a 'serialVersionUID' field"



##########
flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/batch/compact/BatchFileWriter.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.flink.connector.file.table.batch.compact;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.OutputFormatFactory;
+import org.apache.flink.connector.file.table.PartitionComputer;
+import org.apache.flink.connector.file.table.PartitionTempFileManager;
+import org.apache.flink.connector.file.table.PartitionWriter;
+import org.apache.flink.connector.file.table.PartitionWriterFactory;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.CoordinatorInput;
+import org.apache.flink.connector.file.table.stream.compact.CompactMessages.InputFile;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.functions.sink.filesystem.OutputFileConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.api.TableException;
+
+import java.util.LinkedHashMap;
+
+/**
+ * An operator for writing files in batch mode. Once creating a new file to write, the writing
+ * operator will emit the written file to upstream.

Review Comment:
   should this be `to downstream` ?



##########
flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/CompactFileUtils.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.connector.file.table;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.table.stream.compact.CompactContext;
+import org.apache.flink.connector.file.table.stream.compact.CompactReader;
+import org.apache.flink.connector.file.table.stream.compact.CompactWriter;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Utils for compacting files. */
+public class CompactFileUtils {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CompactFileUtils.class);
+
+    public static final String UNCOMPACTED_PREFIX = ".uncompacted-";
+
+    public static final String COMPACTED_PREFIX = "compacted-";
+
+    /** The function interface for move single file. */
+    @FunctionalInterface
+    public interface SingleFileMvFunction<T, U, R> {
+        R apply(T t, U u) throws IOException;
+    }
+
+    /**
+     * Do Compaction: - Target file exists, do nothing. - Can do compaction: - Single file, do
+     * atomic renaming, there are optimizations for FileSystem. - Multiple file, do reading and
+     * writing.
+     */
+    public static <T> Path doCompact(
+            FileSystem fileSystem,
+            String partition,
+            List<Path> paths,
+            Configuration config,
+            SingleFileMvFunction<Path, Path, Boolean> singleFileMvFunc,
+            CompactReader.Factory<T> readerFactory,
+            CompactWriter.Factory<T> writerFactory)
+            throws IOException {
+        if (paths.size() == 0) {
+            return null;
+        }
+
+        Map<Path, Long> inputMap = new HashMap<>();
+        for (Path path : paths) {
+            inputMap.put(path, fileSystem.getFileStatus(path).getLen());
+        }
+
+        Path target = createCompactedFile(paths);
+        if (fileSystem.exists(target)) {
+            return target;
+        }
+
+        checkExist(fileSystem, paths);
+
+        long startMillis = System.currentTimeMillis();
+
+        boolean success = false;
+        if (paths.size() == 1) {
+            // optimizer for single file
+            success = singleFileMvFunc.apply(paths.get(0), target);
+        }
+
+        if (!success) {
+            doMultiFilesCompact(
+                    partition, paths, target, config, fileSystem, readerFactory, writerFactory);
+        }
+
+        Map<Path, Long> targetMap = new HashMap<>();
+        targetMap.put(target, fileSystem.getFileStatus(target).getLen());

Review Comment:
   the `targetMap` seems not necessary



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org