You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by ap...@apache.org on 2015/04/11 02:08:48 UTC

mahout git commit: (NOJIRA) change package of Hadoop1HDFSUtil and associated classes in h2o to avoid conflict with spark classes

Repository: mahout
Updated Branches:
  refs/heads/master fb30375e4 -> 8439846c9


(NOJIRA) change package of Hadoop1HDFSUtil and associated classes in h2o to avoid conflict with spark classes


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

Branch: refs/heads/master
Commit: 8439846c99c2c81dde6a03eb40105ea908555943
Parents: fb30375
Author: Andrew Palumbo <ap...@apache.org>
Authored: Fri Apr 10 20:07:58 2015 -0400
Committer: Andrew Palumbo <ap...@apache.org>
Committed: Fri Apr 10 20:07:58 2015 -0400

----------------------------------------------------------------------
 .../org/apache/mahout/common/DrmMetadata.scala  | 56 -----------------
 .../org/apache/mahout/common/HDFSUtil.scala     | 26 --------
 .../apache/mahout/common/Hadoop1HDFSUtil.scala  | 63 --------------------
 .../apache/mahout/h2o/common/DrmMetadata.scala  | 56 +++++++++++++++++
 .../org/apache/mahout/h2o/common/HDFSUtil.scala | 26 ++++++++
 .../mahout/h2o/common/Hadoop1HDFSUtil.scala     | 63 ++++++++++++++++++++
 .../apache/mahout/h2obindings/H2OEngine.scala   |  2 +-
 7 files changed, 146 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mahout/blob/8439846c/h2o/src/main/scala/org/apache/mahout/common/DrmMetadata.scala
----------------------------------------------------------------------
diff --git a/h2o/src/main/scala/org/apache/mahout/common/DrmMetadata.scala b/h2o/src/main/scala/org/apache/mahout/common/DrmMetadata.scala
deleted file mode 100644
index 5bbccb1..0000000
--- a/h2o/src/main/scala/org/apache/mahout/common/DrmMetadata.scala
+++ /dev/null
@@ -1,56 +0,0 @@
-package org.apache.mahout.common
-
-import scala.reflect.ClassTag
-import org.apache.hadoop.io._
-import java.util.Arrays
-
-class DrmMetadata(
-
-    /** Writable  key type as a sub-type of Writable */
-    val keyTypeWritable: Class[_],
-
-    /** Value writable type, as a sub-type of Writable */
-    val valueTypeWritable: Class[_]
-
-    ) {
-
-  import DrmMetadata._
-
-  val (
-
-      /** Actual drm key class tag once converted out of writable */
-      keyClassTag: ClassTag[_],
-
-      /** Conversion from Writable to value type of the DRM key */
-      keyW2ValFunc: ((Writable) => Any)
-
-      ) = keyTypeWritable match {
-    case cz if (cz == classOf[IntWritable]) => ClassTag.Int -> w2int _
-    case cz if (cz == classOf[LongWritable]) => ClassTag.Long -> w2long _
-    case cz if (cz == classOf[DoubleWritable]) => ClassTag.Double -> w2double _
-    case cz if (cz == classOf[FloatWritable]) => ClassTag.Float -> w2float _
-    case cz if (cz == classOf[Text]) => ClassTag(classOf[String]) -> w2string _
-    case cz if (cz == classOf[BooleanWritable]) => ClassTag(classOf[Boolean]) -> w2bool _
-    case cz if (cz == classOf[BytesWritable]) => ClassTag(classOf[Array[Byte]]) -> w2bytes _
-    case _ => throw new IllegalArgumentException(s"Unsupported DRM key type:${keyTypeWritable.getName}")
-  }
-
-}
-
-object DrmMetadata {
-
-  private[common] def w2int(w: Writable) = w.asInstanceOf[IntWritable].get()
-
-  private[common] def w2long(w: Writable) = w.asInstanceOf[LongWritable].get()
-
-  private[common] def w2double(w: Writable) = w.asInstanceOf[DoubleWritable].get()
-
-  private[common] def w2float(w: Writable) = w.asInstanceOf[FloatWritable].get()
-
-  private[common] def w2string(w: Writable) = w.asInstanceOf[Text].toString()
-
-  private[common] def w2bool(w: Writable) = w.asInstanceOf[BooleanWritable].get()
-
-  private[common] def w2bytes(w: Writable) = Arrays.copyOf(w.asInstanceOf[BytesWritable].getBytes(),
-                                                           w.asInstanceOf[BytesWritable].getLength())
-}

