Skip to content
Snippets Groups Projects
Commit c5eb1df7 authored by Holden Karau's avatar Holden Karau Committed by Sean Owen
Browse files

[SPARK-16814][SQL] Fix deprecated parquet constructor usage

## What changes were proposed in this pull request?

Replace deprecated ParquetWriter with the new builders

## How was this patch tested?

Existing tests

Author: Holden Karau <holden@us.ibm.com>

Closes #14419 from holdenk/SPARK-16814-fix-deprecated-parquet-constructor-usage.
parent 4775eb41
No related branches found
No related tags found
No related merge requests found
......@@ -27,6 +27,7 @@ import org.apache.avro.Schema
import org.apache.avro.generic.IndexedRecord
import org.apache.hadoop.fs.Path
import org.apache.parquet.avro.AvroParquetWriter
import org.apache.parquet.hadoop.ParquetWriter
import org.apache.spark.sql.Row
import org.apache.spark.sql.execution.datasources.parquet.test.avro._
......@@ -35,14 +36,14 @@ import org.apache.spark.sql.test.SharedSQLContext
class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with SharedSQLContext {
private def withWriter[T <: IndexedRecord]
(path: String, schema: Schema)
(f: AvroParquetWriter[T] => Unit): Unit = {
(f: ParquetWriter[T] => Unit): Unit = {
logInfo(
s"""Writing Avro records with the following Avro schema into Parquet file:
|
|${schema.toString(true)}
""".stripMargin)
val writer = new AvroParquetWriter[T](new Path(path), schema)
val writer = AvroParquetWriter.builder[T](new Path(path)).withSchema(schema).build()
try f(writer) finally writer.close()
}
......
......@@ -119,8 +119,18 @@ private[sql] object ParquetCompatibilityTest {
metadata: Map[String, String],
recordWriters: (RecordConsumer => Unit)*): Unit = {
val messageType = MessageTypeParser.parseMessageType(schema)
val writeSupport = new DirectWriteSupport(messageType, metadata)
val parquetWriter = new ParquetWriter[RecordConsumer => Unit](new Path(path), writeSupport)
val testWriteSupport = new DirectWriteSupport(messageType, metadata)
/**
* Provide a builder for constructing a parquet writer - after PARQUET-248 directly constructing
* the writer is deprecated and should be done through a builder. The default builders include
* Avro - but for raw Parquet writing we must create our own builder.
*/
class ParquetWriterBuilder() extends
ParquetWriter.Builder[RecordConsumer => Unit, ParquetWriterBuilder](new Path(path)) {
override def getWriteSupport(conf: Configuration) = testWriteSupport
override def self() = this
}
val parquetWriter = new ParquetWriterBuilder().build()
try recordWriters.foreach(parquetWriter.write) finally parquetWriter.close()
}
}
......@@ -325,8 +325,20 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext {
|}
""".stripMargin)
val writeSupport = new TestGroupWriteSupport(schema)
val writer = new ParquetWriter[Group](path, writeSupport)
val testWriteSupport = new TestGroupWriteSupport(schema)
/**
* Provide a builder for constructing a parquet writer - after PARQUET-248 directly
* constructing the writer is deprecated and should be done through a builder. The default
* builders include Avro - but for raw Parquet writing we must create our own builder.
*/
class ParquetWriterBuilder() extends
ParquetWriter.Builder[Group, ParquetWriterBuilder](path) {
override def getWriteSupport(conf: Configuration) = testWriteSupport
override def self() = this
}
val writer = new ParquetWriterBuilder().build()
(0 until 10).foreach { i =>
val record = new SimpleGroup(schema)
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment