-
Notifications
You must be signed in to change notification settings - Fork 513
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
(#5296) Support Parquet predicates/projections in tests #5309
Merged
Merged
Changes from all commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
f7ede1c
Populate scio-test-parquet
clairemcginty 4214363
TfExample implementation
clairemcginty 4a00498
Refactor test helper method naming
clairemcginty 00fe0ff
Support Avro-->case class projection
clairemcginty 78dd162
Declare dependencies
clairemcginty c8c3140
Separate packages
clairemcginty ee1a204
case class -> class
clairemcginty 288682d
Split packages
clairemcginty 7bb5788
update site
clairemcginty b2b3f95
consistent style
clairemcginty File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
100 changes: 100 additions & 0 deletions
100
scio-test/parquet/src/main/scala/com/spotify/scio/testing/parquet/ParquetTestUtils.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,100 @@ | ||
/* | ||
* Copyright 2024 Spotify AB. | ||
* | ||
* Licensed 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 com.spotify.scio.testing.parquet | ||
|
||
import org.apache.parquet.hadoop.{ParquetReader, ParquetWriter} | ||
import org.apache.parquet.io._ | ||
|
||
import java.io.{ByteArrayInputStream, ByteArrayOutputStream} | ||
|
||
private[parquet] object ParquetTestUtils { | ||
|
||
def roundtrip[T, U]( | ||
writerFn: OutputFile => ParquetWriter[T], | ||
readerFn: InputFile => ParquetReader[U] | ||
)( | ||
records: Iterable[T] | ||
): Iterable[U] = { | ||
val baos = new ByteArrayOutputStream() | ||
val writer = writerFn(new InMemoryOutputFile(baos)) | ||
|
||
records.foreach(writer.write) | ||
writer.close() | ||
|
||
val reader = readerFn(new InMemoryInputFile(baos.toByteArray)) | ||
val roundtripped = Iterator.continually(reader.read()).takeWhile(_ != null).toSeq | ||
reader.close() | ||
roundtripped | ||
} | ||
|
||
private class InMemoryOutputFile(baos: ByteArrayOutputStream) extends OutputFile { | ||
override def create(blockSizeHint: Long): PositionOutputStream = newPositionOutputStream() | ||
|
||
override def createOrOverwrite(blockSizeHint: Long): PositionOutputStream = | ||
newPositionOutputStream() | ||
|
||
override def supportsBlockSize(): Boolean = false | ||
|
||
override def defaultBlockSize(): Long = 0L | ||
|
||
private def newPositionOutputStream(): PositionOutputStream = new PositionOutputStream { | ||
var pos: Long = 0 | ||
|
||
override def getPos: Long = pos | ||
|
||
override def write(b: Int): Unit = { | ||
pos += 1 | ||
baos.write(b) | ||
} | ||
|
||
override def write(b: Array[Byte], off: Int, len: Int): Unit = { | ||
baos.write(b, off, len) | ||
pos += len | ||
} | ||
|
||
override def write(b: Array[Byte]): Unit = write(b, 0, b.length) | ||
|
||
override def flush(): Unit = baos.flush() | ||
|
||
override def close(): Unit = baos.close() | ||
} | ||
} | ||
|
||
private class InMemoryInputFile(bytes: Array[Byte]) extends InputFile { | ||
override def getLength: Long = bytes.length | ||
|
||
override def newStream(): SeekableInputStream = | ||
new DelegatingSeekableInputStream(new ByteArrayInputStream(bytes)) { | ||
override def getPos: Long = bytes.length - getStream.available() | ||
|
||
override def mark(readlimit: Int): Unit = { | ||
if (readlimit != 0) { | ||
throw new UnsupportedOperationException( | ||
"In-memory seekable input stream is intended for testing only, can't mark past 0" | ||
) | ||
} | ||
super.mark(readlimit) | ||
} | ||
|
||
override def seek(newPos: Long): Unit = { | ||
getStream.reset() | ||
getStream.skip(newPos) | ||
} | ||
} | ||
} | ||
} |
81 changes: 81 additions & 0 deletions
81
scio-test/parquet/src/main/scala/com/spotify/scio/testing/parquet/avro/package.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,81 @@ | ||
/* | ||
* Copyright 2024 Spotify AB. | ||
* | ||
* Licensed 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 com.spotify.scio.testing.parquet | ||
|
||
import com.spotify.scio.testing.parquet.ParquetTestUtils._ | ||
import magnolify.parquet.ParquetType | ||
import org.apache.avro.Schema | ||
import org.apache.avro.generic.GenericRecord | ||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.parquet.avro.{AvroParquetReader, AvroParquetWriter, AvroReadSupport} | ||
import org.apache.parquet.filter2.predicate.FilterPredicate | ||
import org.apache.parquet.hadoop.{ParquetInputFormat, ParquetReader} | ||
import org.apache.parquet.io.InputFile | ||
|
||
package object avro { | ||
implicit def toParquetAvroHelpers[U <: GenericRecord]( | ||
records: Iterable[U] | ||
): ParquetAvroHelpers[U] = new ParquetAvroHelpers(records) | ||
|
||
class ParquetAvroHelpers[U <: GenericRecord] private[testing] ( | ||
records: Iterable[U] | ||
) { | ||
def withProjection(projection: Schema): Iterable[U] = { | ||
val configuration = new Configuration() | ||
AvroReadSupport.setRequestedProjection(configuration, projection) | ||
|
||
roundtripAvro( | ||
records, | ||
inputFile => AvroParquetReader.builder[U](inputFile).withConf(configuration).build() | ||
) | ||
} | ||
|
||
def withProjection[V: ParquetType]: Iterable[V] = { | ||
val pt = implicitly[ParquetType[V]] | ||
|
||
roundtripAvro(records, inputFile => pt.readBuilder(inputFile).build()) | ||
} | ||
|
||
def withFilter(filter: FilterPredicate): Iterable[U] = { | ||
val configuration = new Configuration() | ||
ParquetInputFormat.setFilterPredicate(configuration, filter) | ||
|
||
roundtripAvro( | ||
records, | ||
inputFile => AvroParquetReader.builder[U](inputFile).withConf(configuration).build() | ||
) | ||
} | ||
|
||
private def roundtripAvro[V]( | ||
records: Iterable[U], | ||
readerFn: InputFile => ParquetReader[V] | ||
): Iterable[V] = { | ||
records.headOption match { | ||
case None => | ||
Iterable.empty[V] // empty iterable | ||
case Some(head) => | ||
val schema = head.getSchema | ||
|
||
roundtrip( | ||
outputFile => AvroParquetWriter.builder[U](outputFile).withSchema(schema).build(), | ||
readerFn | ||
)(records) | ||
} | ||
} | ||
} | ||
} |
74 changes: 74 additions & 0 deletions
74
scio-test/parquet/src/main/scala/com/spotify/scio/testing/parquet/tensorflow/package.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,74 @@ | ||
/* | ||
* Copyright 2024 Spotify AB. | ||
* | ||
* Licensed 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 com.spotify.scio.testing.parquet | ||
|
||
import com.spotify.parquet.tensorflow.{ | ||
TensorflowExampleParquetReader, | ||
TensorflowExampleParquetWriter, | ||
TensorflowExampleReadSupport | ||
} | ||
import com.spotify.scio.testing.parquet.ParquetTestUtils._ | ||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.parquet.filter2.predicate.FilterPredicate | ||
import org.apache.parquet.hadoop.ParquetInputFormat | ||
import org.tensorflow.metadata.{v0 => tfmd} | ||
import org.tensorflow.proto.example.Example | ||
|
||
package object tensorflow { | ||
implicit def toParquetExampleHelpers( | ||
records: Iterable[Example] | ||
): ParquetExampleHelpers = new ParquetExampleHelpers(records) | ||
|
||
class ParquetExampleHelpers private[testing] (records: Iterable[Example]) { | ||
def withProjection(schema: tfmd.Schema, projection: tfmd.Schema): Iterable[Example] = { | ||
val configuration = new Configuration() | ||
TensorflowExampleReadSupport.setExampleReadSchema( | ||
configuration, | ||
projection | ||
) | ||
TensorflowExampleReadSupport.setRequestedProjection( | ||
configuration, | ||
projection | ||
) | ||
|
||
roundtripExample(records, schema, configuration) | ||
} | ||
|
||
def withFilter(schema: tfmd.Schema, filter: FilterPredicate): Iterable[Example] = { | ||
val configuration = new Configuration() | ||
TensorflowExampleReadSupport.setExampleReadSchema( | ||
configuration, | ||
schema | ||
) | ||
ParquetInputFormat.setFilterPredicate(configuration, filter) | ||
|
||
roundtripExample(records, schema, configuration) | ||
} | ||
|
||
private def roundtripExample( | ||
records: Iterable[Example], | ||
schema: tfmd.Schema, | ||
readConfiguration: Configuration | ||
): Iterable[Example] = roundtrip( | ||
outputFile => TensorflowExampleParquetWriter.builder(outputFile).withSchema(schema).build(), | ||
inputFile => { | ||
TensorflowExampleParquetReader.builder(inputFile).withConf(readConfiguration).build() | ||
} | ||
)(records) | ||
} | ||
} |
44 changes: 44 additions & 0 deletions
44
scio-test/parquet/src/main/scala/com/spotify/scio/testing/parquet/types/types.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,44 @@ | ||
/* | ||
* Copyright 2024 Spotify AB. | ||
* | ||
* Licensed 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 com.spotify.scio.testing.parquet | ||
|
||
import com.spotify.scio.testing.parquet.ParquetTestUtils._ | ||
import magnolify.parquet.ParquetType | ||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.parquet.filter2.predicate.FilterPredicate | ||
import org.apache.parquet.hadoop.ParquetInputFormat | ||
|
||
package object types { | ||
implicit def toParquetMagnolifyHelpers[T: ParquetType]( | ||
records: Iterable[T] | ||
): ParquetMagnolifyHelpers[T] = new ParquetMagnolifyHelpers(records) | ||
|
||
class ParquetMagnolifyHelpers[T: ParquetType] private[testing] (records: Iterable[T]) { | ||
def withFilter(filter: FilterPredicate): Iterable[T] = { | ||
val pt = implicitly[ParquetType[T]] | ||
|
||
val configuration = new Configuration() | ||
ParquetInputFormat.setFilterPredicate(configuration, filter) | ||
|
||
roundtrip( | ||
outputFile => pt.writeBuilder(outputFile).build(), | ||
inputFile => pt.readBuilder(inputFile).withConf(configuration).build() | ||
)(records) | ||
} | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: we should avoid using package object for definitions as it is deprecated in scala 3 (ref),