http://git-wip-us.apache.org/repos/asf/mahout/blob/8439846c/h2o/src/main/scala/org/apache/mahout/common/HDFSUtil.scala
----------------------------------------------------------------------
diff --git a/h2o/src/main/scala/org/apache/mahout/common/HDFSUtil.scala b/h2o/src/main/scala/org/apache/mahout/common/HDFSUtil.scala
deleted file mode 100644
index f5f87d7..0000000
--- a/h2o/src/main/scala/org/apache/mahout/common/HDFSUtil.scala
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.mahout.common
-
-/** High level Hadoop version-specific hdfs manipulations we need in context of our operations. */
-trait HDFSUtil {
-
-  /** Read DRM header information off (H)DFS. */
-  def readDrmHeader(path:String):DrmMetadata
-}
-

http://git-wip-us.apache.org/repos/asf/mahout/blob/8439846c/h2o/src/main/scala/org/apache/mahout/common/Hadoop1HDFSUtil.scala
----------------------------------------------------------------------
diff --git a/h2o/src/main/scala/org/apache/mahout/common/Hadoop1HDFSUtil.scala b/h2o/src/main/scala/org/apache/mahout/common/Hadoop1HDFSUtil.scala
deleted file mode 100644
index e1ec135..0000000
--- a/h2o/src/main/scala/org/apache/mahout/common/Hadoop1HDFSUtil.scala
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.mahout.common
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.io.{SequenceFile, Writable}
-
-/**
- * Deprecated Hadoop 1 api which we currently explicitly import via Mahout dependencies. May not work
- * with Hadoop 2.0
- */
-object Hadoop1HDFSUtil extends HDFSUtil {
-
-  
-  def readDrmHeader(path: String): DrmMetadata = {
-    val dfsPath = new Path(path)
-    val fs = dfsPath.getFileSystem(new Configuration())
-
-    val partFilePath:Path = fs.listStatus(dfsPath)
-
-        // Filter out anything starting with .
-        .filter { s => !s.getPath.getName.startsWith("\\.") && !s.getPath.getName.startsWith("_") && !s.isDir }
-
-        // Take path
-        .map(_.getPath)
-
-        // Take only one, if any
-        .headOption
-
-        // Require there's at least one partition file found.
-        .getOrElse {
-      throw new IllegalArgumentException(s"No partition files found in ${dfsPath.toString}.")
-    }
-
-    val reader = new SequenceFile.Reader(fs, partFilePath, fs.getConf)
-    try {
-      new DrmMetadata(
-        keyTypeWritable = reader.getKeyClass.asSubclass(classOf[Writable]),
-        valueTypeWritable = reader.getValueClass.asSubclass(classOf[Writable])
-      )
-    } finally {
-      reader.close()
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/mahout/blob/8439846c/h2o/src/main/scala/org/apache/mahout/h2o/common/DrmMetadata.scala
----------------------------------------------------------------------
diff --git a/h2o/src/main/scala/org/apache/mahout/h2o/common/DrmMetadata.scala b/h2o/src/main/scala/org/apache/mahout/h2o/common/DrmMetadata.scala
new file mode 100644
index 0000000..3eb0974
--- /dev/null
+++ b/h2o/src/main/scala/org/apache/mahout/h2o/common/DrmMetadata.scala
@@ -0,0 +1,56 @@
+package org.apache.mahout.h2o.common
+
+import scala.reflect.ClassTag
+import org.apache.hadoop.io._
+import java.util.Arrays
+
+class DrmMetadata(
+
+    /** Writable  key type as a sub-type of Writable */
+    val keyTypeWritable: Class[_],
+
+    /** Value writable type, as a sub-type of Writable */
+    val valueTypeWritable: Class[_]
+
+    ) {
+
+  import DrmMetadata._
+
+  val (
+
+      /** Actual drm key class tag once converted out of writable */
+      keyClassTag: ClassTag[_],
+
+      /** Conversion from Writable to value type of the DRM key */
+      keyW2ValFunc: ((Writable) => Any)
+
+      ) = keyTypeWritable match {
+    case cz if (cz == classOf[IntWritable]) => ClassTag.Int -> w2int _
+    case cz if (cz == classOf[LongWritable]) => ClassTag.Long -> w2long _
+    case cz if (cz == classOf[DoubleWritable]) => ClassTag.Double -> w2double _
+    case cz if (cz == classOf[FloatWritable]) => ClassTag.Float -> w2float _
+    case cz if (cz == classOf[Text]) => ClassTag(classOf[String]) -> w2string _
+    case cz if (cz == classOf[BooleanWritable]) => ClassTag(classOf[Boolean]) -> w2bool _
+    case cz if (cz == classOf[BytesWritable]) => ClassTag(classOf[Array[Byte]]) -> w2bytes _
+    case _ => throw new IllegalArgumentException(s"Unsupported DRM key type:${keyTypeWritable.getName}")
+  }
+
+}
+
+object DrmMetadata {
+
+  private[common] def w2int(w: Writable) = w.asInstanceOf[IntWritable].get()
+
+  private[common] def w2long(w: Writable) = w.asInstanceOf[LongWritable].get()
+
+  private[common] def w2double(w: Writable) = w.asInstanceOf[DoubleWritable].get()
+
+  private[common] def w2float(w: Writable) = w.asInstanceOf[FloatWritable].get()
+
+  private[common] def w2string(w: Writable) = w.asInstanceOf[Text].toString()
+
+  private[common] def w2bool(w: Writable) = w.asInstanceOf[BooleanWritable].get()
+
+  private[common] def w2bytes(w: Writable) = Arrays.copyOf(w.asInstanceOf[BytesWritable].getBytes(),
+                                                           w.asInstanceOf[BytesWritable].getLength())
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/8439846c/h2o/src/main/scala/org/apache/mahout/h2o/common/HDFSUtil.scala
----------------------------------------------------------------------
diff --git a/h2o/src/main/scala/org/apache/mahout/h2o/common/HDFSUtil.scala b/h2o/src/main/scala/org/apache/mahout/h2o/common/HDFSUtil.scala
new file mode 100644
index 0000000..266b6aa
--- /dev/null
+++ b/h2o/src/main/scala/org/apache/mahout/h2o/common/HDFSUtil.scala
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+package org.apache.mahout.h2o.common
+
+/** High level Hadoop version-specific hdfs manipulations we need in context of our operations. */
+trait HDFSUtil {
+
+  /** Read DRM header information off (H)DFS. */
+  def readDrmHeader(path:String):DrmMetadata
+}
+

http://git-wip-us.apache.org/repos/asf/mahout/blob/8439846c/h2o/src/main/scala/org/apache/mahout/h2o/common/Hadoop1HDFSUtil.scala
----------------------------------------------------------------------
diff --git a/h2o/src/main/scala/org/apache/mahout/h2o/common/Hadoop1HDFSUtil.scala b/h2o/src/main/scala/org/apache/mahout/h2o/common/Hadoop1HDFSUtil.scala
new file mode 100644
index 0000000..a540cb1
--- /dev/null
+++ b/h2o/src/main/scala/org/apache/mahout/h2o/common/Hadoop1HDFSUtil.scala
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+
+package org.apache.mahout.h2o.common
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.{SequenceFile, Writable}
+
+/**
+ * Deprecated Hadoop 1 api which we currently explicitly import via Mahout dependencies. May not work
+ * with Hadoop 2.0
+ */
+object Hadoop1HDFSUtil extends HDFSUtil {
+
+  
+  def readDrmHeader(path: String): DrmMetadata = {
+    val dfsPath = new Path(path)
+    val fs = dfsPath.getFileSystem(new Configuration())
+
+    val partFilePath:Path = fs.listStatus(dfsPath)
+
+        // Filter out anything starting with .
+        .filter { s => !s.getPath.getName.startsWith("\\.") && !s.getPath.getName.startsWith("_") && !s.isDir }
+
+        // Take path
+        .map(_.getPath)
+
+        // Take only one, if any
+        .headOption
+
+        // Require there's at least one partition file found.
+        .getOrElse {
+      throw new IllegalArgumentException(s"No partition files found in ${dfsPath.toString}.")
+    }
+
+    val reader = new SequenceFile.Reader(fs, partFilePath, fs.getConf)
+    try {
+      new DrmMetadata(
+        keyTypeWritable = reader.getKeyClass.asSubclass(classOf[Writable]),
+        valueTypeWritable = reader.getValueClass.asSubclass(classOf[Writable])
+      )
+    } finally {
+      reader.close()
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/mahout/blob/8439846c/h2o/src/main/scala/org/apache/mahout/h2obindings/H2OEngine.scala
----------------------------------------------------------------------
diff --git a/h2o/src/main/scala/org/apache/mahout/h2obindings/H2OEngine.scala b/h2o/src/main/scala/org/apache/mahout/h2obindings/H2OEngine.scala
index 99bc3ba..4c34f31 100644
--- a/h2o/src/main/scala/org/apache/mahout/h2obindings/H2OEngine.scala
+++ b/h2o/src/main/scala/org/apache/mahout/h2obindings/H2OEngine.scala
@@ -26,7 +26,7 @@ import org.apache.mahout.math.drm._
 import org.apache.mahout.math.drm.logical._
 import org.apache.mahout.h2obindings.ops._
 import org.apache.mahout.h2obindings.drm._
-import org.apache.mahout.common.{Hadoop1HDFSUtil, HDFSUtil}
+import org.apache.mahout.h2o.common.{Hadoop1HDFSUtil, HDFSUtil}
 
 /** H2O specific non-DRM operations */
 object H2OEngine extends DistributedEngine {