1
0

[HUDI-2107] Support Read Log Only MOR Table For Spark (#3193)

This commit is contained in:
pengzhiwei
2021-07-12 17:31:23 +08:00
committed by GitHub
parent 5804ad8e32
commit ca440ccf88
6 changed files with 214 additions and 71 deletions

View File

@@ -18,6 +18,12 @@
package org.apache.hudi.testutils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.avro.Schema;
@@ -118,4 +124,27 @@ public class DataSourceTestUtils {
}
return rows;
}
/**
* Test if there is only log files exists in the table.
*/
public static boolean isLogFileOnly(String basePath) throws IOException {
Configuration conf = new Configuration();
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
.setConf(conf).setBasePath(basePath)
.build();
String baseDataFormat = metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
Path path = new Path(basePath);
FileSystem fs = path.getFileSystem(conf);
RemoteIterator<LocatedFileStatus> files = fs.listFiles(path, true);
while (files.hasNext()) {
LocatedFileStatus file = files.next();
if (file.isFile()) {
if (file.getPath().toString().endsWith(baseDataFormat)) {
return false;
}
}
}
return true;
}
}