1
0

[HUDI-1055] Remove hardcoded parquet in tests (#2740)

* Remove hardcoded parquet in tests
* Use DataFileUtils.getInstance
* Renaming DataFileUtils to BaseFileUtils

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
TeRS-K
2021-05-11 13:01:45 -04:00
committed by GitHub
parent ac72470e10
commit be9db2c4f5
42 changed files with 359 additions and 218 deletions

View File

@@ -25,8 +25,9 @@ import org.apache.hudi.avro.HoodieAvroWriteSupport
import org.apache.hudi.client.SparkTaskContextSupplier
import org.apache.hudi.common.HoodieJsonPayload
import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory}
import org.apache.hudi.common.model.HoodieFileFormat
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.util.ParquetUtils
import org.apache.hudi.common.util.BaseFileUtils
import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig}
import org.apache.hudi.io.storage.{HoodieAvroParquetConfig, HoodieParquetWriter}
import org.apache.parquet.avro.AvroSchemaConverter
@@ -40,7 +41,7 @@ import scala.collection.mutable._
object SparkHelpers {
@throws[Exception]
def skipKeysAndWriteNewFile(instantTime: String, fs: FileSystem, sourceFile: Path, destinationFile: Path, keysToSkip: Set[String]) {
val sourceRecords = ParquetUtils.readAvroRecords(fs.getConf, sourceFile)
val sourceRecords = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET).readAvroRecords(fs.getConf, sourceFile)
val schema: Schema = sourceRecords.get(0).getSchema
val filter: BloomFilter = BloomFilterFactory.createBloomFilter(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_NUM_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP.toDouble,
HoodieIndexConfig.DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_INDEX_FILTER_TYPE);
@@ -125,7 +126,7 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) {
* @return
*/
def fileKeysAgainstBF(conf: Configuration, sqlContext: SQLContext, file: String): Boolean = {
val bf = ParquetUtils.readBloomFilterFromParquetMetadata(conf, new Path(file))
val bf = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET).readBloomFilterFromMetadata(conf, new Path(file))
val foundCount = sqlContext.parquetFile(file)
.select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`")
.collect().count(r => !bf.mightContain(r.getString(0)))

View File

@@ -70,7 +70,7 @@ public class TestRollbacksCommand extends AbstractShellIntegrationTest {
tablePath, tableName, HoodieTableType.MERGE_ON_READ.name(),
"", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload");
metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient());
//Create some commits files and parquet files
//Create some commits files and base files
Map<String, String> partitionAndFileId = new HashMap<String, String>() {
{
put(DEFAULT_FIRST_PARTITION_PATH, "file-1");

View File

@@ -60,7 +60,7 @@ public class TestUpgradeDowngradeCommand extends AbstractShellIntegrationTest {
tablePath, tableName, HoodieTableType.COPY_ON_WRITE.name(),
"", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload");
metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient());
//Create some commits files and parquet files
//Create some commits files and base files
HoodieTestTable.of(metaClient)
.withPartitionMetaFiles(DEFAULT_PARTITION_PATHS)
.addCommit("100")

View File

@@ -71,7 +71,7 @@ public class ITTestCommitsCommand extends AbstractShellIntegrationTest {
*/
@Test
public void testRollbackCommit() throws Exception {
//Create some commits files and parquet files
//Create some commits files and base files
Map<String, String> partitionAndFileId = new HashMap<String, String>() {
{
put(DEFAULT_FIRST_PARTITION_PATH, "file-1");