You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@crunch.apache.org by jw...@apache.org on 2014/05/20 06:07:39 UTC

git commit: CRUNCH-396: Rich IO classes for Scrunch, like Java

Repository: crunch
Updated Branches:
  refs/heads/master 00bc96991 -> bbeb7537a


CRUNCH-396: Rich IO classes for Scrunch, like Java


Project: http://git-wip-us.apache.org/repos/asf/crunch/repo
Commit: http://git-wip-us.apache.org/repos/asf/crunch/commit/bbeb7537
Tree: http://git-wip-us.apache.org/repos/asf/crunch/tree/bbeb7537
Diff: http://git-wip-us.apache.org/repos/asf/crunch/diff/bbeb7537

Branch: refs/heads/master
Commit: bbeb7537a581939c6c22dfdcffe8625b9336589a
Parents: 00bc969
Author: Josh Wills <jw...@apache.org>
Authored: Thu May 15 23:08:18 2014 -0700
Committer: Josh Wills <jw...@apache.org>
Committed: Mon May 19 20:57:10 2014 -0700

----------------------------------------------------------------------
 .../scala/org/apache/crunch/scrunch/IO.scala    | 289 ++++++++++++++++---
 1 file changed, 251 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/crunch/blob/bbeb7537/crunch-scrunch/src/main/scala/org/apache/crunch/scrunch/IO.scala
----------------------------------------------------------------------
diff --git a/crunch-scrunch/src/main/scala/org/apache/crunch/scrunch/IO.scala b/crunch-scrunch/src/main/scala/org/apache/crunch/scrunch/IO.scala
index 99b10e9..c4af454 100644
--- a/crunch-scrunch/src/main/scala/org/apache/crunch/scrunch/IO.scala
+++ b/crunch-scrunch/src/main/scala/org/apache/crunch/scrunch/IO.scala
@@ -1,55 +1,268 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you 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.
  *
- *     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 org.apache.crunch.scrunch
 
-import org.apache.crunch.io.{From => from, To => to, At => at}
-import org.apache.crunch.types.avro.AvroType
-import org.apache.hadoop.conf.Configuration
+import org.apache.crunch.io.{From => from, To => to, At => at, SequentialFileNamingScheme}
 import org.apache.hadoop.fs.Path
