aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test
diff options
context:
space:
mode:
authorHolden Karau <holden@us.ibm.com>2016-08-03 17:08:51 -0700
committerSean Owen <sowen@cloudera.com>2016-08-03 17:08:51 -0700
commitc5eb1df72fea2ecc71369f13416a8aee040b55d2 (patch)
treebed6d38cd9c8e6fb02dc187396065a37711f54ff /sql/core/src/test
parent4775eb414fa8285cfdc301e52dac52a2ef64c9e1 (diff)
downloadspark-c5eb1df72fea2ecc71369f13416a8aee040b55d2.tar.gz
spark-c5eb1df72fea2ecc71369f13416a8aee040b55d2.tar.bz2
spark-c5eb1df72fea2ecc71369f13416a8aee040b55d2.zip
[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.
Diffstat (limited to 'sql/core/src/test')
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala5
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala14
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala16
3 files changed, 29 insertions, 6 deletions
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala
index 6509e04e85..1b99fbedca 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala
@@ -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()
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala
index 57cd70e191..a43a856d16 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala
@@ -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()
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
index fc9ce6bb30..0f74094699 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
@@ -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)