SAMZA-1623: include avro as the file suffix for hdfs producer
authorHai Lu <halu@linkedin.com>
Wed, 21 Mar 2018 20:13:16 +0000 (13:13 -0700)
committerxiliu <xiliu@linkedin.com>
Wed, 21 Mar 2018 20:13:16 +0000 (13:13 -0700)
AvroDataFileHdfsWriter should include avro as the file suffix as some pig jobs couldn't read the avro files if they don't come with the proper suffix

Author: Hai Lu <halu@linkedin.com>

Reviewers: Xinyu Liu <xinyuliu.us@gmail.com>

Closes #452 from lhaiesp/master

samza-hdfs/src/main/scala/org/apache/samza/system/hdfs/writer/AvroDataFileHdfsWriter.scala

index f1868cd..a9823f5 100644 (file)
@@ -66,7 +66,7 @@ class AvroDataFileHdfsWriter (dfs: FileSystem, systemName: String, config: HdfsC
   }
 
   protected def getNextWriter(record: Object): Option[DataFileWriter[Object]] = {
-    val path = bucketer.get.getNextWritePath(dfs)
+    val path = bucketer.get.getNextWritePath(dfs).suffix(".avro")
     val isGenericRecord = record.isInstanceOf[GenericRecord]
     val schema = record match {
       case genericRecord: GenericRecord => genericRecord.getSchema