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

Explicit reset position on SeekableInput #1747

Merged
merged 4 commits into from Mar 12, 2019
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
40 changes: 40 additions & 0 deletions scio-avro/src/it/scala/com/spotify/scio/avro/types/AvroTapIT.scala
@@ -0,0 +1,40 @@
/*
* Copyright 2019 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.avro.types

import com.spotify.scio.avro.AvroTaps
import com.spotify.scio.io.Taps
import org.apache.avro.generic.GenericRecord
import org.apache.beam.sdk.io.FileSystems
import org.apache.beam.sdk.options.PipelineOptionsFactory
import org.scalatest.{FlatSpec, Matchers}

import scala.concurrent.Await
import scala.concurrent.duration.Duration

final class AvroTapIT extends FlatSpec with Matchers {

it should "read avro file" in {
FileSystems.setDefaultPipelineOptions(PipelineOptionsFactory.create)

val asd = AvroTaps(Taps()).avroFile[GenericRecord](
"gs://data-integration-test-eu/avro-integration-test/folder-a/folder-b/shakespeare.avro")
val result = Await.result(asd, Duration.Inf)
result.value.hasNext should be true
}

}
10 changes: 7 additions & 3 deletions scio-core/src/main/scala/com/spotify/scio/io/FileStorage.scala
Expand Up @@ -52,9 +52,13 @@ private[scio] final class FileStorage(protected[scio] val path: String) {
private def getAvroSeekableInput(meta: Metadata): SeekableInput =
new SeekableInput {
require(meta.isReadSeekEfficient)
private val in =
FileSystems.open(meta.resourceId()).asInstanceOf[SeekableByteChannel]
in.read(ByteBuffer.allocate(1)) // read a single byte to initialize the channel
private val in = {
val channel = FileSystems.open(meta.resourceId()).asInstanceOf[SeekableByteChannel]
// metadata is lazy loaded on GCS FS and only triggered upon first read
channel.read(ByteBuffer.allocate(1))
// reset position
channel.position(0)
}
override def read(b: Array[Byte], off: Int, len: Int): Int =
in.read(ByteBuffer.wrap(b, off, len))
override def tell(): Long = in.position()
Expand Down