+import org.apache.hadoop.conf.Configuration
+import org.apache.crunch.types.PType
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
+import org.apache.crunch.types.avro.AvroType
+import org.apache.hadoop.io.Writable
+import org.apache.avro.specific.SpecificRecord
+import org.apache.crunch.Target
+import org.apache.crunch.io.impl.FileTargetImpl
+import org.apache.crunch.io.avro.AvroFileTarget
+import org.apache.crunch.io.seq.SeqFileTarget
+import org.apache.crunch.io.text.TextFileTarget
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
+
+object From {
+  import scala.collection.JavaConversions._
+
+  def formattedFile[K <: Writable, V <: Writable](pathName: String, formatClass: Class[_ <: FileInputFormat[K, V]],
+                          keyClass: Class[K], valueClass: Class[V]) = {
+    from.formattedFile(pathName, formatClass, keyClass, valueClass)
+  }
+
+  def formattedFile[K <: Writable, V <: Writable](path: Path, formatClass: Class[_ <: FileInputFormat[K, V]],
+                          keyClass: Class[K], valueClass: Class[V]) = {
+    from.formattedFile(path, formatClass, keyClass, valueClass)
+  }
+
+  def formattedFile[K <: Writable, V <: Writable](paths: List[Path], formatClass: Class[_ <: FileInputFormat[K, V]],
+                          keyClass: Class[K], valueClass: Class[V]) = {
+    from.formattedFile(paths, formatClass, keyClass, valueClass)
+  }
+
+  def formattedFile[K, V](pathName: String, formatClass: Class[_ <: FileInputFormat[_, _]],
+                          keyType: PType[K], valueType: PType[V]) = {
+    from.formattedFile(pathName, formatClass, keyType, valueType)
+  }
+
+  def formattedFile[K, V](path: Path, formatClass: Class[_ <: FileInputFormat[_, _]],
+                          keyType: PType[K], valueType: PType[V]) = {
+    from.formattedFile(path, formatClass, keyType, valueType)
+  }
+
+  def formattedFile[K, V](paths: List[Path], formatClass: Class[_ <: FileInputFormat[_, _]],
+                          keyType: PType[K], valueType: PType[V]) = {
+    from.formattedFile(paths, formatClass, keyType, valueType)
+  }
+
+  def avroFile[T <: SpecificRecord](pathName: String, avroClass: Class[T]) = {
+    from.avroFile(pathName, avroClass)
+  }
+
+  def avroFile[T <: SpecificRecord](path: Path, avroClass: Class[T]) = {
+    from.avroFile(path, avroClass)
+  }
+
+  def avroFile[T <: SpecificRecord](paths: List[Path], avroClass: Class[T]) = {
+    from.avroFile(paths, avroClass)
+  }
+
+  def avroFile[T](pathName: String, avroType: AvroType[T]) = {
+    from.avroFile(pathName, avroType)
+  }
+
+  def avroFile[T](path: Path, avroType: AvroType[T]) = {
+    from.avroFile(path, avroType)
+  }
+
+  def avroFile[T](paths: List[Path], avroType: AvroType[T]) = {
+    from.avroFile(paths, avroType)
+  }
+
+  def avroFile(pathName: String) = {
+    from.avroFile(pathName)
+  }
+
+  def avroFile(path: Path) = {
+    from.avroFile(path)
+  }
+
+  def avroFile(paths: List[Path]) = {
+    from.avroFile(paths)
+  }
+
+  def avroFile(path: Path, conf: Configuration) = {
+    from.avroFile(path, conf)
+  }
+
+  def avroFile(paths: List[Path], conf: Configuration) = {
+    from.avroFile(paths, conf)
+  }
+
+  def sequenceFile[T <: Writable](pathName: String, valueClass: Class[T]) = {
+    from.sequenceFile(pathName, valueClass)
+  }
+
+  def sequenceFile[T <: Writable](path: Path, valueClass: Class[T]) = {
+    from.sequenceFile(path, valueClass)
+  }
+
+  def sequenceFile[T <: Writable](paths: List[Path], valueClass: Class[T]) = {
+    from.sequenceFile(paths, valueClass)
+  }
+
+  def sequenceFile[T](pathName: String, ptype: PType[T]) = {
+    from.sequenceFile(pathName, ptype)
+  }
+
+  def sequenceFile[T](path: Path, ptype: PType[T]) = {
+    from.sequenceFile(path, ptype)
+  }
+
+  def sequenceFile[T](paths: List[Path], ptype: PType[T]) = {
+    from.sequenceFile(paths, ptype)
+  }
+
+  def sequenceFile[K <: Writable, V <: Writable](pathName: String, keyClass: Class[K], valueClass: Class[V]) = {
+    from.sequenceFile(pathName, keyClass, valueClass)
+  }
+
+  def sequenceFile[K <: Writable, V <: Writable](path: Path, keyClass: Class[K], valueClass: Class[V]) = {
+    from.sequenceFile(path, keyClass, valueClass)
+  }
+
+  def sequenceFile[K <: Writable, V <: Writable](paths: List[Path], keyClass: Class[K], valueClass: Class[V]) = {
+    from.sequenceFile(paths, keyClass, valueClass)
+  }
+
+  def sequenceFile[K, V](pathName: String, keyType: PType[K], valueType: PType[V]) = {
+    from.sequenceFile(pathName, keyType, valueType)
+  }
+
+  def sequenceFile[K, V](path: Path, keyType: PType[K], valueType: PType[V]) = {
+    from.sequenceFile(path, keyType, valueType)
+  }
+
+  def sequenceFile[K, V](paths: List[Path], keyType: PType[K], valueType: PType[V]) = {
+    from.sequenceFile(paths, keyType, valueType)
+  }
+
+  def textFile(pathName: String) = from.textFile(pathName)
 
-trait From {
-  def avroFile[T](path: String, atype: AvroType[T]) = from.avroFile(path, atype)
-  def avroFile[T](path: Path, atype: AvroType[T]) = from.avroFile(path, atype)
-  def avroFile(path: Path) = from.avroFile(path)
-  def avroFile(path: Path, conf: Configuration) = from.avroFile(path, conf)
-  def textFile(path: String) = from.textFile(path)
   def textFile(path: Path) = from.textFile(path)
+
+  def textFile(paths: List[Path]) = from.textFile(paths)
+
+  def textFile[T](pathName: String, ptype: PType[T]) = from.textFile(pathName, ptype)
+
+  def textFile[T](path: Path, ptype: PType[T]) = from.textFile(path, ptype)
+
+  def textFile[T](paths: List[Path], ptype: PType[T]) = from.textFile(paths, ptype)
 }
 
-object From extends From
+object To {
+  def formattedFile[K <: Writable, V <: Writable](pathName: String, formatClass: Class[_ <: FileOutputFormat[K, V]]) = {
+    to.formattedFile(pathName, formatClass)
+  }
+
+  def formattedFile[K <: Writable, V <: Writable](path: Path, formatClass: Class[_ <: FileOutputFormat[K, V]]) = {
+    to.formattedFile(path, formatClass)
+  }
+
+  def avroFile(pathName: String) = to.avroFile(pathName)
+
+  def avroFile(path: Path) = to.avroFile(path)
+
+  def sequenceFile(pathName: String) = to.sequenceFile(pathName)
+
+  def sequenceFile(path: Path) = to.sequenceFile(path)
+
+  def textFile(pathName: String) = to.textFile(pathName)
 
-trait To {
-  def avroFile[T](path: String) = to.avroFile(path)
-  def avroFile[T](path: Path) = to.avroFile(path)
-  def textFile(path: String) = to.textFile(path)
   def textFile(path: Path) = to.textFile(path)
 }
 
-object To extends To
+object At {
+  def avroFile[T <: SpecificRecord](pathName: String, avroClass: Class[T]) = {
+    at.avroFile(pathName, avroClass)
+  }
+
+  def avroFile[T <: SpecificRecord](path: Path, avroClass: Class[T]) = {
+    at.avroFile(path, avroClass)
+  }
+
+  def avroFile[T](pathName: String, avroType: AvroType[T]) = {
+    at.avroFile(pathName, avroType)
+  }
+
+  def avroFile[T](path: Path, avroType: AvroType[T]) = {
+    at.avroFile(path, avroType)
+  }
+
+  def avroFile(pathName: String) = {
+    at.avroFile(pathName)
+  }
+
+  def avroFile(path: Path) = {
+    at.avroFile(path)
+  }
+
+  def avroFile(path: Path, conf: Configuration) = {
+    at.avroFile(path, conf)
+  }
+
+  def sequenceFile[T <: Writable](pathName: String, valueClass: Class[T]) = {
+    at.sequenceFile(pathName, valueClass)
+  }
+
+  def sequenceFile[T <: Writable](path: Path, valueClass: Class[T]) = {
+    at.sequenceFile(path, valueClass)
+  }
+
+  def sequenceFile[T](pathName: String, ptype: PType[T]) = {
+    at.sequenceFile(pathName, ptype)
+  }
+
+  def sequenceFile[T](path: Path, ptype: PType[T]) = {
+    at.sequenceFile(path, ptype)
+  }
+
+  def sequenceFile[K <: Writable, V <: Writable](pathName: String, keyClass: Class[K], valueClass: Class[V]) = {
+    at.sequenceFile(pathName, keyClass, valueClass)
+  }
+
+  def sequenceFile[K <: Writable, V <: Writable](path: Path, keyClass: Class[K], valueClass: Class[V]) = {
+    at.sequenceFile(path, keyClass, valueClass)
+  }
+
+  def sequenceFile[K, V](pathName: String, keyType: PType[K], valueType: PType[V]) = {
+    at.sequenceFile(pathName, keyType, valueType)
+  }
+
+  def sequenceFile[K, V](path: Path, keyType: PType[K], valueType: PType[V]) = {
+    at.sequenceFile(path, keyType, valueType)
+  }
+
+  def textFile(pathName: String) = at.textFile(pathName)
 
-trait At {
-  def avroFile[T](path: String, atype: AvroType[T]) = at.avroFile(path, atype)
-  def avroFile[T](path: Path, atype: AvroType[T]) = at.avroFile(path, atype)
-  def avroFile(path: Path) = at.avroFile(path)
-  def avroFile(path: Path, conf: Configuration) = at.avroFile(path, conf)
-  def textFile(path: String) = at.textFile(path)
   def textFile(path: Path) = at.textFile(path)
-}
 
-object At extends At
+  def textFile[T](pathName: String, ptype: PType[T]) = at.textFile(pathName, ptype)
+
+  def textFile[T](path: Path, ptype: PType[T]) = at.textFile(path, ptype)
+
+}