[HUDI-2107] Support Read Log Only MOR Table For Spark (#3193)
This commit is contained in:
@@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -17,20 +17,24 @@
|
||||
|
||||
package org.apache.hudi.functional
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import org.apache.hudi.DataSourceWriteOptions.{KEYGENERATOR_CLASS_OPT_KEY, PARTITIONPATH_FIELD_OPT_KEY, PAYLOAD_CLASS_OPT_KEY, PRECOMBINE_FIELD_OPT_KEY, RECORDKEY_FIELD_OPT_KEY}
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.DefaultHoodieRecordPayload
|
||||
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieTableType}
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils}
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType
|
||||
import org.apache.hudi.keygen.NonpartitionedKeyGenerator
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieClientTestBase}
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
|
||||
@@ -677,4 +681,23 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
|
||||
assertEquals(partitionCounts("2021/03/03"), count7)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testReadLogOnlyMergeOnReadTable(): Unit = {
|
||||
initMetaClient(HoodieTableType.MERGE_ON_READ)
|
||||
val records1 = dataGen.generateInsertsContainsAllPartitions("000", 20)
|
||||
val inputDF = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
// Use InMemoryIndex to generate log only mor table.
|
||||
.option(HoodieIndexConfig.INDEX_TYPE_PROP.key, IndexType.INMEMORY.toString)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
// There should no base file in the file list.
|
||||
assertTrue(DataSourceTestUtils.isLogFileOnly(basePath))
|
||||
// Test read log only mor table.
|
||||
assertEquals(20, spark.read.format("hudi").load(basePath).count())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,91 @@
|
||||
/*
|
||||
* 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.spark.sql.hudi
|
||||
|
||||
import org.apache.hudi.testutils.DataSourceTestUtils
|
||||
|
||||
class TestMereIntoLogOnlyTable extends TestHoodieSqlBase {
|
||||
|
||||
test("Test Query Log Only MOR Table") {
|
||||
withTempDir { tmp =>
|
||||
// Create table with INMEMORY index to generate log only mor table.
|
||||
val tableName = generateTableName
|
||||
spark.sql(
|
||||
s"""
|
||||
|create table $tableName (
|
||||
| id int,
|
||||
| name string,
|
||||
| price double,
|
||||
| ts long
|
||||
|) using hudi
|
||||
| location '${tmp.getCanonicalPath}'
|
||||
| options (
|
||||
| primaryKey ='id',
|
||||
| type = 'mor',
|
||||
| preCombineField = 'ts',
|
||||
| hoodie.index.type = 'INMEMORY',
|
||||
| hoodie.compact.inline = 'true'
|
||||
| )
|
||||
""".stripMargin)
|
||||
spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)")
|
||||
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000)")
|
||||
spark.sql(s"insert into $tableName values(3, 'a3', 10, 1000)")
|
||||
// 3 commits will not trigger compaction, so it should be log only.
|
||||
assertResult(true)(DataSourceTestUtils.isLogFileOnly(tmp.getCanonicalPath))
|
||||
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
|
||||
Seq(1, "a1", 10.0, 1000),
|
||||
Seq(2, "a2", 10.0, 1000),
|
||||
Seq(3, "a3", 10.0, 1000)
|
||||
)
|
||||
spark.sql(
|
||||
s"""
|
||||
|merge into $tableName h0
|
||||
|using (
|
||||
| select 1 as id, 'a1' as name, 11 as price, 1001 as ts
|
||||
| ) s0
|
||||
| on h0.id = s0.id
|
||||
| when matched then update set *
|
||||
|""".stripMargin)
|
||||
// 4 commits will not trigger compaction, so it should be log only.
|
||||
assertResult(true)(DataSourceTestUtils.isLogFileOnly(tmp.getCanonicalPath))
|
||||
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
|
||||
Seq(1, "a1", 11.0, 1001),
|
||||
Seq(2, "a2", 10.0, 1000),
|
||||
Seq(3, "a3", 10.0, 1000)
|
||||
)
|
||||
spark.sql(
|
||||
s"""
|
||||
|merge into $tableName h0
|
||||
|using (
|
||||
| select 4 as id, 'a4' as name, 11 as price, 1000 as ts
|
||||
| ) s0
|
||||
| on h0.id = s0.id
|
||||
| when matched then update set *
|
||||
|""".stripMargin)
|
||||
|
||||
// 5 commits will trigger compaction.
|
||||
assertResult(false)(DataSourceTestUtils.isLogFileOnly(tmp.getCanonicalPath))
|
||||
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
|
||||
Seq(1, "a1", 11.0, 1001),
|
||||
Seq(2, "a2", 10.0, 1000),
|
||||
Seq(3, "a3", 10.0, 1000),
|
||||
Seq(4, "a4", 11.0, 1000)
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user