1
0

[HUDI-2451] On windows client with hdfs server for wrong file separator (#3687)

Co-authored-by: yao.zhou <yao.zhou@linkflowtech.com>
This commit is contained in:
Carl-Zhou-CN
2021-09-26 21:51:27 +08:00
committed by GitHub
parent bc4966ea73
commit aa546554ff
12 changed files with 27 additions and 36 deletions

View File

@@ -30,7 +30,6 @@ import org.apache.hadoop.fs.Path;
import javax.annotation.Nullable;
import java.io.File;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@@ -89,7 +88,7 @@ public class FileIndex {
}
List<Map<String, String>> partitions = new ArrayList<>();
for (String partitionPath : partitionPaths) {
String[] paths = partitionPath.split(File.separator);
String[] paths = partitionPath.split(Path.SEPARATOR);
Map<String, String> partitionMapping = new LinkedHashMap<>();
if (hivePartition) {
Arrays.stream(paths).forEach(p -> {

View File

@@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@@ -83,7 +82,7 @@ public class FilePathUtils {
* @param partitionKVs The partition key value mapping
* @param hivePartition Whether the partition path is with Hive style,
* e.g. {partition key} = {partition value}
* @param sepSuffix Whether to append the file separator as suffix
* @param sepSuffix Whether to append the path separator as suffix
* @return an escaped, valid partition name
*/
public static String generatePartitionPath(
@@ -97,7 +96,7 @@ public class FilePathUtils {
int i = 0;
for (Map.Entry<String, String> e : partitionKVs.entrySet()) {
if (i > 0) {
suffixBuf.append(File.separator);
suffixBuf.append(Path.SEPARATOR);
}
if (hivePartition) {
suffixBuf.append(escapePathName(e.getKey()));
@@ -107,7 +106,7 @@ public class FilePathUtils {
i++;
}
if (sepSuffix) {
suffixBuf.append(File.separator);
suffixBuf.append(Path.SEPARATOR);
}
return suffixBuf.toString();
}

View File

@@ -27,8 +27,6 @@ import org.apache.hudi.util.StreamerUtil;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
import java.io.File;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
@@ -51,7 +49,7 @@ public class TestUtils {
public static String getSplitPartitionPath(MergeOnReadInputSplit split) {
assertTrue(split.getLogPaths().isPresent());
final String logPath = split.getLogPaths().get().get(0);
String[] paths = logPath.split(File.separator);
String[] paths = logPath.split(Path.SEPARATOR);
return paths[paths.length - 2];
}