Skip to content
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

Fix avro direct encoding #69

Merged
merged 4 commits into from
Jul 11, 2019
Merged
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
Expand Up @@ -201,7 +201,7 @@ public void write(String query) throws Exception {
dataFileWriter.appendEncoded(converter.convertResultSetIntoAvroBytes());
this.metering.incrementRecordCount();
}
this.dataFileWriter.sync();
this.dataFileWriter.flush();
this.metering.exposeWriteElapsedMs(System.currentTimeMillis() - startMs);
this.metering.exposeWrittenBytes(this.countingOutputStream.getCount());
}
Expand All @@ -214,7 +214,7 @@ protected void finishWrite() throws Exception {
connection.close();
}
if (dataFileWriter != null) {
dataFileWriter.flush();
dataFileWriter.close();
}
logger.info("jdbcavroio : Write finished");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,7 @@ public ByteBuffer convertResultSetIntoAvroBytes()
}
}
binaryEncoder.flush();
return ByteBuffer.wrap(out.getBufffer());
return ByteBuffer.wrap(out.getBufffer(), 0, out.size());
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ class JdbcAvroRecordTest extends FlatSpec with Matchers with BeforeAndAfterAll {
private val record1 = JdbcTestFixtures.record1

override def beforeAll(): Unit = {
JdbcTestFixtures.createFixtures(db, Seq(JdbcTestFixtures.record1))
JdbcTestFixtures.createFixtures(db, Seq(JdbcTestFixtures.record1, JdbcTestFixtures.record2))
}

it should "create schema" in {
Expand Down Expand Up @@ -139,20 +139,24 @@ class JdbcAvroRecordTest extends FlatSpec with Matchers with BeforeAndAfterAll {
it should "encode jdbc result set to valid avro" in {
val rs = db.source.createConnection().createStatement().executeQuery(s"SELECT * FROM COFFEES")
val schema = JdbcAvroSchema.createAvroSchema(rs, "dbeam_generated","connection", "doc", false)
rs.next()

val converter = JdbcAvroRecordConverter.create(rs)
val dataFileWriter = new DataFileWriter(new GenericDatumWriter[GenericRecord](schema))
val outputStream = new ByteArrayOutputStream()
dataFileWriter.create(schema, outputStream)
// convert and write
dataFileWriter.appendEncoded(converter.convertResultSetIntoAvroBytes())
while (rs.next()) {
dataFileWriter.appendEncoded(converter.convertResultSetIntoAvroBytes())
}
dataFileWriter.flush()
outputStream.close()
// transform to generic record
val inputStream = new SeekableByteArrayInput(outputStream.toByteArray)
val dataFileReader = new DataFileReader[GenericRecord](inputStream,
new GenericDatumReader[GenericRecord](schema))
val record: GenericRecord = dataFileReader.iterator().next()
val records: Seq[GenericRecord] =
dataFileReader.asInstanceOf[java.lang.Iterable[GenericRecord]].asScala.toSeq
records.size should be (2)
val record: GenericRecord = records.filter(r => r.get(0).toString == record1._1).head

record shouldNot be (null)
record.getSchema should be (schema)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,19 +17,20 @@

package com.spotify.dbeam.jobs

import com.spotify.dbeam.JdbcTestFixtures
import com.spotify.dbeam.avro.JdbcAvroMetering
import com.spotify.dbeam.options.OutputOptions

import java.io.File
import java.nio.file.{Files, Path}
import java.util
import java.util.stream.{Collectors, StreamSupport}
import java.util.{Comparator, UUID}

import com.spotify.dbeam.JdbcTestFixtures
import com.spotify.dbeam.avro.JdbcAvroMetering
import com.spotify.dbeam.options.OutputOptions
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.beam.sdk.io.AvroSource
import org.apache.avro.file.DataFileReader
import org.apache.avro.generic.{GenericDatumReader, GenericRecord}
import org.apache.beam.sdk.options.PipelineOptionsFactory
import org.apache.beam.sdk.testing.SourceTestUtils
import org.junit.runner.RunWith
import org.scalatest._
import org.scalatest.junit.JUnitRunner
Expand Down Expand Up @@ -74,22 +75,30 @@ class JdbcAvroJobTest extends FlatSpec with Matchers with BeforeAndAfterAll {
"--username=",
"--passwordFile=" + passwordFile.getAbsolutePath,
"--table=COFFEES",
"--output=" + dir.getAbsolutePath)
)
"--output=" + dir.getAbsolutePath,
"--avroCodec=deflate1"
))
val files: Array[File] = dir.listFiles()
files.map(_.getName) should contain theSameElementsAs Seq(
"_AVRO_SCHEMA.avsc", "_METRICS.json", "_SERVICE_METRICS.json",
"_queries", "part-00000-of-00001.avro")
files.filter(_.getName.equals("_queries"))(0).listFiles().map(_.getName) should
contain theSameElementsAs Seq("query_0.sql")
val schema = new Schema.Parser().parse(new File(dir, "_AVRO_SCHEMA.avsc"))
val source: AvroSource[GenericRecord] = AvroSource
.from(new File(dir, "part-00000-of-00001.avro").toString)
.withSchema(schema)
val records: util.List[GenericRecord] = SourceTestUtils.readFromSource(source, null)
val records: util.List[GenericRecord] =
readAvroRecords(new File(dir, "part-00000-of-00001.avro"), schema)
records should have size 2
}

private def readAvroRecords(avroFile: File, schema: Schema): util.List[GenericRecord] = {
val datum: GenericDatumReader[GenericRecord] = new GenericDatumReader(schema)
val dataFileReader = new DataFileReader(avroFile, datum)
val records: util.List[GenericRecord] = StreamSupport.stream(dataFileReader.spliterator(), false)
.collect(Collectors.toList())
dataFileReader.close()
records
}

"JdbcAvroJob" should "have a default exit code" in {
ExceptionHandling.exitCode(new IllegalStateException()) should be (49)
}
Expand Down