Skip to content

Commit

Permalink
Add first test
Browse files Browse the repository at this point in the history
  • Loading branch information
junyuc25 authored and yihua committed Sep 6, 2022
1 parent eb9b695 commit d3f7897
Showing 1 changed file with 26 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.hudi.functional

import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hudi.HoodieConversionUtils.toJavaOption
import org.apache.hudi.QuickstartUtils.{convertToStringList, getQuickstartWriteConfigs}
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.table.timeline.HoodieInstant
Expand All @@ -28,13 +29,14 @@ import org.apache.hudi.common.testutils.RawTripTestPayload.{deleteRecordsToStrin
import org.apache.hudi.common.util
import org.apache.hudi.common.util.PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.config.metrics.HoodieMetricsConfig
import org.apache.hudi.exception.{HoodieException, HoodieUpsertException}
import org.apache.hudi.keygen._
import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config
import org.apache.hudi.metrics.Metrics
import org.apache.hudi.testutils.HoodieClientTestBase
import org.apache.hudi.util.JFunction
import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils}
import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, QuickstartUtils}
import org.apache.spark.sql._
import org.apache.spark.sql.functions.{col, concat, lit, udf}
import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
Expand Down Expand Up @@ -1026,4 +1028,27 @@ class TestCOWDataSource extends HoodieClientTestBase {
.saveAsTable("hoodie_test")
assertEquals(spark.read.format("hudi").load(basePath).count(), 9)
}

@Test
def testInsertMetricsReporterEnabled(): Unit = {
val dataGenerator = new QuickstartUtils.DataGenerator()
val records = convertToStringList(dataGenerator.generateInserts( 10))
// println("Printing Records: " + records)
val recordsRDD = spark.sparkContext.parallelize(records, 2)
val inputDF = spark.read.json(sparkSession.createDataset(recordsRDD)(Encoders.STRING))
inputDF.write.format("hudi")
.options(getQuickstartWriteConfigs)
.option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "uuid")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionpath")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "ts")
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(HoodieWriteConfig.TBL_NAME.key, "hoodie_test")
.option(HoodieMetricsConfig.TURN_METRICS_ON.key(), "true")
.option(HoodieMetricsConfig.METRICS_REPORTER_TYPE_VALUE.key(), "JMX")
.mode(SaveMode.Overwrite)
.save(basePath)

assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
assertEquals(false, Metrics.isInitialized, "Metrics should be shutdown")
}
}

0 comments on commit d3f7897

Please sign in to comment.