Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1608#discussion_r15438442
  
    --- Diff: 
external/hbase/src/main/scala/org/apache/spark/hbase/HBaseContext.scala ---
    @@ -0,0 +1,544 @@
    +/*
    + * 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.spark.hbase
    +
    +import org.apache.hadoop.hbase.HBaseConfiguration
    +import org.apache.spark.rdd.RDD
    +import java.io.ByteArrayOutputStream
    +import java.io.DataOutputStream
    +import java.io.ByteArrayInputStream
    +import java.io.DataInputStream
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.hbase.client.HConnectionManager
    +import org.apache.spark.api.java.JavaPairRDD
    +import java.io.OutputStream
    +import org.apache.hadoop.hbase.client.HTable
    +import org.apache.hadoop.hbase.client.Scan
    +import org.apache.hadoop.hbase.client.Get
    +import java.util.ArrayList
    +import org.apache.hadoop.hbase.client.Result
    +import scala.reflect.ClassTag
    +import org.apache.hadoop.hbase.client.HConnection
    +import org.apache.hadoop.hbase.client.Put
    +import org.apache.hadoop.hbase.client.Increment
    +import org.apache.hadoop.hbase.client.Delete
    +import org.apache.spark.SparkContext
    +import org.apache.hadoop.hbase.mapreduce.TableInputFormat
    +import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil
    +import org.apache.hadoop.hbase.io.ImmutableBytesWritable
    +import org.apache.hadoop.hbase.util.Bytes
    +import org.apache.hadoop.mapreduce.Job
    +import org.apache.hadoop.hbase.mapreduce.TableMapper
    +import org.apache.hadoop.hbase.mapreduce.IdentityTableMapper
    +import org.apache.hadoop.hbase.protobuf.ProtobufUtil
    +import org.apache.hadoop.hbase.util.Base64
    +import org.apache.hadoop.hbase.mapreduce.MutationSerialization
    +import org.apache.hadoop.hbase.mapreduce.ResultSerialization
    +import org.apache.hadoop.hbase.mapreduce.KeyValueSerialization
    +import org.apache.spark.rdd.HadoopRDD
    +import org.apache.spark.broadcast.Broadcast
    +import org.apache.spark.SerializableWritable
    +import java.util.HashMap
    +import org.apache.hadoop.hbase.protobuf.generated.MasterProtos
    +import org.apache.hadoop.hbase.protobuf.generated.MasterProtos
    +import java.util.concurrent.atomic.AtomicInteger
    +import org.apache.hadoop.hbase.HConstants
    +import java.util.concurrent.atomic.AtomicLong
    +import java.util.Timer
    +import java.util.TimerTask
    +import org.apache.hadoop.hbase.client.Mutation
    +import scala.collection.mutable.MutableList
    +import org.apache.spark.streaming.dstream.DStream
    +
    +/**
    + * HBaseContext is a façade of simple and complex HBase operations
    + * like bulk put, get, increment, delete, and scan
    + *
    + * HBase Context will take the responsibilities to happen to
    + * complexity of disseminating the configuration information
    + * to the working and managing the life cycle of HConnections.
    + *
    + * First constructor:
    + *  @param sc - active SparkContext
    + *  @param broadcastedConf - This is a Broadcast object that holds a
    + * serializable Configuration object
    + *
    + */
    +@serializable class HBaseContext(@transient sc: SparkContext,
    +  broadcastedConf: Broadcast[SerializableWritable[Configuration]]) {
    +
    +  /**
    +   * Second constructor option:
    +   *  @param sc     active SparkContext
    +   *  @param config Configuration object to make connection to HBase
    +   */
    +  def this(@transient sc: SparkContext, @transient config: Configuration) {
    +    this(sc, sc.broadcast(new SerializableWritable(config)))
    +  }
    +
    +  /**
    +   * A simple enrichment of the traditional Spark RDD foreachPartition.
    +   * This function differs from the original in that it offers the 
    +   * developer access to a already connected HConnection object
    +   * 
    +   * Note: Do not close the HConnection object.  All HConnection
    +   * management is handled outside this method
    +   * 
    +   * @param RDD[t]  Original RDD with data to iterate over
    +   * @param f       function to be given a iterator to iterate through
    +   *                the RDD values and a HConnection object to interact 
    +   *                with HBase 
    +   */
    +  def foreachPartition[T](rdd: RDD[T],
    +    f: (Iterator[T], HConnection) => Unit) = {
    +    rdd.foreachPartition(
    +      it => hbaseForeachPartition(broadcastedConf, it, f))
    +  }
    +
    +  /**
    +   * A simple enrichment of the traditional Spark Streaming dStream foreach
    +   * This function differs from the original in that it offers the 
    +   * developer access to a already connected HConnection object
    +   * 
    +   * Note: Do not close the HConnection object.  All HConnection
    +   * management is handled outside this method
    +   * 
    +   * @param DStream[t]  Original DStream with data to iterate over
    +   * @param f           function to be given a iterator to iterate through
    +   *                    the DStream values and a HConnection object to 
    +   *                    interact with HBase 
    +   */
    +  def streamForeach[T](dstream: DStream[T],
    +    f: (Iterator[T], HConnection) => Unit) = {
    +    dstream.foreach((rdd, time) => {
    +      foreachPartition(rdd, f)
    +    })
    +  }
    +
    +  /**
    +   * A simple enrichment of the traditional Spark RDD mapPartition.
    +   * This function differs from the original in that it offers the 
    +   * developer access to a already connected HConnection object
    +   * 
    +   * Note: Do not close the HConnection object.  All HConnection
    +   * management is handled outside this method
    +   * 
    +   * Note: Make sure to partition correctly to avoid memory issue when
    +   *       getting data from HBase
    +   * 
    +   * @param RDD[t]  Original RDD with data to iterate over
    +   * @param mp      function to be given a iterator to iterate through
    +   *                the RDD values and a HConnection object to interact 
    +   *                with HBase
    +   * @return        Returns a new RDD generated by the user definition
    +   *                function just like normal mapPartition
    +   */
    +  def mapPartition[T, U: ClassTag](rdd: RDD[T],
    +    mp: (Iterator[T], HConnection) => Iterator[U]): RDD[U] = {
    +
    +    rdd.mapPartitions[U](it => hbaseMapPartition[T, U](broadcastedConf,
    +      it,
    +      mp), true)
    +  }
    +
    +  /**
    +   * A simple enrichment of the traditional Spark Streaming DStream
    +   * mapPartition.
    +   * 
    +   * This function differs from the original in that it offers the 
    +   * developer access to a already connected HConnection object
    +   * 
    +   * Note: Do not close the HConnection object.  All HConnection
    +   * management is handled outside this method
    +   * 
    +   * Note: Make sure to partition correctly to avoid memory issue when
    +   *       getting data from HBase
    +   * 
    +   * @param DStream[t] Original DStream with data to iterate over
    +   * @param mp         function to be given a iterator to iterate through
    +   *                   the DStream values and a HConnection object to 
    +   *                   interact with HBase
    +   * @return           Returns a new DStream generated by the user 
    +   *                   definition function just like normal mapPartition
    +   */
    +  def streamMap[T, U: ClassTag](dstream: DStream[T],
    +    mp: (Iterator[T], HConnection) => Iterator[U]): DStream[U] = {
    +
    +    dstream.mapPartitions(it => hbaseMapPartition[T, U](broadcastedConf,
    +      it,
    +      mp), true)
    +  }
    +
    +  /**
    +   * A simple abstraction over the HBaseContext.foreachPartition method.
    +   * 
    +   * It allow addition support for a user to take RDD 
    +   * and generate puts and send them to HBase.  
    +   * The complexity of even the HConnection is 
    +   * removed from the developer
    +   * 
    +   * @param RDD[t]  Original RDD with data to iterate over
    +   * @tableName     The name of the table to put into 
    +   * @param f       function to convert a value in the RDD to a HBase Put
    +   * @autoFlush     if autoFlush should be turned on
    +   */
    +  def bulkPut[T](rdd: RDD[T], tableName: String, f: (T) => Put, autoFlush: 
Boolean) {
    +
    +    rdd.foreachPartition(
    +      it => hbaseForeachPartition[T](
    +        broadcastedConf,
    +        it,
    +        (iterator, hConnection) => {
    +          val htable = hConnection.getTable(tableName)
    +          htable.setAutoFlush(autoFlush, true)
    +          iterator.foreach(T => htable.put(f(T)))
    +          htable.flushCommits()
    +          htable.close()
    +        }))
    +  }
    +
    +  /**
    +   * A simple abstraction over the HBaseContext.streamMapPartition method.
    +   * 
    +   * It allow addition support for a user to take a DStream and 
    +   * generate puts and send them to HBase.  
    +   * 
    +   * The complexity of even the HConnection is 
    +   * removed from the developer
    +   * 
    +   * @param DStream[t] Original DStream with data to iterate over
    +   * @tableName        The name of the table to put into 
    +   * @param f          function to convert a value in the RDD to a HBase 
Put
    +   * @autoFlush        if autoFlush should be turned on
    +   */
    +  def streamBulkPut[T](dstream: DStream[T],
    +    tableName: String,
    +    f: (T) => Put,
    +    autoFlush: Boolean) = {
    +    dstream.foreach((rdd, time) => {
    +      bulkPut(rdd, tableName, f, autoFlush)
    +    })
    +  }
    +
    +  /**
    +   * A simple abstraction over the HBaseContext.foreachPartition method.
    +   * 
    +   * It allow addition support for a user to take a RDD and 
    +   * generate increments and send them to HBase.  
    +   * 
    +   * The complexity of even the HConnection is 
    +   * removed from the developer
    +   * 
    +   * @param RDD[t]  Original RDD with data to iterate over
    +   * @tableName     The name of the table to increment to 
    +   * @param f       function to convert a value in the RDD to a 
    +   *                HBase Increments
    +   * @batchSize     The number of increments to batch before sending to 
HBase
    +   */
    +  def bulkIncrement[T](rdd: RDD[T], tableName:String, f:(T) => Increment, 
batchSize: Integer) {
    --- End diff --
    
    batchSize is declared as a `java.lang.Integer` instead of a `scala.Int`. I 
bet it all kind of works but would `Int` not be more standard?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to