-
Notifications
You must be signed in to change notification settings - Fork 265
chore: Make parquet reader options Comet options instead of Hadoop options #968
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
ce603ea
0d396f7
02ca31c
575bd78
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -67,6 +67,8 @@ | |
| import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; | ||
| import org.apache.parquet.schema.Types; | ||
|
|
||
| import org.apache.comet.CometConf; | ||
|
|
||
| import static org.apache.parquet.column.Encoding.*; | ||
| import static org.apache.parquet.format.converter.ParquetMetadataConverter.MAX_STATS_SIZE; | ||
| import static org.junit.Assert.*; | ||
|
|
@@ -615,12 +617,12 @@ public void testColumnIndexReadWrite() throws Exception { | |
| @Test | ||
| public void testWriteReadMergeScanRange() throws Throwable { | ||
| Configuration conf = new Configuration(); | ||
| conf.set(ReadOptions.COMET_IO_MERGE_RANGES, Boolean.toString(true)); | ||
| conf.set(CometConf.COMET_IO_MERGE_RANGES().key(), Boolean.toString(true)); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This test is setting the config values in a Hadoop Configuration still, and not setting them in the Spark config. Would it make sense to update the test?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There is no spark context in this test. I've added a new test with the configuration set thru the spark config |
||
| // Set the merge range delta so small that ranges do not get merged | ||
| conf.set(ReadOptions.COMET_IO_MERGE_RANGES_DELTA, Integer.toString(1024)); | ||
| conf.set(CometConf.COMET_IO_MERGE_RANGES_DELTA().key(), Integer.toString(1024)); | ||
| testReadWrite(conf, 2, 1024); | ||
| // Set the merge range delta so large that all ranges get merged | ||
| conf.set(ReadOptions.COMET_IO_MERGE_RANGES_DELTA, Integer.toString(1024 * 1024)); | ||
| conf.set(CometConf.COMET_IO_MERGE_RANGES_DELTA().key(), Integer.toString(1024 * 1024)); | ||
| testReadWrite(conf, 2, 1024); | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1280,6 +1280,93 @@ abstract class ParquetReadSuite extends CometTestBase { | |
| } | ||
| } | ||
| } | ||
|
|
||
| test("test merge scan range") { | ||
| def makeRawParquetFile(path: Path, n: Int): Seq[Option[Int]] = { | ||
| val dictionaryPageSize = 1024 | ||
| val pageRowCount = 500 | ||
| val schemaStr = | ||
| """ | ||
| |message root { | ||
| | optional int32 _1(INT_16); | ||
| | optional int32 _2; | ||
| | optional int64 _3; | ||
| |} | ||
| """.stripMargin | ||
|
|
||
| val schema = MessageTypeParser.parseMessageType(schemaStr) | ||
| val writer = createParquetWriter( | ||
| schema, | ||
| path, | ||
| dictionaryEnabled = true, | ||
| dictionaryPageSize = dictionaryPageSize, | ||
| pageRowCountLimit = pageRowCount) | ||
|
|
||
| val rand = scala.util.Random | ||
| val expected = (0 until n).map { i => | ||
| // use a single value for the first page, to make sure dictionary encoding kicks in | ||
| val value = if (i < pageRowCount) i % 8 else i | ||
| if (rand.nextBoolean()) None | ||
| else Some(value) | ||
| } | ||
|
|
||
| expected.foreach { opt => | ||
| val record = new SimpleGroup(schema) | ||
| opt match { | ||
| case Some(i) => | ||
| record.add(0, i.toShort) | ||
| record.add(1, i) | ||
| record.add(2, i.toLong) | ||
| case _ => | ||
| } | ||
| writer.write(record) | ||
| } | ||
|
|
||
| writer.close() | ||
| expected | ||
| } | ||
|
|
||
| Seq(16, 128).foreach { batchSize => | ||
| Seq(1024, 1024 * 1024).foreach { mergeRangeDelta => | ||
| { | ||
| withSQLConf( | ||
| CometConf.COMET_BATCH_SIZE.key -> batchSize.toString, | ||
| CometConf.COMET_IO_MERGE_RANGES.key -> "true", | ||
| CometConf.COMET_IO_MERGE_RANGES_DELTA.key -> mergeRangeDelta.toString) { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is it possible to test that this config is actually making into ReadOptions? If I comment out all of the code in ReadOptions that reads these configs, the test still passes. Perhaps we just need a specific test to show that setting the config on a Spark context causes a change to the ReadOptions?
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I added some debug logging and I do see that it is working correctly, but would be good to have a test to confirm (and prevent regressions)
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I added an additional check for the config. The configuration passed in to read options is not accessible but I tried to simulate the next best thing. See if that makes sense.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks, Parth. LGTM. Looks like you need to run
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks @andygrove. Fixed style. |
||
| withTempDir { dir => | ||
| val path = new Path(dir.toURI.toString, "part-r-0.parquet") | ||
| val expected = makeRawParquetFile(path, 10000) | ||
| val schema = StructType( | ||
| Seq(StructField("_1", ShortType, true), StructField("_3", LongType, true))) | ||
| readParquetFile(path.toString, Some(schema)) { df => | ||
| { | ||
| // CometScanExec calls sessionState.newHadoopConfWithOptions which copies | ||
| // the sqlConf and some additional options to the hadoopConf and then | ||
| // uses the result to create the inputRDD (https://github.com/apache/datafusion-comet/blob/3783faaa01078a35bee93b299368f8c72869198d/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala#L181). | ||
| // We don't have access to the created hadoop Conf, but can confirm that the | ||
| // result does contain the correct configuration | ||
| assert( | ||
| df.sparkSession.sessionState | ||
| .newHadoopConfWithOptions(Map.empty) | ||
| .get(CometConf.COMET_IO_MERGE_RANGES_DELTA.key) | ||
| .equals(mergeRangeDelta.toString)) | ||
| checkAnswer( | ||
| df, | ||
| expected.map { | ||
| case None => | ||
| Row(null, null) | ||
| case Some(i) => | ||
| Row(i.toShort, i.toLong) | ||
| }) | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| def testScanner(cometEnabled: String, scanner: String, v1: Option[String] = None): Unit = { | ||
| withSQLConf( | ||
| CometConf.COMET_ENABLED.key -> cometEnabled, | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is reading from the Hadoop conf. If I set the new configs on my Spark context, how would they get propagated to the Hadoop conf?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Spark sql copies over configs that are not from spark into the hadoop config when the sql context is created. There are other settings that also use this ( e.g.
COMET_USE_LAZY_MATERIALIZATION,COMET_SCAN_PREFETCH_ENABLED)