Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,94 @@
/*
* 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.avro;

import java.io.IOException;
import java.io.OutputStream;
import java.util.Map;

import org.apache.avro.Schema;
import org.apache.avro.file.CodecFactory;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapreduce.AvroKeyOutputFormat;
import org.apache.avro.mapreduce.Syncable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;

// A variant of `AvroKeyOutputFormat`, which is used to inject the custom `RecordWriterFactory` so
// that we can set avro file metadata.
public class SparkAvroKeyOutputFormat extends AvroKeyOutputFormat<GenericRecord> {
public SparkAvroKeyOutputFormat(Map<String, String> metadata) {
super(new SparkRecordWriterFactory(metadata));
}

static class SparkRecordWriterFactory extends RecordWriterFactory<GenericRecord> {
private final Map<String, String> metadata;
SparkRecordWriterFactory(Map<String, String> metadata) {
this.metadata = metadata;
}

protected RecordWriter<AvroKey<GenericRecord>, NullWritable> create(
Schema writerSchema,
GenericData dataModel,
CodecFactory compressionCodec,
OutputStream outputStream,
int syncInterval) throws IOException {
return new SparkAvroKeyRecordWriter(
writerSchema, dataModel, compressionCodec, outputStream, syncInterval, metadata);
}
}
}

// This a fork of org.apache.avro.mapreduce.AvroKeyRecordWriter, in order to set file metadata.
class SparkAvroKeyRecordWriter<T> extends RecordWriter<AvroKey<T>, NullWritable>
implements Syncable {

private final DataFileWriter<T> mAvroFileWriter;

SparkAvroKeyRecordWriter(
Schema writerSchema,
GenericData dataModel,
CodecFactory compressionCodec,
OutputStream outputStream,
int syncInterval,
Map<String, String> metadata) throws IOException {
this.mAvroFileWriter = new DataFileWriter(dataModel.createDatumWriter(writerSchema));
for (Map.Entry<String, String> entry : metadata.entrySet()) {
this.mAvroFileWriter.setMeta(entry.getKey(), entry.getValue());
}
this.mAvroFileWriter.setCodec(compressionCodec);
this.mAvroFileWriter.setSyncInterval(syncInterval);
this.mAvroFileWriter.create(writerSchema, outputStream);
}

public void write(AvroKey<T> record, NullWritable ignore) throws IOException {
this.mAvroFileWriter.append(record.datum());
}

public void close(TaskAttemptContext context) throws IOException {
this.mAvroFileWriter.close();
}

public long sync() throws IOException {
return this.mAvroFileWriter.sync();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,17 @@ package org.apache.spark.sql.avro

import java.io.{IOException, OutputStream}

import scala.collection.JavaConverters._

import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.avro.mapred.AvroKey
import org.apache.avro.mapreduce.AvroKeyOutputFormat
import org.apache.hadoop.fs.Path
import org.apache.hadoop.io.NullWritable
import org.apache.hadoop.mapreduce.{RecordWriter, TaskAttemptContext}

import org.apache.spark.SPARK_VERSION_SHORT
import org.apache.spark.sql.SPARK_VERSION_METADATA_KEY
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.OutputWriter
import org.apache.spark.sql.types._
Expand All @@ -45,8 +48,9 @@ private[avro] class AvroOutputWriter(
* Overrides the couple of methods responsible for generating the output streams / files so
* that the data can be correctly partitioned
*/
private val recordWriter: RecordWriter[AvroKey[GenericRecord], NullWritable] =
new AvroKeyOutputFormat[GenericRecord]() {
private val recordWriter: RecordWriter[AvroKey[GenericRecord], NullWritable] = {
val sparkVersion = Map(SPARK_VERSION_METADATA_KEY -> SPARK_VERSION_SHORT).asJava
new SparkAvroKeyOutputFormat(sparkVersion) {

override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = {
new Path(path)
Expand All @@ -57,8 +61,8 @@ private[avro] class AvroOutputWriter(
val path = getDefaultWorkFile(context, ".avro")
path.getFileSystem(context.getConfiguration).create(path)
}

}.getRecordWriter(context)
}

override def write(row: InternalRow): Unit = {
val key = new AvroKey(serializer.serialize(row).asInstanceOf[GenericRecord])
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ import org.apache.avro.generic.{GenericData, GenericDatumReader, GenericDatumWri
import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed}
import org.apache.commons.io.FileUtils

import org.apache.spark.SparkException
import org.apache.spark.{SPARK_VERSION_SHORT, SparkException}
import org.apache.spark.sql._
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.internal.SQLConf
Expand Down Expand Up @@ -1357,4 +1357,16 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
|}
""".stripMargin)
}

test("SPARK-31327: Write Spark version into Avro file metadata") {
withTempPath { path =>
spark.range(1).repartition(1).write.format("avro").save(path.getCanonicalPath)
val avroFiles = path.listFiles()
.filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_"))
assert(avroFiles.length === 1)
val reader = DataFileReader.openReader(avroFiles(0), new GenericDatumReader[GenericRecord]())
val version = reader.asInstanceOf[DataFileReader[_]].getMetaString(SPARK_VERSION_METADATA_KEY)
assert(version === SPARK_VERSION_SHORT)
}
}
}
1 change: 1 addition & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/package.scala
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ package object sql {
* Metadata key which is used to write Spark version in the followings:
* - Parquet file metadata
* - ORC file metadata
* - Avro file metadata
*
* Note that Hive table property `spark.sql.create.version` also has Spark version.
*/
Expand Down