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 org.apache.hadoop.conf.Configuration; 020import org.apache.hadoop.hbase.HBaseConfiguration; 021import org.apache.hadoop.hbase.HConstants; 022import org.apache.hadoop.hbase.TableName; 023import org.apache.hadoop.hbase.spark.FamilyHFileWriteOptions; 024import org.apache.hadoop.hbase.spark.JavaHBaseContext; 025import org.apache.hadoop.hbase.spark.KeyFamilyQualifier; 026import org.apache.hadoop.hbase.util.Bytes; 027import org.apache.hadoop.hbase.util.Pair; 028import org.apache.spark.api.java.JavaRDD; 029import org.apache.spark.api.java.JavaSparkContext; 030import org.apache.spark.SparkConf; 031import org.apache.spark.api.java.function.Function; 032 033import java.util.ArrayList; 034import java.util.HashMap; 035import java.util.List; 036 037/** 038 * Run this example using command below: 039 * 040 * SPARK_HOME/bin/spark-submit --master local[2] --class org.apache.hadoop.hbase.spark.example.hbasecontext.JavaHBaseBulkLoadExample 041 * path/to/hbase-spark.jar {path/to/output/HFiles} 042 * 043 * This example will output put hfiles in {path/to/output/HFiles}, and user can run 044 * 'hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles' to load the HFiles into table to verify this example. 045 */ 046final public class JavaHBaseBulkLoadExample { 047 private JavaHBaseBulkLoadExample() {} 048 049 public static void main(String[] args) { 050 if (args.length < 1) { 051 System.out.println("JavaHBaseBulkLoadExample " + "{outputPath}"); 052 return; 053 } 054 055 String tableName = "bulkload-table-test"; 056 String columnFamily1 = "f1"; 057 String columnFamily2 = "f2"; 058 059 SparkConf sparkConf = new SparkConf().setAppName("JavaHBaseBulkLoadExample " + tableName); 060 JavaSparkContext jsc = new JavaSparkContext(sparkConf); 061 062 try { 063 List<String> list= new ArrayList<String>(); 064 // row1 065 list.add("1," + columnFamily1 + ",b,1"); 066 // row3 067 list.add("3," + columnFamily1 + ",a,2"); 068 list.add("3," + columnFamily1 + ",b,1"); 069 list.add("3," + columnFamily2 + ",a,1"); 070 /* row2 */ 071 list.add("2," + columnFamily2 + ",a,3"); 072 list.add("2," + columnFamily2 + ",b,3"); 073 074 JavaRDD<String> rdd = jsc.parallelize(list); 075 076 Configuration conf = HBaseConfiguration.create(); 077 JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf); 078 079 080 081 hbaseContext.bulkLoad(rdd, TableName.valueOf(tableName),new BulkLoadFunction(), args[0], 082 new HashMap<byte[], FamilyHFileWriteOptions>(), false, HConstants.DEFAULT_MAX_FILE_SIZE); 083 } finally { 084 jsc.stop(); 085 } 086 } 087 088 public static class BulkLoadFunction implements Function<String, Pair<KeyFamilyQualifier, byte[]>> { 089 090 @Override 091 public Pair<KeyFamilyQualifier, byte[]> call(String v1) throws Exception { 092 if (v1 == null) 093 return null; 094 String[] strs = v1.split(","); 095 if(strs.length != 4) 096 return null; 097 KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]), Bytes.toBytes(strs[1]), 098 Bytes.toBytes(strs[2])); 099 return new Pair(kfq, Bytes.toBytes(strs[3])); 100 } 101 } 102}