001/* 002 * Licensed to the Apache Software Foundation (ASF) under one or more 003 * contributor license agreements. See the NOTICE file distributed with 004 * this work for additional information regarding copyright ownership. 005 * The ASF licenses this file to You under the Apache License, Version 2.0 006 * (the "License"); you may not use this file except in compliance with 007 * the License. You may obtain a copy of the License at 008 * 009 * http://www.apache.org/licenses/LICENSE-2.0 010 * 011 * Unless required by applicable law or agreed to in writing, software 012 * distributed under the License is distributed on an "AS IS" BASIS, 013 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 014 * See the License for the specific language governing permissions and 015 * limitations under the License. 016 */ 017package org.apache.hadoop.hbase.spark.example.hbasecontext; 018 019import java.util.ArrayList; 020import java.util.Iterator; 021import java.util.List; 022 023import org.apache.hadoop.conf.Configuration; 024import org.apache.hadoop.hbase.HBaseConfiguration; 025import org.apache.hadoop.hbase.TableName; 026import org.apache.hadoop.hbase.client.BufferedMutator; 027import org.apache.hadoop.hbase.client.Connection; 028import org.apache.hadoop.hbase.client.Get; 029import org.apache.hadoop.hbase.client.Put; 030import org.apache.hadoop.hbase.client.Result; 031import org.apache.hadoop.hbase.client.Table; 032import org.apache.hadoop.hbase.spark.JavaHBaseContext; 033import org.apache.hadoop.hbase.util.Bytes; 034import org.apache.spark.SparkConf; 035import org.apache.spark.api.java.JavaRDD; 036import org.apache.spark.api.java.JavaSparkContext; 037import org.apache.spark.api.java.function.Function; 038import org.apache.spark.api.java.function.VoidFunction; 039 040import scala.Tuple2; 041 042/** 043 * This is a simple example of using the foreachPartition 044 * method with a HBase connection 045 */ 046final public class JavaHBaseMapGetPutExample { 047 048 private JavaHBaseMapGetPutExample() {} 049 050 public static void main(String[] args) { 051 if (args.length < 1) { 052 System.out.println("JavaHBaseBulkGetExample {tableName}"); 053 return; 054 } 055 056 final String tableName = args[0]; 057 058 SparkConf sparkConf = new SparkConf().setAppName("JavaHBaseBulkGetExample " + tableName); 059 JavaSparkContext jsc = new JavaSparkContext(sparkConf); 060 061 try { 062 List<byte[]> list = new ArrayList<>(5); 063 list.add(Bytes.toBytes("1")); 064 list.add(Bytes.toBytes("2")); 065 list.add(Bytes.toBytes("3")); 066 list.add(Bytes.toBytes("4")); 067 list.add(Bytes.toBytes("5")); 068 069 JavaRDD<byte[]> rdd = jsc.parallelize(list); 070 Configuration conf = HBaseConfiguration.create(); 071 072 JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf); 073 074 hbaseContext.foreachPartition(rdd, 075 new VoidFunction<Tuple2<Iterator<byte[]>, Connection>>() { 076 public void call(Tuple2<Iterator<byte[]>, Connection> t) 077 throws Exception { 078 Table table = t._2().getTable(TableName.valueOf(tableName)); 079 BufferedMutator mutator = t._2().getBufferedMutator(TableName.valueOf(tableName)); 080 081 while (t._1().hasNext()) { 082 byte[] b = t._1().next(); 083 Result r = table.get(new Get(b)); 084 if (r.getExists()) { 085 mutator.mutate(new Put(b)); 086 } 087 } 088 089 mutator.flush(); 090 mutator.close(); 091 table.close(); 092 } 093 }); 094 } finally { 095 jsc.stop(); 096 } 097 } 098 099 public static class GetFunction implements Function<byte[], Get> { 100 private static final long serialVersionUID = 1L; 101 public Get call(byte[] v) throws Exception { 102 return new Get(v); 103 } 104 } 105}