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

add IoCommands #72

Merged
merged 2 commits into from
Apr 13, 2016
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
5 changes: 4 additions & 1 deletion build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ val breezeVersion ="0.12"
val chillVersion = "0.8.0"
val commonsIoVersion = "2.4"
val commonsMath3Version = "3.6"
val csvVersion = "0.1.8"
val guavaVersion = "19.0"
val hadoopVersion = "2.7.2"
val hamcrestVersion = "1.3"
Expand Down Expand Up @@ -307,6 +308,7 @@ lazy val scioRepl: Project = Project(
"jline" % "jline" % scalaBinaryVersion.value,
"org.scala-lang" % "scala-compiler" % scalaVersion.value,
"org.scala-lang" % "scala-reflect" % scalaVersion.value,
"com.nrinaudo" %% "kantan.csv" % csvVersion,
paradiseDependency
),
libraryDependencies ++= (
Expand All @@ -319,7 +321,8 @@ lazy val scioRepl: Project = Project(
).settings(
assemblyJarName in assembly := s"scio-repl-${version.value}.jar"
).dependsOn(
scioCore
scioCore,
scioHdfs
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm kinda on the edge on this one - let's merge but - I think we will remove it next time we have an issue with repl and all the HDFS deps.

)

// =======================================================================
Expand Down
151 changes: 151 additions & 0 deletions scio-repl/src/main/scala/com/spotify/scio/repl/IoCommands.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,151 @@
/*
* Copyright 2016 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.repl

import java.io._
import java.nio.channels.Channels

import com.google.cloud.dataflow.sdk.options.PipelineOptions
import com.google.cloud.dataflow.sdk.util.GcsUtil
import com.google.cloud.dataflow.sdk.util.GcsUtil.GcsUtilFactory
import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath
import com.spotify.scio.util.ScioUtil
import kantan.csv.{RowDecoder, RowEncoder}
import org.apache.avro.file.{DataFileStream, DataFileWriter}
import org.apache.avro.generic.{GenericDatumReader, GenericDatumWriter, GenericRecord}
import org.apache.avro.specific.{SpecificDatumReader, SpecificDatumWriter, SpecificRecordBase}
import org.apache.commons.io.{Charsets, IOUtils}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}

import scala.collection.JavaConverters._
import scala.reflect.ClassTag

/** Commands for simple file I/O in the REPL. */
// scalastyle:off regex
class IoCommands(options: PipelineOptions) {

private val TEXT = "text/plain"
private val BINARY = "application/octet-stream"

private val gcsUtil: GcsUtil = new GcsUtilFactory().create(options)
private val fs = FileSystem.get(new Configuration())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Something to keep in mind - probably not a problem here - but if someone was to use this functions in a long running service, configuration (on disk/resources) may change - thus you may need to refresh configuration. Just something to remember.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not a problem here since it's in scio-repl, not intended for a pipeline job.


// =======================================================================
// Read operations
// =======================================================================

/** Read from an Avro file on local filesystem, GCS, or HDFS. */
def readAvro[T : ClassTag](path: String): Iterator[T] = {
val cls = ScioUtil.classOf[T]
val reader = if (classOf[SpecificRecordBase] isAssignableFrom cls) {
new SpecificDatumReader[T]()
} else {
new GenericDatumReader[T]()
}
new DataFileStream[T](inputStream(path), reader).iterator().asScala
}

/** Read from a text file on local filesystem, GCS, or HDFS. */
def readText(path: String): Iterator[String] =
IOUtils.lineIterator(inputStream(path), Charsets.UTF_8).asScala

/** Read from a CSV file on local filesystem, GCS, or HDFS. */
def readCsv[T: RowDecoder](path: String,
sep: Char = ',',
header: Boolean = false): Iterator[T] = {
import kantan.csv.ops._
implicit val codec = scala.io.Codec.UTF8
inputStream(path).asUnsafeCsvReader[T](sep, header).toIterator
}

/** Read from a TSV file on local filesystem, GCS, or HDFS. */
def readTsv[T: RowDecoder](path: String,
sep: Char = '\t',
header: Boolean = false): Iterator[T] = {
import kantan.csv.ops._
implicit val codec = scala.io.Codec.UTF8
inputStream(path).asUnsafeCsvReader[T](sep, header).toIterator
}

// =======================================================================
// Write operations
// =======================================================================

private def plural[T](data: Seq[T]): String = if (data.size > 1) "s" else ""

/** Write to an Avro file on local filesystem, GCS, or HDFS. */
def writeAvro[T: ClassTag](path: String, data: Seq[T]): Unit = {
val cls = ScioUtil.classOf[T]
val (writer, schema) = if (classOf[SpecificRecordBase] isAssignableFrom cls) {
(new SpecificDatumWriter[T](cls), data.head.asInstanceOf[SpecificRecordBase].getSchema)
} else {
(new GenericDatumWriter[T](), data.head.asInstanceOf[GenericRecord].getSchema)
}
val fileWriter = new DataFileWriter[T](writer).create(schema, outputStream(path, BINARY))
data.foreach(fileWriter.append)
fileWriter.close()
println(s"${data.size} record${plural(data)} written to $path")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Log?

}

/** Write to a text file on local filesystem, GCS, or HDFS. */
def writeText(path: String, data: Seq[String]): Unit = {
IOUtils.writeLines(data.asJava, IOUtils.LINE_SEPARATOR, outputStream(path, TEXT))
println(s"${data.size} line${plural(data)} written to $path")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this be a log?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Log will also print out a bunch of junk we don't need.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But log you can control, can't see a reason for println.

}

/** Write to a CSV file on local filesystem, GCS, or HDFS. */
def writeCsv[T: RowEncoder](path: String, data: Seq[T],
sep: Char = ',',
header: Seq[String] = Seq.empty): Unit = {
import kantan.csv.ops._
IOUtils.write(data.asCsv(sep, header), outputStream(path, TEXT))
}

/** Write to a TSV file on local filesystem, GCS, or HDFS. */
def writeTsv[T: RowEncoder](path: String, data: Seq[T],
sep: Char = '\t',
header: Seq[String] = Seq.empty): Unit = {
import kantan.csv.ops._
IOUtils.write(data.asCsv(sep, header), outputStream(path, TEXT))
}

// =======================================================================
// Utilities
// =======================================================================

private def inputStream(path: String): InputStream =
if (path.startsWith("hdfs://")) {
fs.open(new Path(path))
} else if (path.startsWith("gs://")) {
Channels.newInputStream(gcsUtil.open(GcsPath.fromUri(path)))
} else {
new FileInputStream(path)
}

private def outputStream(path: String, contentType: String): OutputStream =
if (path.startsWith("hdfs://")) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have seen a couple of these - here and there - should we have a function that checks if paths is hdfs or gs?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Any reason not to support all hadoop supported schemas via https://hadoop.apache.org/docs/r2.7.2/api/org/apache/hadoop/fs/FileSystem.html#get(java.net.URI, org.apache.hadoop.conf.Configuration) ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah I should do that but I'm putting this PR on hold since adding hdfs dep may cause other problems.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@sisidra the only problem with that is that then you need to depend on hadoop deps - and that is something we mostly want to stay away from (in like scio-core). But again if a job needs HDFS (or repl in this case would have Hadoop), then we already have hadoop, but anyway, what @nevillelyh said. Generic function for all supported schemas could go to scio-hdfs tho.

fs.create(new Path(path), false)
} else if (path.startsWith("gs://")) {
Channels.newOutputStream(gcsUtil.create(GcsPath.fromUri(path), contentType))
} else {
new FileOutputStream(path)
}

}
// scalastyle:on regex
Original file line number Diff line number Diff line change
Expand Up @@ -227,13 +227,22 @@ class ScioILoop(scioClassLoader: ScioReplClassLoader,
}
}

private def loadIoCommands(): IR.Result = {
intp.interpret(
"""
|val _ioCommands = new com.spotify.scio.repl.IoCommands(sc.options)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So if I would like to use IOCommands with different opts - i have to manually create it myself right? And by default you get the IOCommands from your startup opts?

|import _ioCommands._
""".stripMargin)
}

override def createInterpreter(): Unit = {
super.createInterpreter()
welcome()
intp.beQuietDuring {
addImports()
createBigQueryClient()
newScioCmdImpl("sc")
loadIoCommands()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder if we should think about doing some of this stuff async to save on startup time? Probably doesn't make sense here (yet) - but I think spark and scala repls is doing some stuff in async fashion to help with startup time.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldn't be a huge difference? Let's not over optimize for now?

}
}

Expand Down