HBASE-18817 pull the hbase-spark module out of branch-2.

Signed-off-by: Michael Stack <stack@apache.org>
This commit is contained in:
Sean Busbey 2017-09-21 14:41:43 -05:00
parent 6d88c495f7
commit 516d370b4b
75 changed files with 2 additions and 15922 deletions

View File

@ -246,9 +246,4 @@
<Source name="~.*\.scala" />
</Match>
<Match>
<Package name="org.apache.hadoop.hbase.spark.protobuf.generated"/>
</Match>
</FindBugsFilter>

View File

@ -37,22 +37,6 @@
</properties>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-enforcer-plugin</artifactId>
<executions>
<!-- hbase-spark is ok in the assembly -->
<execution>
<id>banned-hbase-spark</id>
<goals>
<goal>enforce</goal>
</goals>
<configuration>
<skip>true</skip>
</configuration>
</execution>
</executions>
</plugin>
<!-- licensing info from our dependencies -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
@ -266,11 +250,6 @@
<version>${project.version}</version>
<optional>true</optional>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-spark</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
@ -279,12 +258,6 @@
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpcore</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-spark-it</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-backup</artifactId>

View File

@ -46,7 +46,6 @@
<include>org.apache.hbase:hbase-rsgroup</include>
<include>org.apache.hbase:hbase-server</include>
<include>org.apache.hbase:hbase-shell</include>
<include>org.apache.hbase:hbase-spark</include>
<include>org.apache.hbase:hbase-thrift</include>
<include>org.apache.hbase:hbase-external-blockcache</include>
<include>org.apache.hbase:hbase-backup</include>

View File

@ -1,333 +0,0 @@
<?xml version="1.0"?>
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<!--
/**
* 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.
*/
-->
<modelVersion>4.0.0</modelVersion>
<parent>
<artifactId>hbase-build-configuration</artifactId>
<groupId>org.apache.hbase</groupId>
<version>2.0.0-beta-1.SNAPSHOT</version>
<relativePath>../hbase-build-configuration</relativePath>
</parent>
<artifactId>hbase-spark-it</artifactId>
<name>Apache HBase - Spark Integration Tests</name>
<description>Integration and System tests for HBase</description>
<properties>
<spark.version>1.6.0</spark.version>
<scala.version>2.10.4</scala.version>
<scala.binary.version>2.10</scala.binary.version>
<!-- Test inclusion patterns used by failsafe configuration -->
<unittest.include>**/Test*.java</unittest.include>
<integrationtest.include>**/IntegrationTest*.java</integrationtest.include>
<!-- To Run Tests with a particular Xmx Value use -Dfailsafe.Xmx=XXXg -->
<failsafe.Xmx>4g</failsafe.Xmx>
<!-- To run a single integration test, use -Dit.test=IntegrationTestXXX -->
</properties>
<build>
<pluginManagement>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-site-plugin</artifactId>
<configuration>
<skip>true</skip>
</configuration>
</plugin>
<!-- Make a jar and put the sources in the jar -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-source-plugin</artifactId>
</plugin>
<plugin>
<!--Make it so assembly:single does nothing in here-->
<artifactId>maven-assembly-plugin</artifactId>
<configuration>
<skipAssembly>true</skipAssembly>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-failsafe-plugin</artifactId>
<version>${surefire.version}</version>
<dependencies>
<dependency>
<groupId>org.apache.maven.surefire</groupId>
<artifactId>surefire-junit4</artifactId>
<version>${surefire.version}</version>
</dependency>
</dependencies>
<configuration>
<includes>
<include>${integrationtest.include}</include>
</includes>
<excludes>
<exclude>${unittest.include}</exclude>
<exclude>**/*$*</exclude>
</excludes>
<redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
<failIfNoTests>false</failIfNoTests>
<testFailureIgnore>false</testFailureIgnore>
</configuration>
<executions>
<execution>
<id>integration-test</id>
<phase>integration-test</phase>
<goals>
<goal>integration-test</goal>
</goals>
</execution>
<execution>
<id>verify</id>
<phase>verify</phase>
<goals>
<goal>verify</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</pluginManagement>
<plugins>
<!-- Run integration tests with mvn verify -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-failsafe-plugin</artifactId>
<configuration>
<skip>false</skip>
<forkMode>always</forkMode>
<!-- TODO: failsafe does timeout, but verify does not fail the build because of the timeout.
I believe it is a failsafe bug, we may consider using surefire -->
<forkedProcessTimeoutInSeconds>1800</forkedProcessTimeoutInSeconds>
<argLine>-enableassertions -Xmx${failsafe.Xmx}
-Djava.security.egd=file:/dev/./urandom -XX:+CMSClassUnloadingEnabled
-verbose:gc -XX:+PrintCommandLineFlags -XX:+PrintFlagsFinal</argLine>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-enforcer-plugin</artifactId>
<executions>
<!-- purposefully have jsr 305 exclusion only warn in this module -->
<execution>
<id>banned-jsr305</id>
<goals>
<goal>enforce</goal>
</goals>
<configuration>
<fail>false</fail>
</configuration>
</execution>
<!-- hbase-spark is ok in this modules -->
<execution>
<id>banned-hbase-spark</id>
<goals>
<goal>enforce</goal>
</goals>
<configuration>
<skip>true</skip>
</configuration>
</execution>
<execution>
<id>banned-scala</id>
<goals>
<goal>enforce</goal>
</goals>
<configuration>
<skip>true</skip>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<artifactId>maven-dependency-plugin</artifactId>
<executions>
<execution>
<id>create-mrapp-generated-classpath</id>
<phase>generate-test-resources</phase>
<goals>
<goal>build-classpath</goal>
</goals>
<configuration>
<!-- needed to run the unit test for DS to generate
the required classpath that is required in the env
of the launch container in the mini cluster
-->
<outputFile>${project.build.directory}/test-classes/spark-generated-classpath</outputFile>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
<dependencies>
<!-- Intra-project dependencies -->
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-common</artifactId>
<type>jar</type>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-server</artifactId>
<type>jar</type>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-server</artifactId>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-spark</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-it</artifactId>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>${compat.module}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-testing-util</artifactId>
</dependency>
<dependency>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</dependency>
<dependency>
<groupId>commons-cli</groupId>
<artifactId>commons-cli</artifactId>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
<!-- Hadoop needs Netty 3.x at test scope for the minicluster -->
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>
<version>${netty.hadoop.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<!-- make sure wrong scala version is not pulled in -->
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
</exclusion>
<exclusion>
<!-- make sure wrong scala version is not pulled in -->
<groupId>org.scala-lang</groupId>
<artifactId>scalap</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<type>test-jar</type>
<classifier>tests</classifier>
<scope>test</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<profiles>
<!-- Skip the tests in this module -->
<profile>
<id>skipIntegrationTests</id>
<activation>
<property>
<name>skipIntegrationTests</name>
</property>
</activation>
<properties>
<skipTests>true</skipTests>
</properties>
</profile>
</profiles>
<reporting>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-report-plugin</artifactId>
<version>2.7.2</version>
<reportSets>
<reportSet>
<id>spark-integration-tests</id>
<reports>
<report>report-only</report>
</reports>
<configuration>
<outputName>failsafe-report</outputName>
<reportsDirectories>
<reportsDirectory>${project.build.directory}/failsafe-reports</reportsDirectory>
</reportsDirectories>
</configuration>
</reportSet>
</reportSets>
</plugin>
</plugins>
</reporting>
</project>

View File

@ -1,663 +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.hadoop.hbase.spark;
import com.google.common.collect.Sets;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.IntegrationTestBase;
import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.IntegrationTestBulkLoad;
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.RegionSplitter;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.ToolRunner;
import org.apache.spark.SerializableWritable;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.Partitioner;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.Function2;
import org.apache.spark.api.java.function.PairFlatMapFunction;
import org.apache.spark.api.java.function.VoidFunction;
import org.junit.Test;
import scala.Tuple2;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
/**
* Test Bulk Load and Spark on a distributed cluster.
* It starts an Spark job that creates linked chains.
* This test mimic {@link IntegrationTestBulkLoad} in mapreduce.
*
* Usage on cluster:
* First add hbase related jars and hbase-spark.jar into spark classpath.
*
* spark-submit --class org.apache.hadoop.hbase.spark.IntegrationTestSparkBulkLoad
* HBASE_HOME/lib/hbase-spark-it-XXX-tests.jar -m slowDeterministic -Dhbase.spark.bulkload.chainlength=300
*/
public class IntegrationTestSparkBulkLoad extends IntegrationTestBase {
private static final Log LOG = LogFactory.getLog(IntegrationTestSparkBulkLoad.class);
// The number of partitions for random generated data
private static String BULKLOAD_PARTITIONS_NUM = "hbase.spark.bulkload.partitionsnum";
private static int DEFAULT_BULKLOAD_PARTITIONS_NUM = 3;
private static String BULKLOAD_CHAIN_LENGTH = "hbase.spark.bulkload.chainlength";
private static int DEFAULT_BULKLOAD_CHAIN_LENGTH = 200000;
private static String BULKLOAD_IMPORT_ROUNDS = "hbase.spark.bulkload.importround";
private static int DEFAULT_BULKLOAD_IMPORT_ROUNDS = 1;
private static String CURRENT_ROUND_NUM = "hbase.spark.bulkload.current.roundnum";
private static String NUM_REPLICA_COUNT_KEY = "hbase.spark.bulkload.replica.countkey";
private static int DEFAULT_NUM_REPLICA_COUNT = 1;
private static String BULKLOAD_TABLE_NAME = "hbase.spark.bulkload.tableName";
private static String DEFAULT_BULKLOAD_TABLE_NAME = "IntegrationTestSparkBulkLoad";
private static String BULKLOAD_OUTPUT_PATH = "hbase.spark.bulkload.output.path";
private static final String OPT_LOAD = "load";
private static final String OPT_CHECK = "check";
private boolean load = false;
private boolean check = false;
private static final byte[] CHAIN_FAM = Bytes.toBytes("L");
private static final byte[] SORT_FAM = Bytes.toBytes("S");
private static final byte[] DATA_FAM = Bytes.toBytes("D");
/**
* Running spark job to load data into hbase table
*/
public void runLoad() throws Exception {
setupTable();
int numImportRounds = getConf().getInt(BULKLOAD_IMPORT_ROUNDS, DEFAULT_BULKLOAD_IMPORT_ROUNDS);
LOG.info("Running load with numIterations:" + numImportRounds);
for (int i = 0; i < numImportRounds; i++) {
runLinkedListSparkJob(i);
}
}
/**
* Running spark job to create LinkedList for testing
* @param iteration iteration th of this job
* @throws Exception
*/
public void runLinkedListSparkJob(int iteration) throws Exception {
String jobName = IntegrationTestSparkBulkLoad.class.getSimpleName() + " _load " +
EnvironmentEdgeManager.currentTime();
LOG.info("Running iteration " + iteration + "in Spark Job");
Path output = null;
if (conf.get(BULKLOAD_OUTPUT_PATH) == null) {
output = util.getDataTestDirOnTestFS(getTablename() + "-" + iteration);
} else {
output = new Path(conf.get(BULKLOAD_OUTPUT_PATH));
}
SparkConf sparkConf = new SparkConf().setAppName(jobName).setMaster("local");
Configuration hbaseConf = new Configuration(getConf());
hbaseConf.setInt(CURRENT_ROUND_NUM, iteration);
int partitionNum = hbaseConf.getInt(BULKLOAD_PARTITIONS_NUM, DEFAULT_BULKLOAD_PARTITIONS_NUM);
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, hbaseConf);
LOG.info("Partition RDD into " + partitionNum + " parts");
List<String> temp = new ArrayList<>();
JavaRDD<List<byte[]>> rdd = jsc.parallelize(temp, partitionNum).
mapPartitionsWithIndex(new LinkedListCreationMapper(new SerializableWritable<>(hbaseConf)), false);
hbaseContext.bulkLoad(rdd, getTablename(), new ListToKeyValueFunc(), output.toUri().getPath(),
new HashMap<>(), false, HConstants.DEFAULT_MAX_FILE_SIZE);
try (Connection conn = ConnectionFactory.createConnection(conf);
Admin admin = conn.getAdmin();
Table table = conn.getTable(getTablename());
RegionLocator regionLocator = conn.getRegionLocator(getTablename())) {
// Create a new loader.
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
// Load the HFiles into table.
loader.doBulkLoad(output, admin, table, regionLocator);
}
// Delete the files.
util.getTestFileSystem().delete(output, true);
jsc.close();
}
// See mapreduce.IntegrationTestBulkLoad#LinkedListCreationMapper
// Used to generate test data
public static class LinkedListCreationMapper implements
Function2<Integer, Iterator<String>, Iterator<List<byte[]>>> {
SerializableWritable swConfig = null;
private Random rand = new Random();
public LinkedListCreationMapper(SerializableWritable conf) {
this.swConfig = conf;
}
@Override
public Iterator<List<byte[]>> call(Integer v1, Iterator v2) throws Exception {
Configuration config = (Configuration) swConfig.value();
int partitionId = v1.intValue();
LOG.info("Starting create List in Partition " + partitionId);
int partitionNum = config.getInt(BULKLOAD_PARTITIONS_NUM, DEFAULT_BULKLOAD_PARTITIONS_NUM);
int chainLength = config.getInt(BULKLOAD_CHAIN_LENGTH, DEFAULT_BULKLOAD_CHAIN_LENGTH);
int iterationsNum = config.getInt(BULKLOAD_IMPORT_ROUNDS, DEFAULT_BULKLOAD_IMPORT_ROUNDS);
int iterationsCur = config.getInt(CURRENT_ROUND_NUM, 0);
List<List<byte[]>> res = new LinkedList<>();
long tempId = partitionId + iterationsCur * partitionNum;
long totalPartitionNum = partitionNum * iterationsNum;
long chainId = Math.abs(rand.nextLong());
chainId = chainId - (chainId % totalPartitionNum) + tempId;
byte[] chainIdArray = Bytes.toBytes(chainId);
long currentRow = 0;
long nextRow = getNextRow(0, chainLength);
for(long i = 0; i < chainLength; i++) {
byte[] rk = Bytes.toBytes(currentRow);
// Insert record into a list
List<byte[]> tmp1 = Arrays.asList(rk, CHAIN_FAM, chainIdArray, Bytes.toBytes(nextRow));
List<byte[]> tmp2 = Arrays.asList(rk, SORT_FAM, chainIdArray, Bytes.toBytes(i));
List<byte[]> tmp3 = Arrays.asList(rk, DATA_FAM, chainIdArray, Bytes.toBytes(
RandomStringUtils.randomAlphabetic(50)));
res.add(tmp1);
res.add(tmp2);
res.add(tmp3);
currentRow = nextRow;
nextRow = getNextRow(i+1, chainLength);
}
return res.iterator();
}
/** Returns a unique row id within this chain for this index */
private long getNextRow(long index, long chainLength) {
long nextRow = Math.abs(new Random().nextLong());
// use significant bits from the random number, but pad with index to ensure it is unique
// this also ensures that we do not reuse row = 0
// row collisions from multiple mappers are fine, since we guarantee unique chainIds
nextRow = nextRow - (nextRow % chainLength) + index;
return nextRow;
}
}
public static class ListToKeyValueFunc implements
Function<List<byte[]>, Pair<KeyFamilyQualifier, byte[]>> {
@Override
public Pair<KeyFamilyQualifier, byte[]> call(List<byte[]> v1) throws Exception {
if (v1 == null || v1.size() != 4) {
return null;
}
KeyFamilyQualifier kfq = new KeyFamilyQualifier(v1.get(0), v1.get(1), v1.get(2));
return new Pair<>(kfq, v1.get(3));
}
}
/**
* After adding data to the table start a mr job to
* @throws IOException
* @throws ClassNotFoundException
* @throws InterruptedException
*/
public void runCheck() throws Exception {
LOG.info("Running check");
String jobName = IntegrationTestSparkBulkLoad.class.getSimpleName() + "_check" + EnvironmentEdgeManager.currentTime();
SparkConf sparkConf = new SparkConf().setAppName(jobName).setMaster("local");
Configuration hbaseConf = new Configuration(getConf());
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, hbaseConf);
Scan scan = new Scan();
scan.addFamily(CHAIN_FAM);
scan.addFamily(SORT_FAM);
scan.setMaxVersions(1);
scan.setCacheBlocks(false);
scan.setBatch(1000);
int replicaCount = conf.getInt(NUM_REPLICA_COUNT_KEY, DEFAULT_NUM_REPLICA_COUNT);
if (replicaCount != DEFAULT_NUM_REPLICA_COUNT) {
scan.setConsistency(Consistency.TIMELINE);
}
// 1. Using TableInputFormat to get data from HBase table
// 2. Mimic LinkedListCheckingMapper in mapreduce.IntegrationTestBulkLoad
// 3. Sort LinkKey by its order ID
// 4. Group LinkKey if they have same chainId, and repartition RDD by NaturalKeyPartitioner
// 5. Check LinkList in each Partition using LinkedListCheckingFlatMapFunc
hbaseContext.hbaseRDD(getTablename(), scan).flatMapToPair(new LinkedListCheckingFlatMapFunc())
.sortByKey()
.combineByKey(new createCombinerFunc(), new mergeValueFunc(), new mergeCombinersFunc(),
new NaturalKeyPartitioner(new SerializableWritable<>(hbaseConf)))
.foreach(new LinkedListCheckingForeachFunc(new SerializableWritable<>(hbaseConf)));
jsc.close();
}
private void runCheckWithRetry() throws Exception {
try {
runCheck();
} catch (Throwable t) {
LOG.warn("Received " + StringUtils.stringifyException(t));
LOG.warn("Running the check MR Job again to see whether an ephemeral problem or not");
runCheck();
throw t; // we should still fail the test even if second retry succeeds
}
// everything green
}
/**
* PairFlatMapFunction used to transfer <Row, Result> to Tuple <SparkLinkKey, SparkLinkChain>
*/
public static class LinkedListCheckingFlatMapFunc implements
PairFlatMapFunction<Tuple2<ImmutableBytesWritable, Result>, SparkLinkKey, SparkLinkChain> {
@Override
public Iterable<Tuple2<SparkLinkKey, SparkLinkChain>> call(Tuple2<ImmutableBytesWritable, Result> v)
throws Exception {
Result value = v._2();
long longRk = Bytes.toLong(value.getRow());
List<Tuple2<SparkLinkKey, SparkLinkChain>> list = new LinkedList<>();
for (Map.Entry<byte[], byte[]> entry : value.getFamilyMap(CHAIN_FAM).entrySet()) {
long chainId = Bytes.toLong(entry.getKey());
long next = Bytes.toLong(entry.getValue());
Cell c = value.getColumnCells(SORT_FAM, entry.getKey()).get(0);
long order = Bytes.toLong(CellUtil.cloneValue(c));
Tuple2<SparkLinkKey, SparkLinkChain> tuple2 =
new Tuple2<>(new SparkLinkKey(chainId, order), new SparkLinkChain(longRk, next));
list.add(tuple2);
}
return list;
}
}
public static class createCombinerFunc implements
Function<SparkLinkChain, List<SparkLinkChain>> {
@Override
public List<SparkLinkChain> call(SparkLinkChain v1) throws Exception {
List<SparkLinkChain> list = new LinkedList<>();
list.add(v1);
return list;
}
}
public static class mergeValueFunc implements
Function2<List<SparkLinkChain>, SparkLinkChain, List<SparkLinkChain>> {
@Override
public List<SparkLinkChain> call(List<SparkLinkChain> v1, SparkLinkChain v2) throws Exception {
if (v1 == null)
v1 = new LinkedList<>();
v1.add(v2);
return v1;
}
}
public static class mergeCombinersFunc implements
Function2<List<SparkLinkChain>, List<SparkLinkChain>, List<SparkLinkChain>> {
@Override
public List<SparkLinkChain> call(List<SparkLinkChain> v1, List<SparkLinkChain> v2) throws Exception {
v1.addAll(v2);
return v1;
}
}
/**
* Class to figure out what partition to send a link in the chain to. This is based upon
* the linkKey's ChainId.
*/
public static class NaturalKeyPartitioner extends Partitioner {
private int numPartions = 0;
public NaturalKeyPartitioner(SerializableWritable swConf) {
Configuration hbaseConf = (Configuration) swConf.value();
numPartions = hbaseConf.getInt(BULKLOAD_PARTITIONS_NUM, DEFAULT_BULKLOAD_PARTITIONS_NUM);
}
@Override
public int numPartitions() {
return numPartions;
}
@Override
public int getPartition(Object key) {
if (!(key instanceof SparkLinkKey))
return -1;
int hash = ((SparkLinkKey) key).getChainId().hashCode();
return Math.abs(hash % numPartions);
}
}
/**
* Sort all LinkChain for one LinkKey, and test List<LinkChain>
*/
public static class LinkedListCheckingForeachFunc
implements VoidFunction<Tuple2<SparkLinkKey, List<SparkLinkChain>>> {
private SerializableWritable swConf = null;
public LinkedListCheckingForeachFunc(SerializableWritable conf) {
swConf = conf;
}
@Override
public void call(Tuple2<SparkLinkKey, List<SparkLinkChain>> v1) throws Exception {
long next = -1L;
long prev = -1L;
long count = 0L;
SparkLinkKey key = v1._1();
List<SparkLinkChain> values = v1._2();
for (SparkLinkChain lc : values) {
if (next == -1) {
if (lc.getRk() != 0L) {
String msg = "Chains should all start at rk 0, but read rk " + lc.getRk()
+ ". Chain:" + key.getChainId() + ", order:" + key.getOrder();
throw new RuntimeException(msg);
}
next = lc.getNext();
} else {
if (next != lc.getRk()) {
String msg = "Missing a link in the chain. Prev rk " + prev + " was, expecting "
+ next + " but got " + lc.getRk() + ". Chain:" + key.getChainId()
+ ", order:" + key.getOrder();
throw new RuntimeException(msg);
}
prev = lc.getRk();
next = lc.getNext();
}
count++;
}
Configuration hbaseConf = (Configuration) swConf.value();
int expectedChainLen = hbaseConf.getInt(BULKLOAD_CHAIN_LENGTH, DEFAULT_BULKLOAD_CHAIN_LENGTH);
if (count != expectedChainLen) {
String msg = "Chain wasn't the correct length. Expected " + expectedChainLen + " got "
+ count + ". Chain:" + key.getChainId() + ", order:" + key.getOrder();
throw new RuntimeException(msg);
}
}
}
/**
* Writable class used as the key to group links in the linked list.
*
* Used as the key emited from a pass over the table.
*/
public static class SparkLinkKey implements java.io.Serializable, Comparable<SparkLinkKey> {
private Long chainId;
private Long order;
public Long getOrder() {
return order;
}
public Long getChainId() {
return chainId;
}
public SparkLinkKey(long chainId, long order) {
this.chainId = chainId;
this.order = order;
}
@Override
public int hashCode() {
return this.getChainId().hashCode();
}
@Override
public boolean equals(Object other) {
if (!(other instanceof SparkLinkKey))
return false;
SparkLinkKey otherKey = (SparkLinkKey) other;
return this.getChainId().equals(otherKey.getChainId());
}
@Override
public int compareTo(SparkLinkKey other) {
int res = getChainId().compareTo(other.getChainId());
if (res == 0)
res= getOrder().compareTo(other.getOrder());
return res;
}
}
/**
* Writable used as the value emitted from a pass over the hbase table.
*/
public static class SparkLinkChain implements java.io.Serializable, Comparable<SparkLinkChain>{
public Long getNext() {
return next;
}
public Long getRk() {
return rk;
}
public SparkLinkChain(Long rk, Long next) {
this.rk = rk;
this.next = next;
}
private Long rk;
private Long next;
@Override
public int compareTo(SparkLinkChain linkChain) {
int res = getRk().compareTo(linkChain.getRk());
if (res == 0) {
res = getNext().compareTo(linkChain.getNext());
}
return res;
}
@Override
public int hashCode() {
return getRk().hashCode() ^ getNext().hashCode();
}
@Override
public boolean equals(Object other) {
if (!(other instanceof SparkLinkChain))
return false;
SparkLinkChain otherKey = (SparkLinkChain) other;
return this.getRk().equals(otherKey.getRk()) && this.getNext().equals(otherKey.getNext());
}
}
/**
* Allow the scan to go to replica, this would not affect the runCheck()
* Since data are BulkLoaded from HFile into table
* @throws IOException
* @throws InterruptedException
*/
private void installSlowingCoproc() throws IOException, InterruptedException {
int replicaCount = conf.getInt(NUM_REPLICA_COUNT_KEY, DEFAULT_NUM_REPLICA_COUNT);
if (replicaCount == DEFAULT_NUM_REPLICA_COUNT) return;
TableName t = getTablename();
Admin admin = util.getAdmin();
HTableDescriptor desc = admin.getTableDescriptor(t);
desc.addCoprocessor(IntegrationTestBulkLoad.SlowMeCoproScanOperations.class.getName());
HBaseTestingUtility.modifyTableSync(admin, desc);
}
@Test
public void testBulkLoad() throws Exception {
runLoad();
installSlowingCoproc();
runCheckWithRetry();
}
private byte[][] getSplits(int numRegions) {
RegionSplitter.UniformSplit split = new RegionSplitter.UniformSplit();
split.setFirstRow(Bytes.toBytes(0L));
split.setLastRow(Bytes.toBytes(Long.MAX_VALUE));
return split.split(numRegions);
}
private void setupTable() throws IOException, InterruptedException {
if (util.getAdmin().tableExists(getTablename())) {
util.deleteTable(getTablename());
}
util.createTable(
getTablename(),
new byte[][]{CHAIN_FAM, SORT_FAM, DATA_FAM},
getSplits(16)
);
int replicaCount = conf.getInt(NUM_REPLICA_COUNT_KEY, DEFAULT_NUM_REPLICA_COUNT);
if (replicaCount == DEFAULT_NUM_REPLICA_COUNT) return;
TableName t = getTablename();
HBaseTestingUtility.setReplicas(util.getAdmin(), t, replicaCount);
}
@Override
public void setUpCluster() throws Exception {
util = getTestingUtil(getConf());
util.initializeCluster(1);
int replicaCount = getConf().getInt(NUM_REPLICA_COUNT_KEY, DEFAULT_NUM_REPLICA_COUNT);
if (LOG.isDebugEnabled() && replicaCount != DEFAULT_NUM_REPLICA_COUNT) {
LOG.debug("Region Replicas enabled: " + replicaCount);
}
// Scale this up on a real cluster
if (util.isDistributedCluster()) {
util.getConfiguration().setIfUnset(BULKLOAD_PARTITIONS_NUM, String.valueOf(DEFAULT_BULKLOAD_PARTITIONS_NUM));
util.getConfiguration().setIfUnset(BULKLOAD_IMPORT_ROUNDS, "1");
} else {
util.startMiniMapReduceCluster();
}
}
@Override
protected void addOptions() {
super.addOptions();
super.addOptNoArg(OPT_CHECK, "Run check only");
super.addOptNoArg(OPT_LOAD, "Run load only");
}
@Override
protected void processOptions(CommandLine cmd) {
super.processOptions(cmd);
check = cmd.hasOption(OPT_CHECK);
load = cmd.hasOption(OPT_LOAD);
}
@Override
public int runTestFromCommandLine() throws Exception {
if (load) {
runLoad();
} else if (check) {
installSlowingCoproc();
runCheckWithRetry();
} else {
testBulkLoad();
}
return 0;
}
@Override
public TableName getTablename() {
return getTableName(getConf());
}
public static TableName getTableName(Configuration conf) {
return TableName.valueOf(conf.get(BULKLOAD_TABLE_NAME, DEFAULT_BULKLOAD_TABLE_NAME));
}
@Override
protected Set<String> getColumnFamilies() {
return Sets.newHashSet(Bytes.toString(CHAIN_FAM) , Bytes.toString(DATA_FAM),
Bytes.toString(SORT_FAM));
}
public static void main(String[] args) throws Exception {
Configuration conf = HBaseConfiguration.create();
IntegrationTestingUtility.setUseDistributedCluster(conf);
int status = ToolRunner.run(conf, new IntegrationTestSparkBulkLoad(), args);
System.exit(status);
}
}

View File

@ -1,6 +0,0 @@
ON PROTOBUFS
This maven module has core protobuf definition files ('.protos') used by hbase
Spark that ship with hbase core including tests.
Generation of java files from protobuf .proto files included here is done as
part of the build.

View File

@ -1,702 +0,0 @@
<?xml version="1.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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<artifactId>hbase-build-configuration</artifactId>
<groupId>org.apache.hbase</groupId>
<version>2.0.0-beta-1.SNAPSHOT</version>
<relativePath>../hbase-build-configuration</relativePath>
</parent>
<artifactId>hbase-spark</artifactId>
<name>Apache HBase - Spark</name>
<properties>
<spark.version>1.6.0</spark.version>
<scala.version>2.10.4</scala.version>
<scala.binary.version>2.10</scala.binary.version>
<top.dir>${project.basedir}/..</top.dir>
</properties>
<dependencies>
<dependency>
<groupId>org.apache.hbase.thirdparty</groupId>
<artifactId>hbase-shaded-miscellaneous</artifactId>
</dependency>
<!-- Force import of Spark's servlet API for unit tests -->
<dependency>
<groupId>javax.servlet</groupId>
<artifactId>javax.servlet-api</artifactId>
<scope>test</scope>
</dependency>
<!-- Mark Spark / Scala as provided -->
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<version>${scala.version}</version>
<scope>provided</scope>
</dependency>
<!-- we exclude jsr305 below and then expressly relist it as
provided / optional to avoid dependency resolution possibly
bringing it back into runtime scope.
-->
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<!-- make sure wrong scala version is not pulled in -->
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
</exclusion>
<exclusion>
<!-- make sure wrong scala version is not pulled in -->
<groupId>org.scala-lang</groupId>
<artifactId>scalap</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
<version>1.3.9</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<type>test-jar</type>
<classifier>tests</classifier>
<scope>test</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
<artifactId>scalatest_${scala.binary.version}</artifactId>
<version>2.2.4</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalamock</groupId>
<artifactId>scalamock-scalatest-support_${scala.binary.version}</artifactId>
<version>3.1.4</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.module</groupId>
<artifactId>jackson-module-scala_${scala.binary.version}</artifactId>
<version>${jackson.version}</version>
<exclusions>
<exclusion>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
</exclusion>
<exclusion>
<groupId>org.scala-lang</groupId>
<artifactId>scala-reflect</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<version>${hadoop-two.version}</version>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>servlet-api</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet.jsp</groupId>
<artifactId>jsp-api</artifactId>
</exclusion>
<exclusion>
<groupId>org.jruby</groupId>
<artifactId>jruby-complete</artifactId>
</exclusion>
<exclusion>
<groupId>org.jboss.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
<exclusion>
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<version>${hadoop-two.version}</version>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>servlet-api</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet.jsp</groupId>
<artifactId>jsp-api</artifactId>
</exclusion>
<exclusion>
<groupId>org.jruby</groupId>
<artifactId>jruby-complete</artifactId>
</exclusion>
<exclusion>
<groupId>org.jboss.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
<exclusion>
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
<version>${hadoop-two.version}</version>
<type>test-jar</type>
<scope>test</scope>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>servlet-api</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet.jsp</groupId>
<artifactId>jsp-api</artifactId>
</exclusion>
<exclusion>
<groupId>org.jruby</groupId>
<artifactId>jruby-complete</artifactId>
</exclusion>
<exclusion>
<groupId>org.jboss.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
<exclusion>
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
<version>${hadoop-two.version}</version>
<type>test-jar</type>
<scope>test</scope>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>servlet-api</artifactId>
</exclusion>
<exclusion>
<groupId>javax.servlet.jsp</groupId>
<artifactId>jsp-api</artifactId>
</exclusion>
<exclusion>
<groupId>org.jruby</groupId>
<artifactId>jruby-complete</artifactId>
</exclusion>
<exclusion>
<groupId>org.jboss.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
<exclusion>
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
<exclusion>
<groupId>xerces</groupId>
<artifactId>xercesImpl</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.thrift</groupId>
<artifactId>thrift</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jsp-2.1</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jsp-api-2.1</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>servlet-api-2.5</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-core</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-json</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-server</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jetty</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jetty-util</artifactId>
</exclusion>
<exclusion>
<groupId>tomcat</groupId>
<artifactId>jasper-runtime</artifactId>
</exclusion>
<exclusion>
<groupId>tomcat</groupId>
<artifactId>jasper-compiler</artifactId>
</exclusion>
<exclusion>
<groupId>org.jboss.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
<exclusion>
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-protocol</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-protocol-shaded</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-annotations</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-annotations</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-hadoop-compat</artifactId>
<version>${project.version}</version>
<scope>test</scope>
<type>test-jar</type>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.thrift</groupId>
<artifactId>thrift</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jsp-2.1</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jsp-api-2.1</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>servlet-api-2.5</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-core</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-json</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-server</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jetty</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jetty-util</artifactId>
</exclusion>
<exclusion>
<groupId>tomcat</groupId>
<artifactId>jasper-runtime</artifactId>
</exclusion>
<exclusion>
<groupId>tomcat</groupId>
<artifactId>jasper-compiler</artifactId>
</exclusion>
<exclusion>
<groupId>org.jboss.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
<exclusion>
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-hadoop2-compat</artifactId>
<version>${project.version}</version>
<scope>test</scope>
<type>test-jar</type>
<exclusions>
<exclusion>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.thrift</groupId>
<artifactId>thrift</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jsp-2.1</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jsp-api-2.1</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>servlet-api-2.5</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-core</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-json</artifactId>
</exclusion>
<exclusion>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-server</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jetty</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jetty-util</artifactId>
</exclusion>
<exclusion>
<groupId>tomcat</groupId>
<artifactId>jasper-runtime</artifactId>
</exclusion>
<exclusion>
<groupId>tomcat</groupId>
<artifactId>jasper-compiler</artifactId>
</exclusion>
<exclusion>
<groupId>org.jboss.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
<exclusion>
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-server</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-server</artifactId>
<version>${project.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-mapreduce</artifactId>
</dependency>
<dependency>
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
</dependency>
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-mapreduce-client-jobclient</artifactId>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
</plugin>
<plugin>
<groupId>net.alchim31.maven</groupId>
<artifactId>scala-maven-plugin</artifactId>
<version>3.2.0</version>
<configuration>
<charset>${project.build.sourceEncoding}</charset>
<scalaVersion>${scala.version}</scalaVersion>
<args>
<arg>-feature</arg>
</args>
</configuration>
<executions>
<execution>
<id>scala-compile-first</id>
<phase>process-resources</phase>
<goals>
<goal>add-source</goal>
<goal>compile</goal>
</goals>
</execution>
<execution>
<id>scala-test-compile</id>
<phase>process-test-resources</phase>
<goals>
<goal>testCompile</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.scalatest</groupId>
<artifactId>scalatest-maven-plugin</artifactId>
<version>1.0</version>
<configuration>
<reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory>
<junitxml>.</junitxml>
<filereports>WDF TestSuite.txt</filereports>
<parallel>false</parallel>
<systemProperties>
<org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>org.apache.hadoop.hbase.shaded.</org.apache.hadoop.hbase.shaded.io.netty.packagePrefix>
</systemProperties>
</configuration>
<executions>
<execution>
<id>test</id>
<phase>test</phase>
<goals>
<goal>test</goal>
</goals>
<configuration>
<argLine>
-Xmx1536m -XX:ReservedCodeCacheSize=512m
</argLine>
<parallel>false</parallel>
</configuration>
</execution>
</executions>
</plugin>
<!-- clover fails due to scala/java cross compile. This guarantees that the scala is
compiled before the java that will be evaluated by code coverage (scala will not be).
https://confluence.atlassian.com/display/CLOVERKB/Java-+Scala+cross-compilation+error+-+cannot+find+symbol
-->
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<executions>
<execution>
<id>add-source</id>
<phase>validate</phase>
<goals>
<goal>add-source</goal>
</goals>
<configuration>
<sources>
<source>src/main/scala</source>
</sources>
</configuration>
</execution>
<execution>
<id>add-test-source</id>
<phase>validate</phase>
<goals>
<goal>add-test-source</goal>
</goals>
<configuration>
<sources>
<source>src/test/scala</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.xolstice.maven.plugins</groupId>
<artifactId>protobuf-maven-plugin</artifactId>
<executions>
<execution>
<id>compile-protoc</id>
<phase>generate-sources</phase>
<goals>
<goal>compile</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-enforcer-plugin</artifactId>
<executions>
<!-- purposefully have jsr 305 exclusion only warn in this module -->
<execution>
<id>banned-jsr305</id>
<goals>
<goal>enforce</goal>
</goals>
<configuration>
<fail>false</fail>
</configuration>
</execution>
<!-- scala is ok in the spark modules -->
<execution>
<id>banned-scala</id>
<goals>
<goal>enforce</goal>
</goals>
<configuration>
<skip>true</skip>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
<profiles>
<!-- Skip the tests in this module -->
<profile>
<id>skipSparkTests</id>
<activation>
<property>
<name>skipSparkTests</name>
</property>
</activation>
<properties>
<surefire.skipFirstPart>true</surefire.skipFirstPart>
<surefire.skipSecondPart>true</surefire.skipSecondPart>
<skipTests>true</skipTests>
</properties>
</profile>
</profiles>
</project>

View File

@ -1,275 +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.hadoop.hbase.spark;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.FilterBase;
import org.apache.hadoop.hbase.spark.datasources.BytesEncoder;
import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder;
import org.apache.hadoop.hbase.spark.protobuf.generated.SparkFilterProtos;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.spark.sql.datasources.hbase.Field;
import scala.collection.mutable.MutableList;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ByteString;
/**
* This filter will push down all qualifier logic given to us
* by SparkSQL so that we have make the filters at the region server level
* and avoid sending the data back to the client to be filtered.
*/
@InterfaceAudience.Private
public class SparkSQLPushDownFilter extends FilterBase{
protected static final Log log = LogFactory.getLog(SparkSQLPushDownFilter.class);
//The following values are populated with protobuffer
DynamicLogicExpression dynamicLogicExpression;
byte[][] valueFromQueryArray;
HashMap<ByteArrayComparable, HashMap<ByteArrayComparable, String>>
currentCellToColumnIndexMap;
//The following values are transient
HashMap<String, ByteArrayComparable> columnToCurrentRowValueMap = null;
static final byte[] rowKeyFamily = new byte[0];
static final byte[] rowKeyQualifier = Bytes.toBytes("key");
String encoderClassName;
public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression,
byte[][] valueFromQueryArray,
HashMap<ByteArrayComparable,
HashMap<ByteArrayComparable, String>>
currentCellToColumnIndexMap, String encoderClassName) {
this.dynamicLogicExpression = dynamicLogicExpression;
this.valueFromQueryArray = valueFromQueryArray;
this.currentCellToColumnIndexMap = currentCellToColumnIndexMap;
this.encoderClassName = encoderClassName;
}
public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression,
byte[][] valueFromQueryArray,
MutableList<Field> fields, String encoderClassName) {
this.dynamicLogicExpression = dynamicLogicExpression;
this.valueFromQueryArray = valueFromQueryArray;
this.encoderClassName = encoderClassName;
//generate family qualifier to index mapping
this.currentCellToColumnIndexMap =
new HashMap<>();
for (int i = 0; i < fields.size(); i++) {
Field field = fields.apply(i);
byte[] cfBytes = field.cfBytes();
ByteArrayComparable familyByteComparable =
new ByteArrayComparable(cfBytes, 0, cfBytes.length);
HashMap<ByteArrayComparable, String> qualifierIndexMap =
currentCellToColumnIndexMap.get(familyByteComparable);
if (qualifierIndexMap == null) {
qualifierIndexMap = new HashMap<>();
currentCellToColumnIndexMap.put(familyByteComparable, qualifierIndexMap);
}
byte[] qBytes = field.colBytes();
ByteArrayComparable qualifierByteComparable =
new ByteArrayComparable(qBytes, 0, qBytes.length);
qualifierIndexMap.put(qualifierByteComparable, field.colName());
}
}
@Override
public ReturnCode filterCell(final Cell c) throws IOException {
//If the map RowValueMap is empty then we need to populate
// the row key
if (columnToCurrentRowValueMap == null) {
columnToCurrentRowValueMap = new HashMap<>();
HashMap<ByteArrayComparable, String> qualifierColumnMap =
currentCellToColumnIndexMap.get(
new ByteArrayComparable(rowKeyFamily, 0, rowKeyFamily.length));
if (qualifierColumnMap != null) {
String rowKeyColumnName =
qualifierColumnMap.get(
new ByteArrayComparable(rowKeyQualifier, 0,
rowKeyQualifier.length));
//Make sure that the rowKey is part of the where clause
if (rowKeyColumnName != null) {
columnToCurrentRowValueMap.put(rowKeyColumnName,
new ByteArrayComparable(c.getRowArray(),
c.getRowOffset(), c.getRowLength()));
}
}
}
//Always populate the column value into the RowValueMap
ByteArrayComparable currentFamilyByteComparable =
new ByteArrayComparable(c.getFamilyArray(),
c.getFamilyOffset(),
c.getFamilyLength());
HashMap<ByteArrayComparable, String> qualifierColumnMap =
currentCellToColumnIndexMap.get(
currentFamilyByteComparable);
if (qualifierColumnMap != null) {
String columnName =
qualifierColumnMap.get(
new ByteArrayComparable(c.getQualifierArray(),
c.getQualifierOffset(),
c.getQualifierLength()));
if (columnName != null) {
columnToCurrentRowValueMap.put(columnName,
new ByteArrayComparable(c.getValueArray(),
c.getValueOffset(), c.getValueLength()));
}
}
return ReturnCode.INCLUDE;
}
@Override
public boolean filterRow() throws IOException {
try {
boolean result =
dynamicLogicExpression.execute(columnToCurrentRowValueMap,
valueFromQueryArray);
columnToCurrentRowValueMap = null;
return !result;
} catch (Throwable e) {
log.error("Error running dynamic logic on row", e);
}
return false;
}
/**
* @param pbBytes A pb serialized instance
* @return An instance of SparkSQLPushDownFilter
* @throws org.apache.hadoop.hbase.exceptions.DeserializationException
*/
@SuppressWarnings("unused")
public static SparkSQLPushDownFilter parseFrom(final byte[] pbBytes)
throws DeserializationException {
SparkFilterProtos.SQLPredicatePushDownFilter proto;
try {
proto = SparkFilterProtos.SQLPredicatePushDownFilter.parseFrom(pbBytes);
} catch (InvalidProtocolBufferException e) {
throw new DeserializationException(e);
}
String encoder = proto.getEncoderClassName();
BytesEncoder enc = JavaBytesEncoder.create(encoder);
//Load DynamicLogicExpression
DynamicLogicExpression dynamicLogicExpression =
DynamicLogicExpressionBuilder.build(proto.getDynamicLogicExpression(), enc);
//Load valuesFromQuery
final List<ByteString> valueFromQueryArrayList = proto.getValueFromQueryArrayList();
byte[][] valueFromQueryArray = new byte[valueFromQueryArrayList.size()][];
for (int i = 0; i < valueFromQueryArrayList.size(); i++) {
valueFromQueryArray[i] = valueFromQueryArrayList.get(i).toByteArray();
}
//Load mapping from HBase family/qualifier to Spark SQL columnName
HashMap<ByteArrayComparable, HashMap<ByteArrayComparable, String>>
currentCellToColumnIndexMap = new HashMap<>();
for (SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping
sqlPredicatePushDownCellToColumnMapping :
proto.getCellToColumnMappingList()) {
byte[] familyArray =
sqlPredicatePushDownCellToColumnMapping.getColumnFamily().toByteArray();
ByteArrayComparable familyByteComparable =
new ByteArrayComparable(familyArray, 0, familyArray.length);
HashMap<ByteArrayComparable, String> qualifierMap =
currentCellToColumnIndexMap.get(familyByteComparable);
if (qualifierMap == null) {
qualifierMap = new HashMap<>();
currentCellToColumnIndexMap.put(familyByteComparable, qualifierMap);
}
byte[] qualifierArray =
sqlPredicatePushDownCellToColumnMapping.getQualifier().toByteArray();
ByteArrayComparable qualifierByteComparable =
new ByteArrayComparable(qualifierArray, 0 ,qualifierArray.length);
qualifierMap.put(qualifierByteComparable,
sqlPredicatePushDownCellToColumnMapping.getColumnName());
}
return new SparkSQLPushDownFilter(dynamicLogicExpression,
valueFromQueryArray, currentCellToColumnIndexMap, encoder);
}
/**
* @return The filter serialized using pb
*/
public byte[] toByteArray() {
SparkFilterProtos.SQLPredicatePushDownFilter.Builder builder =
SparkFilterProtos.SQLPredicatePushDownFilter.newBuilder();
SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping.Builder columnMappingBuilder =
SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping.newBuilder();
builder.setDynamicLogicExpression(dynamicLogicExpression.toExpressionString());
for (byte[] valueFromQuery: valueFromQueryArray) {
builder.addValueFromQueryArray(ByteStringer.wrap(valueFromQuery));
}
for (Map.Entry<ByteArrayComparable, HashMap<ByteArrayComparable, String>>
familyEntry : currentCellToColumnIndexMap.entrySet()) {
for (Map.Entry<ByteArrayComparable, String> qualifierEntry :
familyEntry.getValue().entrySet()) {
columnMappingBuilder.setColumnFamily(
ByteStringer.wrap(familyEntry.getKey().bytes()));
columnMappingBuilder.setQualifier(
ByteStringer.wrap(qualifierEntry.getKey().bytes()));
columnMappingBuilder.setColumnName(qualifierEntry.getValue());
builder.addCellToColumnMapping(columnMappingBuilder.build());
}
}
builder.setEncoderClassName(encoderClassName);
return builder.build().toByteArray();
}
}

View File

@ -1,80 +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.hadoop.hbase.spark.example.hbasecontext;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.spark.JavaHBaseContext;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
import java.util.ArrayList;
import java.util.List;
/**
* This is a simple example of deleting records in HBase
* with the bulkDelete function.
*/
final public class JavaHBaseBulkDeleteExample {
private JavaHBaseBulkDeleteExample() {}
public static void main(String[] args) {
if (args.length < 1) {
System.out.println("JavaHBaseBulkDeleteExample {tableName}");
return;
}
String tableName = args[0];
SparkConf sparkConf = new SparkConf().setAppName("JavaHBaseBulkDeleteExample " + tableName);
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
try {
List<byte[]> list = new ArrayList<>(5);
list.add(Bytes.toBytes("1"));
list.add(Bytes.toBytes("2"));
list.add(Bytes.toBytes("3"));
list.add(Bytes.toBytes("4"));
list.add(Bytes.toBytes("5"));
JavaRDD<byte[]> rdd = jsc.parallelize(list);
Configuration conf = HBaseConfiguration.create();
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
hbaseContext.bulkDelete(rdd,
TableName.valueOf(tableName), new DeleteFunction(), 4);
} finally {
jsc.stop();
}
}
public static class DeleteFunction implements Function<byte[], Delete> {
private static final long serialVersionUID = 1L;
public Delete call(byte[] v) throws Exception {
return new Delete(v);
}
}
}

View File

@ -1,115 +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.hadoop.hbase.spark.example.hbasecontext;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.spark.JavaHBaseContext;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
/**
* This is a simple example of getting records in HBase
* with the bulkGet function.
*/
final public class JavaHBaseBulkGetExample {
private JavaHBaseBulkGetExample() {}
public static void main(String[] args) {
if (args.length < 1) {
System.out.println("JavaHBaseBulkGetExample {tableName}");
return;
}
String tableName = args[0];
SparkConf sparkConf = new SparkConf().setAppName("JavaHBaseBulkGetExample " + tableName);
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
try {
List<byte[]> list = new ArrayList<>(5);
list.add(Bytes.toBytes("1"));
list.add(Bytes.toBytes("2"));
list.add(Bytes.toBytes("3"));
list.add(Bytes.toBytes("4"));
list.add(Bytes.toBytes("5"));
JavaRDD<byte[]> rdd = jsc.parallelize(list);
Configuration conf = HBaseConfiguration.create();
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
hbaseContext.bulkGet(TableName.valueOf(tableName), 2, rdd, new GetFunction(),
new ResultFunction());
} finally {
jsc.stop();
}
}
public static class GetFunction implements Function<byte[], Get> {
private static final long serialVersionUID = 1L;
public Get call(byte[] v) throws Exception {
return new Get(v);
}
}
public static class ResultFunction implements Function<Result, String> {
private static final long serialVersionUID = 1L;
public String call(Result result) throws Exception {
Iterator<Cell> it = result.listCells().iterator();
StringBuilder b = new StringBuilder();
b.append(Bytes.toString(result.getRow())).append(":");
while (it.hasNext()) {
Cell cell = it.next();
String q = Bytes.toString(cell.getQualifierArray());
if (q.equals("counter")) {
b.append("(")
.append(Bytes.toString(cell.getQualifierArray()))
.append(",")
.append(Bytes.toLong(cell.getValueArray()))
.append(")");
} else {
b.append("(")
.append(Bytes.toString(cell.getQualifierArray()))
.append(",")
.append(Bytes.toString(cell.getValueArray()))
.append(")");
}
}
return b.toString();
}
}
}

View File

@ -1,102 +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.hadoop.hbase.spark.example.hbasecontext;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.spark.FamilyHFileWriteOptions;
import org.apache.hadoop.hbase.spark.JavaHBaseContext;
import org.apache.hadoop.hbase.spark.KeyFamilyQualifier;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.Function;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
/**
* Run this example using command below:
*
* SPARK_HOME/bin/spark-submit --master local[2] --class org.apache.hadoop.hbase.spark.example.hbasecontext.JavaHBaseBulkLoadExample
* path/to/hbase-spark.jar {path/to/output/HFiles}
*
* This example will output put hfiles in {path/to/output/HFiles}, and user can run
* 'hbase org.apache.hadoop.hbase.tool.LoadIncrementalHFiles' to load the HFiles into table to verify this example.
*/
final public class JavaHBaseBulkLoadExample {
private JavaHBaseBulkLoadExample() {}
public static void main(String[] args) {
if (args.length < 1) {
System.out.println("JavaHBaseBulkLoadExample " + "{outputPath}");
return;
}
String tableName = "bulkload-table-test";
String columnFamily1 = "f1";
String columnFamily2 = "f2";
SparkConf sparkConf = new SparkConf().setAppName("JavaHBaseBulkLoadExample " + tableName);
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
try {
List<String> list= new ArrayList<String>();
// row1
list.add("1," + columnFamily1 + ",b,1");
// row3
list.add("3," + columnFamily1 + ",a,2");
list.add("3," + columnFamily1 + ",b,1");
list.add("3," + columnFamily2 + ",a,1");
/* row2 */
list.add("2," + columnFamily2 + ",a,3");
list.add("2," + columnFamily2 + ",b,3");
JavaRDD<String> rdd = jsc.parallelize(list);
Configuration conf = HBaseConfiguration.create();
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
hbaseContext.bulkLoad(rdd, TableName.valueOf(tableName),new BulkLoadFunction(), args[0],
new HashMap<byte[], FamilyHFileWriteOptions>(), false, HConstants.DEFAULT_MAX_FILE_SIZE);
} finally {
jsc.stop();
}
}
public static class BulkLoadFunction implements Function<String, Pair<KeyFamilyQualifier, byte[]>> {
@Override
public Pair<KeyFamilyQualifier, byte[]> call(String v1) throws Exception {
if (v1 == null)
return null;
String[] strs = v1.split(",");
if(strs.length != 4)
return null;
KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]), Bytes.toBytes(strs[1]),
Bytes.toBytes(strs[2]));
return new Pair(kfq, Bytes.toBytes(strs[3]));
}
}
}

View File

@ -1,90 +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.hadoop.hbase.spark.example.hbasecontext;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.spark.JavaHBaseContext;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
/**
* This is a simple example of putting records in HBase
* with the bulkPut function.
*/
final public class JavaHBaseBulkPutExample {
private JavaHBaseBulkPutExample() {}
public static void main(String[] args) {
if (args.length < 2) {
System.out.println("JavaHBaseBulkPutExample " +
"{tableName} {columnFamily}");
return;
}
String tableName = args[0];
String columnFamily = args[1];
SparkConf sparkConf = new SparkConf().setAppName("JavaHBaseBulkPutExample " + tableName);
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
try {
List<String> list = new ArrayList<>(5);
list.add("1," + columnFamily + ",a,1");
list.add("2," + columnFamily + ",a,2");
list.add("3," + columnFamily + ",a,3");
list.add("4," + columnFamily + ",a,4");
list.add("5," + columnFamily + ",a,5");
JavaRDD<String> rdd = jsc.parallelize(list);
Configuration conf = HBaseConfiguration.create();
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
hbaseContext.bulkPut(rdd,
TableName.valueOf(tableName),
new PutFunction());
} finally {
jsc.stop();
}
}
public static class PutFunction implements Function<String, Put> {
private static final long serialVersionUID = 1L;
public Put call(String v) throws Exception {
String[] cells = v.split(",");
Put put = new Put(Bytes.toBytes(cells[0]));
put.addColumn(Bytes.toBytes(cells[1]), Bytes.toBytes(cells[2]),
Bytes.toBytes(cells[3]));
return put;
}
}
}

View File

@ -1,81 +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.hadoop.hbase.spark.example.hbasecontext;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.spark.JavaHBaseContext;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
import scala.Tuple2;
/**
* This is a simple example of scanning records from HBase
* with the hbaseRDD function.
*/
final public class JavaHBaseDistributedScan {
private JavaHBaseDistributedScan() {}
public static void main(String[] args) {
if (args.length < 1) {
System.out.println("JavaHBaseDistributedScan {tableName}");
return;
}
String tableName = args[0];
SparkConf sparkConf = new SparkConf().setAppName("JavaHBaseDistributedScan " + tableName);
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
try {
Configuration conf = HBaseConfiguration.create();
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
Scan scan = new Scan();
scan.setCaching(100);
JavaRDD<Tuple2<ImmutableBytesWritable, Result>> javaRdd =
hbaseContext.hbaseRDD(TableName.valueOf(tableName), scan);
List<String> results = javaRdd.map(new ScanConvertFunction()).collect();
System.out.println("Result Size: " + results.size());
} finally {
jsc.stop();
}
}
private static class ScanConvertFunction implements
Function<Tuple2<ImmutableBytesWritable, Result>, String> {
@Override
public String call(Tuple2<ImmutableBytesWritable, Result> v1) throws Exception {
return Bytes.toString(v1._1().copyBytes());
}
}
}

View File

@ -1,105 +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.hadoop.hbase.spark.example.hbasecontext;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.spark.JavaHBaseContext;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.VoidFunction;
import scala.Tuple2;
/**
* This is a simple example of using the foreachPartition
* method with a HBase connection
*/
final public class JavaHBaseMapGetPutExample {
private JavaHBaseMapGetPutExample() {}
public static void main(String[] args) {
if (args.length < 1) {
System.out.println("JavaHBaseBulkGetExample {tableName}");
return;
}
final String tableName = args[0];
SparkConf sparkConf = new SparkConf().setAppName("JavaHBaseBulkGetExample " + tableName);
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
try {
List<byte[]> list = new ArrayList<>(5);
list.add(Bytes.toBytes("1"));
list.add(Bytes.toBytes("2"));
list.add(Bytes.toBytes("3"));
list.add(Bytes.toBytes("4"));
list.add(Bytes.toBytes("5"));
JavaRDD<byte[]> rdd = jsc.parallelize(list);
Configuration conf = HBaseConfiguration.create();
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
hbaseContext.foreachPartition(rdd,
new VoidFunction<Tuple2<Iterator<byte[]>, Connection>>() {
public void call(Tuple2<Iterator<byte[]>, Connection> t)
throws Exception {
Table table = t._2().getTable(TableName.valueOf(tableName));
BufferedMutator mutator = t._2().getBufferedMutator(TableName.valueOf(tableName));
while (t._1().hasNext()) {
byte[] b = t._1().next();
Result r = table.get(new Get(b));
if (r.getExists()) {
mutator.mutate(new Put(b));
}
}
mutator.flush();
mutator.close();
table.close();
}
});
} finally {
jsc.stop();
}
}
public static class GetFunction implements Function<byte[], Get> {
private static final long serialVersionUID = 1L;
public Get call(byte[] v) throws Exception {
return new Get(v);
}
}
}

View File

@ -1,90 +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.hadoop.hbase.spark.example.hbasecontext;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.spark.JavaHBaseContext;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.streaming.Duration;
import org.apache.spark.streaming.api.java.JavaReceiverInputDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;
/**
* This is a simple example of BulkPut with Spark Streaming
*/
final public class JavaHBaseStreamingBulkPutExample {
private JavaHBaseStreamingBulkPutExample() {}
public static void main(String[] args) {
if (args.length < 4) {
System.out.println("JavaHBaseBulkPutExample " +
"{host} {port} {tableName}");
return;
}
String host = args[0];
String port = args[1];
String tableName = args[2];
SparkConf sparkConf =
new SparkConf().setAppName("JavaHBaseStreamingBulkPutExample " +
tableName + ":" + port + ":" + tableName);
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
try {
JavaStreamingContext jssc =
new JavaStreamingContext(jsc, new Duration(1000));
JavaReceiverInputDStream<String> javaDstream =
jssc.socketTextStream(host, Integer.parseInt(port));
Configuration conf = HBaseConfiguration.create();
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
hbaseContext.streamBulkPut(javaDstream,
TableName.valueOf(tableName),
new PutFunction());
} finally {
jsc.stop();
}
}
public static class PutFunction implements Function<String, Put> {
private static final long serialVersionUID = 1L;
public Put call(String v) throws Exception {
String[] part = v.split(",");
Put put = new Put(Bytes.toBytes(part[0]));
put.addColumn(Bytes.toBytes(part[1]),
Bytes.toBytes(part[2]),
Bytes.toBytes(part[3]));
return put;
}
}
}

View File

@ -1,40 +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.
*/
// This file contains protocol buffers that are used for Spark filters
// over in the hbase-spark module
package hbase.pb;
option java_package = "org.apache.hadoop.hbase.spark.protobuf.generated";
option java_outer_classname = "SparkFilterProtos";
option java_generic_services = true;
option java_generate_equals_and_hash = true;
option optimize_for = SPEED;
message SQLPredicatePushDownCellToColumnMapping {
required bytes column_family = 1;
required bytes qualifier = 2;
required string column_name = 3;
}
message SQLPredicatePushDownFilter {
required string dynamic_logic_expression = 1;
repeated bytes value_from_query_array = 2;
repeated SQLPredicatePushDownCellToColumnMapping cell_to_column_mapping = 3;
optional string encoderClassName = 4;
}

View File

@ -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.hadoop.hbase.spark
import java.util
import java.util.Comparator
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.Partitioner
/**
* A Partitioner implementation that will separate records to different
* HBase Regions based on region splits
*
* @param startKeys The start keys for the given table
*/
@InterfaceAudience.Public
class BulkLoadPartitioner(startKeys:Array[Array[Byte]])
extends Partitioner {
// when table not exist, startKeys = Byte[0][]
override def numPartitions: Int = if (startKeys.length == 0) 1 else startKeys.length
override def getPartition(key: Any): Int = {
val comparator: Comparator[Array[Byte]] = new Comparator[Array[Byte]] {
override def compare(o1: Array[Byte], o2: Array[Byte]): Int = {
Bytes.compareTo(o1, o2)
}
}
val rowKey:Array[Byte] =
key match {
case qualifier: KeyFamilyQualifier =>
qualifier.rowKey
case wrapper: ByteArrayWrapper =>
wrapper.value
case _ =>
key.asInstanceOf[Array[Byte]]
}
var partition = util.Arrays.binarySearch(startKeys, rowKey, comparator)
if (partition < 0)
partition = partition * -1 + -2
if (partition < 0)
partition = 0
partition
}
}

View File

@ -1,49 +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.hadoop.hbase.spark
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes
@InterfaceAudience.Public
class ByteArrayComparable(val bytes:Array[Byte], val offset:Int = 0, var length:Int = -1)
extends Comparable[ByteArrayComparable] {
if (length == -1) {
length = bytes.length
}
override def compareTo(o: ByteArrayComparable): Int = {
Bytes.compareTo(bytes, offset, length, o.bytes, o.offset, o.length)
}
override def hashCode(): Int = {
Bytes.hashCode(bytes, offset, length)
}
override def equals (obj: Any): Boolean = {
obj match {
case b: ByteArrayComparable =>
Bytes.equals(bytes, offset, length, b.bytes, b.offset, b.length)
case _ =>
false
}
}
}

View File

@ -1,47 +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.hadoop.hbase.spark
import java.io.Serializable
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes
/**
* This is a wrapper over a byte array so it can work as
* a key in a hashMap
*
* @param value The Byte Array value
*/
@InterfaceAudience.Public
class ByteArrayWrapper (var value:Array[Byte])
extends Comparable[ByteArrayWrapper] with Serializable {
override def compareTo(valueOther: ByteArrayWrapper): Int = {
Bytes.compareTo(value,valueOther.value)
}
override def equals(o2: Any): Boolean = {
o2 match {
case wrapper: ByteArrayWrapper =>
Bytes.equals(value, wrapper.value)
case _ =>
false
}
}
override def hashCode():Int = {
Bytes.hashCode(value)
}
}

View File

@ -1,75 +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.hadoop.hbase.spark
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes
/**
* A wrapper class that will allow both columnFamily and qualifier to
* be the key of a hashMap. Also allow for finding the value in a hashmap
* with out cloning the HBase value from the HBase Cell object
* @param columnFamily ColumnFamily byte array
* @param columnFamilyOffSet Offset of columnFamily value in the array
* @param columnFamilyLength Length of the columnFamily value in the columnFamily array
* @param qualifier Qualifier byte array
* @param qualifierOffSet Offset of qualifier value in the array
* @param qualifierLength Length of the qualifier value with in the array
*/
@InterfaceAudience.Public
class ColumnFamilyQualifierMapKeyWrapper(val columnFamily:Array[Byte],
val columnFamilyOffSet:Int,
val columnFamilyLength:Int,
val qualifier:Array[Byte],
val qualifierOffSet:Int,
val qualifierLength:Int)
extends Serializable{
override def equals(other:Any): Boolean = {
val otherWrapper = other.asInstanceOf[ColumnFamilyQualifierMapKeyWrapper]
Bytes.compareTo(columnFamily,
columnFamilyOffSet,
columnFamilyLength,
otherWrapper.columnFamily,
otherWrapper.columnFamilyOffSet,
otherWrapper.columnFamilyLength) == 0 && Bytes.compareTo(qualifier,
qualifierOffSet,
qualifierLength,
otherWrapper.qualifier,
otherWrapper.qualifierOffSet,
otherWrapper.qualifierLength) == 0
}
override def hashCode():Int = {
Bytes.hashCode(columnFamily, columnFamilyOffSet, columnFamilyLength) +
Bytes.hashCode(qualifier, qualifierOffSet, qualifierLength)
}
def cloneColumnFamily():Array[Byte] = {
val resultArray = new Array[Byte](columnFamilyLength)
System.arraycopy(columnFamily, columnFamilyOffSet, resultArray, 0, columnFamilyLength)
resultArray
}
def cloneQualifier():Array[Byte] = {
val resultArray = new Array[Byte](qualifierLength)
System.arraycopy(qualifier, qualifierOffSet, resultArray, 0, qualifierLength)
resultArray
}
}

View File

@ -1,260 +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.hadoop.hbase.spark
import java.util
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.spark.datasources.{BytesEncoder, JavaBytesEncoder}
import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder.JavaBytesEncoder
import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.sql.datasources.hbase.{Field, Utils}
import org.apache.spark.sql.types._
/**
* Dynamic logic for SQL push down logic there is an instance for most
* common operations and a pass through for other operations not covered here
*
* Logic can be nested with And or Or operators.
*
* A logic tree can be written out as a string and reconstructed from that string
*
*/
@InterfaceAudience.Private
trait DynamicLogicExpression {
def execute(columnToCurrentRowValueMap: util.HashMap[String, ByteArrayComparable],
valueFromQueryValueArray:Array[Array[Byte]]): Boolean
def toExpressionString: String = {
val strBuilder = new StringBuilder
appendToExpression(strBuilder)
strBuilder.toString()
}
def filterOps: JavaBytesEncoder = JavaBytesEncoder.Unknown
def appendToExpression(strBuilder:StringBuilder)
var encoder: BytesEncoder = _
def setEncoder(enc: BytesEncoder): DynamicLogicExpression = {
encoder = enc
this
}
}
@InterfaceAudience.Private
trait CompareTrait {
self: DynamicLogicExpression =>
def columnName: String
def valueFromQueryIndex: Int
def execute(columnToCurrentRowValueMap:
util.HashMap[String, ByteArrayComparable],
valueFromQueryValueArray:Array[Array[Byte]]): Boolean = {
val currentRowValue = columnToCurrentRowValueMap.get(columnName)
val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex)
currentRowValue != null &&
encoder.filter(currentRowValue.bytes, currentRowValue.offset, currentRowValue.length,
valueFromQuery, 0, valueFromQuery.length, filterOps)
}
}
@InterfaceAudience.Private
class AndLogicExpression (val leftExpression:DynamicLogicExpression,
val rightExpression:DynamicLogicExpression)
extends DynamicLogicExpression{
override def execute(columnToCurrentRowValueMap:
util.HashMap[String, ByteArrayComparable],
valueFromQueryValueArray:Array[Array[Byte]]): Boolean = {
leftExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray) &&
rightExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)
}
override def appendToExpression(strBuilder: StringBuilder): Unit = {
strBuilder.append("( ")
strBuilder.append(leftExpression.toExpressionString)
strBuilder.append(" AND ")
strBuilder.append(rightExpression.toExpressionString)
strBuilder.append(" )")
}
}
@InterfaceAudience.Private
class OrLogicExpression (val leftExpression:DynamicLogicExpression,
val rightExpression:DynamicLogicExpression)
extends DynamicLogicExpression{
override def execute(columnToCurrentRowValueMap:
util.HashMap[String, ByteArrayComparable],
valueFromQueryValueArray:Array[Array[Byte]]): Boolean = {
leftExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray) ||
rightExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)
}
override def appendToExpression(strBuilder: StringBuilder): Unit = {
strBuilder.append("( ")
strBuilder.append(leftExpression.toExpressionString)
strBuilder.append(" OR ")
strBuilder.append(rightExpression.toExpressionString)
strBuilder.append(" )")
}
}
@InterfaceAudience.Private
class EqualLogicExpression (val columnName:String,
val valueFromQueryIndex:Int,
val isNot:Boolean) extends DynamicLogicExpression{
override def execute(columnToCurrentRowValueMap:
util.HashMap[String, ByteArrayComparable],
valueFromQueryValueArray:Array[Array[Byte]]): Boolean = {
val currentRowValue = columnToCurrentRowValueMap.get(columnName)
val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex)
currentRowValue != null &&
Bytes.equals(valueFromQuery,
0, valueFromQuery.length, currentRowValue.bytes,
currentRowValue.offset, currentRowValue.length) != isNot
}
override def appendToExpression(strBuilder: StringBuilder): Unit = {
val command = if (isNot) "!=" else "=="
strBuilder.append(columnName + " " + command + " " + valueFromQueryIndex)
}
}
@InterfaceAudience.Private
class IsNullLogicExpression (val columnName:String,
val isNot:Boolean) extends DynamicLogicExpression{
override def execute(columnToCurrentRowValueMap:
util.HashMap[String, ByteArrayComparable],
valueFromQueryValueArray:Array[Array[Byte]]): Boolean = {
val currentRowValue = columnToCurrentRowValueMap.get(columnName)
(currentRowValue == null) != isNot
}
override def appendToExpression(strBuilder: StringBuilder): Unit = {
val command = if (isNot) "isNotNull" else "isNull"
strBuilder.append(columnName + " " + command)
}
}
@InterfaceAudience.Private
class GreaterThanLogicExpression (override val columnName:String,
override val valueFromQueryIndex:Int)
extends DynamicLogicExpression with CompareTrait{
override val filterOps = JavaBytesEncoder.Greater
override def appendToExpression(strBuilder: StringBuilder): Unit = {
strBuilder.append(columnName + " > " + valueFromQueryIndex)
}
}
@InterfaceAudience.Private
class GreaterThanOrEqualLogicExpression (override val columnName:String,
override val valueFromQueryIndex:Int)
extends DynamicLogicExpression with CompareTrait{
override val filterOps = JavaBytesEncoder.GreaterEqual
override def appendToExpression(strBuilder: StringBuilder): Unit = {
strBuilder.append(columnName + " >= " + valueFromQueryIndex)
}
}
@InterfaceAudience.Private
class LessThanLogicExpression (override val columnName:String,
override val valueFromQueryIndex:Int)
extends DynamicLogicExpression with CompareTrait {
override val filterOps = JavaBytesEncoder.Less
override def appendToExpression(strBuilder: StringBuilder): Unit = {
strBuilder.append(columnName + " < " + valueFromQueryIndex)
}
}
@InterfaceAudience.Private
class LessThanOrEqualLogicExpression (val columnName:String,
val valueFromQueryIndex:Int)
extends DynamicLogicExpression with CompareTrait{
override val filterOps = JavaBytesEncoder.LessEqual
override def appendToExpression(strBuilder: StringBuilder): Unit = {
strBuilder.append(columnName + " <= " + valueFromQueryIndex)
}
}
@InterfaceAudience.Private
class PassThroughLogicExpression() extends DynamicLogicExpression {
override def execute(columnToCurrentRowValueMap:
util.HashMap[String, ByteArrayComparable],
valueFromQueryValueArray: Array[Array[Byte]]): Boolean = true
override def appendToExpression(strBuilder: StringBuilder): Unit = {
// Fix the offset bug by add dummy to avoid crash the region server.
// because in the DynamicLogicExpressionBuilder.build function, the command is always retrieved from offset + 1 as below
// val command = expressionArray(offSet + 1)
// we have to padding it so that `Pass` is on the right offset.
strBuilder.append("dummy Pass -1")
}
}
@InterfaceAudience.Private
object DynamicLogicExpressionBuilder {
def build(expressionString: String, encoder: BytesEncoder): DynamicLogicExpression = {
val expressionAndOffset = build(expressionString.split(' '), 0, encoder)
expressionAndOffset._1
}
private def build(expressionArray:Array[String],
offSet:Int, encoder: BytesEncoder): (DynamicLogicExpression, Int) = {
val expr = {
if (expressionArray(offSet).equals("(")) {
val left = build(expressionArray, offSet + 1, encoder)
val right = build(expressionArray, left._2 + 1, encoder)
if (expressionArray(left._2).equals("AND")) {
(new AndLogicExpression(left._1, right._1), right._2 + 1)
} else if (expressionArray(left._2).equals("OR")) {
(new OrLogicExpression(left._1, right._1), right._2 + 1)
} else {
throw new Throwable("Unknown gate:" + expressionArray(left._2))
}
} else {
val command = expressionArray(offSet + 1)
if (command.equals("<")) {
(new LessThanLogicExpression(expressionArray(offSet),
expressionArray(offSet + 2).toInt), offSet + 3)
} else if (command.equals("<=")) {
(new LessThanOrEqualLogicExpression(expressionArray(offSet),
expressionArray(offSet + 2).toInt), offSet + 3)
} else if (command.equals(">")) {
(new GreaterThanLogicExpression(expressionArray(offSet),
expressionArray(offSet + 2).toInt), offSet + 3)
} else if (command.equals(">=")) {
(new GreaterThanOrEqualLogicExpression(expressionArray(offSet),
expressionArray(offSet + 2).toInt), offSet + 3)
} else if (command.equals("==")) {
(new EqualLogicExpression(expressionArray(offSet),
expressionArray(offSet + 2).toInt, false), offSet + 3)
} else if (command.equals("!=")) {
(new EqualLogicExpression(expressionArray(offSet),
expressionArray(offSet + 2).toInt, true), offSet + 3)
} else if (command.equals("isNull")) {
(new IsNullLogicExpression(expressionArray(offSet), false), offSet + 2)
} else if (command.equals("isNotNull")) {
(new IsNullLogicExpression(expressionArray(offSet), true), offSet + 2)
} else if (command.equals("Pass")) {
(new PassThroughLogicExpression, offSet + 3)
} else {
throw new Throwable("Unknown logic command:" + command)
}
}
}
expr._1.setEncoder(encoder)
expr
}
}

View File

@ -1,68 +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.hadoop.hbase.spark
import java.util
import org.apache.yetus.audience.InterfaceAudience;
/**
* This object is a clean way to store and sort all cells that will be bulk
* loaded into a single row
*/
@InterfaceAudience.Public
class FamiliesQualifiersValues extends Serializable {
//Tree maps are used because we need the results to
// be sorted when we read them
val familyMap = new util.TreeMap[ByteArrayWrapper,
util.TreeMap[ByteArrayWrapper, Array[Byte]]]()
//normally in a row there are more columns then
//column families this wrapper is reused for column
//family look ups
val reusableWrapper = new ByteArrayWrapper(null)
/**
* Adds a new cell to an existing row
* @param family HBase column family
* @param qualifier HBase column qualifier
* @param value HBase cell value
*/
def += (family: Array[Byte], qualifier: Array[Byte], value: Array[Byte]): Unit = {
reusableWrapper.value = family
var qualifierValues = familyMap.get(reusableWrapper)
if (qualifierValues == null) {
qualifierValues = new util.TreeMap[ByteArrayWrapper, Array[Byte]]()
familyMap.put(new ByteArrayWrapper(family), qualifierValues)
}
qualifierValues.put(new ByteArrayWrapper(qualifier), value)
}
/**
* A wrapper for "+=" method above, can be used by Java
* @param family HBase column family
* @param qualifier HBase column qualifier
* @param value HBase cell value
*/
def add(family: Array[Byte], qualifier: Array[Byte], value: Array[Byte]): Unit = {
this += (family, qualifier, value)
}
}

View File

@ -1,38 +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.hadoop.hbase.spark
import java.io.Serializable
import org.apache.yetus.audience.InterfaceAudience;
/**
* This object will hold optional data for how a given column family's
* writer will work
*
* @param compression String to define the Compression to be used in the HFile
* @param bloomType String to define the bloom type to be used in the HFile
* @param blockSize The block size to be used in the HFile
* @param dataBlockEncoding String to define the data block encoding to be used
* in the HFile
*/
@InterfaceAudience.Public
class FamilyHFileWriteOptions( val compression:String,
val bloomType: String,
val blockSize: Int,
val dataBlockEncoding: String) extends Serializable

View File

@ -1,265 +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.hadoop.hbase.spark
import java.io.IOException
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.hbase.client.{Admin, Connection, ConnectionFactory, RegionLocator, Table}
import org.apache.hadoop.hbase.ipc.RpcControllerFactory
import org.apache.hadoop.hbase.security.{User, UserProvider}
import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf
import org.apache.hadoop.hbase.{HConstants, TableName}
import org.apache.spark.Logging
import scala.collection.mutable
private[spark] object HBaseConnectionCache extends Logging {
// A hashmap of Spark-HBase connections. Key is HBaseConnectionKey.
val connectionMap = new mutable.HashMap[HBaseConnectionKey, SmartConnection]()
val cacheStat = HBaseConnectionCacheStat(0, 0, 0)
// in milliseconds
private final val DEFAULT_TIME_OUT: Long = HBaseSparkConf.DEFAULT_CONNECTION_CLOSE_DELAY
private var timeout = DEFAULT_TIME_OUT
private var closed: Boolean = false
var housekeepingThread = new Thread(new Runnable {
override def run() {
while (true) {
try {
Thread.sleep(timeout)
} catch {
case e: InterruptedException =>
// setTimeout() and close() may interrupt the sleep and it's safe
// to ignore the exception
}
if (closed)
return
performHousekeeping(false)
}
}
})
housekeepingThread.setDaemon(true)
housekeepingThread.start()
def getStat: HBaseConnectionCacheStat = {
connectionMap.synchronized {
cacheStat.numActiveConnections = connectionMap.size
cacheStat.copy()
}
}
def close(): Unit = {
try {
connectionMap.synchronized {
if (closed)
return
closed = true
housekeepingThread.interrupt()
housekeepingThread = null
HBaseConnectionCache.performHousekeeping(true)
}
} catch {
case e: Exception => logWarning("Error in finalHouseKeeping", e)
}
}
def performHousekeeping(forceClean: Boolean) = {
val tsNow: Long = System.currentTimeMillis()
connectionMap.synchronized {
connectionMap.foreach {
x => {
if(x._2.refCount < 0) {
logError(s"Bug to be fixed: negative refCount of connection ${x._2}")
}
if(forceClean || ((x._2.refCount <= 0) && (tsNow - x._2.timestamp > timeout))) {
try{
x._2.connection.close()
} catch {
case e: IOException => logWarning(s"Fail to close connection ${x._2}", e)
}
connectionMap.remove(x._1)
}
}
}
}
}
// For testing purpose only
def getConnection(key: HBaseConnectionKey, conn: => Connection): SmartConnection = {
connectionMap.synchronized {
if (closed)
return null
cacheStat.numTotalRequests += 1
val sc = connectionMap.getOrElseUpdate(key, {cacheStat.numActualConnectionsCreated += 1
new SmartConnection(conn)})
sc.refCount += 1
sc
}
}
def getConnection(conf: Configuration): SmartConnection =
getConnection(new HBaseConnectionKey(conf), ConnectionFactory.createConnection(conf))
// For testing purpose only
def setTimeout(to: Long): Unit = {
connectionMap.synchronized {
if (closed)
return
timeout = to
housekeepingThread.interrupt()
}
}
}
private[hbase] case class SmartConnection (
connection: Connection, var refCount: Int = 0, var timestamp: Long = 0) {
def getTable(tableName: TableName): Table = connection.getTable(tableName)
def getRegionLocator(tableName: TableName): RegionLocator = connection.getRegionLocator(tableName)
def isClosed: Boolean = connection.isClosed
def getAdmin: Admin = connection.getAdmin
def close() = {
HBaseConnectionCache.connectionMap.synchronized {
refCount -= 1
if(refCount <= 0)
timestamp = System.currentTimeMillis()
}
}
}
/**
* Denotes a unique key to an HBase Connection instance.
* Please refer to 'org.apache.hadoop.hbase.client.HConnectionKey'.
*
* In essence, this class captures the properties in Configuration
* that may be used in the process of establishing a connection.
*
*/
class HBaseConnectionKey(c: Configuration) extends Logging {
val conf: Configuration = c
val CONNECTION_PROPERTIES: Array[String] = Array[String](
HConstants.ZOOKEEPER_QUORUM,
HConstants.ZOOKEEPER_ZNODE_PARENT,
HConstants.ZOOKEEPER_CLIENT_PORT,
HConstants.ZOOKEEPER_RECOVERABLE_WAITTIME,
HConstants.HBASE_CLIENT_PAUSE,
HConstants.HBASE_CLIENT_RETRIES_NUMBER,
HConstants.HBASE_RPC_TIMEOUT_KEY,
HConstants.HBASE_META_SCANNER_CACHING,
HConstants.HBASE_CLIENT_INSTANCE_ID,
HConstants.RPC_CODEC_CONF_KEY,
HConstants.USE_META_REPLICAS,
RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY)
var username: String = _
var m_properties = mutable.HashMap.empty[String, String]
if (conf != null) {
for (property <- CONNECTION_PROPERTIES) {
val value: String = conf.get(property)
if (value != null) {
m_properties.+=((property, value))
}
}
try {
val provider: UserProvider = UserProvider.instantiate(conf)
val currentUser: User = provider.getCurrent
if (currentUser != null) {
username = currentUser.getName
}
}
catch {
case e: IOException => {
logWarning("Error obtaining current user, skipping username in HBaseConnectionKey", e)
}
}
}
// make 'properties' immutable
val properties = m_properties.toMap
override def hashCode: Int = {
val prime: Int = 31
var result: Int = 1
if (username != null) {
result = username.hashCode
}
for (property <- CONNECTION_PROPERTIES) {
val value: Option[String] = properties.get(property)
if (value.isDefined) {
result = prime * result + value.hashCode
}
}
result
}
override def equals(obj: Any): Boolean = {
if (obj == null) return false
if (getClass ne obj.getClass) return false
val that: HBaseConnectionKey = obj.asInstanceOf[HBaseConnectionKey]
if (this.username != null && !(this.username == that.username)) {
return false
}
else if (this.username == null && that.username != null) {
return false
}
if (this.properties == null) {
if (that.properties != null) {
return false
}
}
else {
if (that.properties == null) {
return false
}
var flag: Boolean = true
for (property <- CONNECTION_PROPERTIES) {
val thisValue: Option[String] = this.properties.get(property)
val thatValue: Option[String] = that.properties.get(property)
flag = true
if (thisValue eq thatValue) {
flag = false //continue, so make flag be false
}
if (flag && (thisValue == null || !(thisValue == thatValue))) {
return false
}
}
}
true
}
override def toString: String = {
"HBaseConnectionKey{" + "properties=" + properties + ", username='" + username + '\'' + '}'
}
}
/**
* To log the state of 'HBaseConnectionCache'
*
* @param numTotalRequests number of total connection requests to the cache
* @param numActualConnectionsCreated number of actual HBase connections the cache ever created
* @param numActiveConnections number of current alive HBase connections the cache is holding
*/
case class HBaseConnectionCacheStat(var numTotalRequests: Long,
var numActualConnectionsCreated: Long,
var numActiveConnections: Long)

View File

@ -1,160 +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.hadoop.hbase.spark
import org.apache.hadoop.hbase.TableName
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client._
import org.apache.hadoop.hbase.io.ImmutableBytesWritable
import org.apache.spark.streaming.dstream.DStream
import scala.reflect.ClassTag
/**
* HBaseDStreamFunctions contains a set of implicit functions that can be
* applied to a Spark DStream so that we can easily interact with HBase
*/
@InterfaceAudience.Public
object HBaseDStreamFunctions {
/**
* These are implicit methods for a DStream that contains any type of
* data.
*
* @param dStream This is for dStreams of any type
* @tparam T Type T
*/
implicit class GenericHBaseDStreamFunctions[T](val dStream: DStream[T]) {
/**
* Implicit method that gives easy access to HBaseContext's bulk
* put. This will not return a new Stream. Think of it like a foreach
*
* @param hc The hbaseContext object to identify which
* HBase cluster connection to use
* @param tableName The tableName that the put will be sent to
* @param f The function that will turn the DStream values
* into HBase Put objects.
*/
def hbaseBulkPut(hc: HBaseContext,
tableName: TableName,
f: (T) => Put): Unit = {
hc.streamBulkPut(dStream, tableName, f)
}
/**
* Implicit method that gives easy access to HBaseContext's bulk
* get. This will return a new DStream. Think about it as a DStream map
* function. In that every DStream value will get a new value out of
* HBase. That new value will populate the newly generated DStream.
*
* @param hc The hbaseContext object to identify which
* HBase cluster connection to use
* @param tableName The tableName that the put will be sent to
* @param batchSize How many gets to execute in a single batch
* @param f The function that will turn the RDD values
* in HBase Get objects
* @param convertResult The function that will convert a HBase
* Result object into a value that will go
* into the resulting DStream
* @tparam R The type of Object that will be coming
* out of the resulting DStream
* @return A resulting DStream with type R objects
*/
def hbaseBulkGet[R: ClassTag](hc: HBaseContext,
tableName: TableName,
batchSize:Int, f: (T) => Get, convertResult: (Result) => R):
DStream[R] = {
hc.streamBulkGet[T, R](tableName, batchSize, dStream, f, convertResult)
}
/**
* Implicit method that gives easy access to HBaseContext's bulk
* get. This will return a new DStream. Think about it as a DStream map
* function. In that every DStream value will get a new value out of
* HBase. That new value will populate the newly generated DStream.
*
* @param hc The hbaseContext object to identify which
* HBase cluster connection to use
* @param tableName The tableName that the put will be sent to
* @param batchSize How many gets to execute in a single batch
* @param f The function that will turn the RDD values
* in HBase Get objects
* @return A resulting DStream with type R objects
*/
def hbaseBulkGet(hc: HBaseContext,
tableName: TableName, batchSize:Int,
f: (T) => Get): DStream[(ImmutableBytesWritable, Result)] = {
hc.streamBulkGet[T, (ImmutableBytesWritable, Result)](
tableName, batchSize, dStream, f,
result => (new ImmutableBytesWritable(result.getRow), result))
}
/**
* Implicit method that gives easy access to HBaseContext's bulk
* Delete. This will not return a new DStream.
*
* @param hc The hbaseContext object to identify which HBase
* cluster connection to use
* @param tableName The tableName that the deletes will be sent to
* @param f The function that will convert the DStream value into
* a HBase Delete Object
* @param batchSize The number of Deletes to be sent in a single batch
*/
def hbaseBulkDelete(hc: HBaseContext,
tableName: TableName,
f:(T) => Delete, batchSize:Int): Unit = {
hc.streamBulkDelete(dStream, tableName, f, batchSize)
}
/**
* Implicit method that gives easy access to HBaseContext's
* foreachPartition method. This will ack very much like a normal DStream
* foreach method but for the fact that you will now have a HBase connection
* while iterating through the values.
*
* @param hc The hbaseContext object to identify which HBase
* cluster connection to use
* @param f This function will get an iterator for a Partition of an
* DStream along with a connection object to HBase
*/
def hbaseForeachPartition(hc: HBaseContext,
f: (Iterator[T], Connection) => Unit): Unit = {
hc.streamForeachPartition(dStream, f)
}
/**
* Implicit method that gives easy access to HBaseContext's
* mapPartitions method. This will ask very much like a normal DStream
* map partitions method but for the fact that you will now have a
* HBase connection while iterating through the values
*
* @param hc The hbaseContext object to identify which HBase
* cluster connection to use
* @param f This function will get an iterator for a Partition of an
* DStream along with a connection object to HBase
* @tparam R This is the type of objects that will go into the resulting
* DStream
* @return A resulting DStream of type R
*/
def hbaseMapPartitions[R: ClassTag](hc: HBaseContext,
f: (Iterator[T], Connection) => Iterator[R]):
DStream[R] = {
hc.streamMapPartitions(dStream, f)
}
}
}

View File

@ -1,253 +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.hadoop.hbase.spark
import java.util
import org.apache.hadoop.hbase.{HConstants, TableName}
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client._
import org.apache.hadoop.hbase.io.ImmutableBytesWritable
import org.apache.spark.rdd.RDD
import scala.reflect.ClassTag
/**
* HBaseRDDFunctions contains a set of implicit functions that can be
* applied to a Spark RDD so that we can easily interact with HBase
*/
@InterfaceAudience.Public
object HBaseRDDFunctions
{
/**
* These are implicit methods for a RDD that contains any type of
* data.
*
* @param rdd This is for rdd of any type
* @tparam T This is any type
*/
implicit class GenericHBaseRDDFunctions[T](val rdd: RDD[T]) {
/**
* Implicit method that gives easy access to HBaseContext's bulk
* put. This will not return a new RDD. Think of it like a foreach
*
* @param hc The hbaseContext object to identify which
* HBase cluster connection to use
* @param tableName The tableName that the put will be sent to
* @param f The function that will turn the RDD values
* into HBase Put objects.
*/
def hbaseBulkPut(hc: HBaseContext,
tableName: TableName,
f: (T) => Put): Unit = {
hc.bulkPut(rdd, tableName, f)
}
/**
* Implicit method that gives easy access to HBaseContext's bulk
* get. This will return a new RDD. Think about it as a RDD map
* function. In that every RDD value will get a new value out of
* HBase. That new value will populate the newly generated RDD.
*
* @param hc The hbaseContext object to identify which
* HBase cluster connection to use
* @param tableName The tableName that the put will be sent to
* @param batchSize How many gets to execute in a single batch
* @param f The function that will turn the RDD values
* in HBase Get objects
* @param convertResult The function that will convert a HBase
* Result object into a value that will go
* into the resulting RDD
* @tparam R The type of Object that will be coming
* out of the resulting RDD
* @return A resulting RDD with type R objects
*/
def hbaseBulkGet[R: ClassTag](hc: HBaseContext,
tableName: TableName, batchSize:Int,
f: (T) => Get, convertResult: (Result) => R): RDD[R] = {
hc.bulkGet[T, R](tableName, batchSize, rdd, f, convertResult)
}
/**
* Implicit method that gives easy access to HBaseContext's bulk
* get. This will return a new RDD. Think about it as a RDD map
* function. In that every RDD value will get a new value out of
* HBase. That new value will populate the newly generated RDD.
*
* @param hc The hbaseContext object to identify which
* HBase cluster connection to use
* @param tableName The tableName that the put will be sent to
* @param batchSize How many gets to execute in a single batch
* @param f The function that will turn the RDD values
* in HBase Get objects
* @return A resulting RDD with type R objects
*/
def hbaseBulkGet(hc: HBaseContext,
tableName: TableName, batchSize:Int,
f: (T) => Get): RDD[(ImmutableBytesWritable, Result)] = {
hc.bulkGet[T, (ImmutableBytesWritable, Result)](tableName,
batchSize, rdd, f,
result => if (result != null && result.getRow != null) {
(new ImmutableBytesWritable(result.getRow), result)
} else {
null
})
}
/**
* Implicit method that gives easy access to HBaseContext's bulk
* Delete. This will not return a new RDD.
*
* @param hc The hbaseContext object to identify which HBase
* cluster connection to use
* @param tableName The tableName that the deletes will be sent to
* @param f The function that will convert the RDD value into
* a HBase Delete Object
* @param batchSize The number of Deletes to be sent in a single batch
*/
def hbaseBulkDelete(hc: HBaseContext,
tableName: TableName, f:(T) => Delete, batchSize:Int): Unit = {
hc.bulkDelete(rdd, tableName, f, batchSize)
}
/**
* Implicit method that gives easy access to HBaseContext's
* foreachPartition method. This will ack very much like a normal RDD
* foreach method but for the fact that you will now have a HBase connection
* while iterating through the values.
*
* @param hc The hbaseContext object to identify which HBase
* cluster connection to use
* @param f This function will get an iterator for a Partition of an
* RDD along with a connection object to HBase
*/
def hbaseForeachPartition(hc: HBaseContext,
f: (Iterator[T], Connection) => Unit): Unit = {
hc.foreachPartition(rdd, f)
}
/**
* Implicit method that gives easy access to HBaseContext's
* mapPartitions method. This will ask very much like a normal RDD
* map partitions method but for the fact that you will now have a
* HBase connection while iterating through the values
*
* @param hc The hbaseContext object to identify which HBase
* cluster connection to use
* @param f This function will get an iterator for a Partition of an
* RDD along with a connection object to HBase
* @tparam R This is the type of objects that will go into the resulting
* RDD
* @return A resulting RDD of type R
*/
def hbaseMapPartitions[R: ClassTag](hc: HBaseContext,
f: (Iterator[T], Connection) => Iterator[R]):
RDD[R] = {
hc.mapPartitions[T,R](rdd, f)
}
/**
* Spark Implementation of HBase Bulk load for wide rows or when
* values are not already combined at the time of the map process
*
* A Spark Implementation of HBase Bulk load
*
* This will take the content from an existing RDD then sort and shuffle
* it with respect to region splits. The result of that sort and shuffle
* will be written to HFiles.
*
* After this function is executed the user will have to call
* LoadIncrementalHFiles.doBulkLoad(...) to move the files into HBase
*
* Also note this version of bulk load is different from past versions in
* that it includes the qualifier as part of the sort process. The
* reason for this is to be able to support rows will very large number
* of columns.
*
* @param tableName The HBase table we are loading into
* @param flatMap A flapMap function that will make every row in the RDD
* into N cells for the bulk load
* @param stagingDir The location on the FileSystem to bulk load into
* @param familyHFileWriteOptionsMap Options that will define how the HFile for a
* column family is written
* @param compactionExclude Compaction excluded for the HFiles
* @param maxSize Max size for the HFiles before they roll
*/
def hbaseBulkLoad(hc: HBaseContext,
tableName: TableName,
flatMap: (T) => Iterator[(KeyFamilyQualifier, Array[Byte])],
stagingDir:String,
familyHFileWriteOptionsMap:
util.Map[Array[Byte], FamilyHFileWriteOptions] =
new util.HashMap[Array[Byte], FamilyHFileWriteOptions](),
compactionExclude: Boolean = false,
maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE):Unit = {
hc.bulkLoad(rdd, tableName,
flatMap, stagingDir, familyHFileWriteOptionsMap,
compactionExclude, maxSize)
}
/**
* Implicit method that gives easy access to HBaseContext's
* bulkLoadThinRows method.
*
* Spark Implementation of HBase Bulk load for short rows some where less then
* a 1000 columns. This bulk load should be faster for tables will thinner
* rows then the other spark implementation of bulk load that puts only one
* value into a record going into a shuffle
*
* This will take the content from an existing RDD then sort and shuffle
* it with respect to region splits. The result of that sort and shuffle
* will be written to HFiles.
*
* After this function is executed the user will have to call
* LoadIncrementalHFiles.doBulkLoad(...) to move the files into HBase
*
* In this implementation only the rowKey is given to the shuffle as the key
* and all the columns are already linked to the RowKey before the shuffle
* stage. The sorting of the qualifier is done in memory out side of the
* shuffle stage
*
* @param tableName The HBase table we are loading into
* @param mapFunction A function that will convert the RDD records to
* the key value format used for the shuffle to prep
* for writing to the bulk loaded HFiles
* @param stagingDir The location on the FileSystem to bulk load into
* @param familyHFileWriteOptionsMap Options that will define how the HFile for a
* column family is written
* @param compactionExclude Compaction excluded for the HFiles
* @param maxSize Max size for the HFiles before they roll
*/
def hbaseBulkLoadThinRows(hc: HBaseContext,
tableName: TableName,
mapFunction: (T) =>
(ByteArrayWrapper, FamiliesQualifiersValues),
stagingDir:String,
familyHFileWriteOptionsMap:
util.Map[Array[Byte], FamilyHFileWriteOptions] =
new util.HashMap[Array[Byte], FamilyHFileWriteOptions](),
compactionExclude: Boolean = false,
maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE):Unit = {
hc.bulkLoadThinRows(rdd, tableName,
mapFunction, stagingDir, familyHFileWriteOptionsMap,
compactionExclude, maxSize)
}
}
}

View File

@ -1,408 +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.hadoop.hbase.spark
import java.util.Map
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.hbase.TableName
import org.apache.hadoop.hbase.util.Pair
import org.apache.yetus.audience.InterfaceAudience
import org.apache.hadoop.hbase.client.{Connection, Delete, Get, Put, Result, Scan}
import org.apache.hadoop.hbase.io.ImmutableBytesWritable
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
import org.apache.spark.api.java.function.{FlatMapFunction, Function, VoidFunction}
import org.apache.spark.streaming.api.java.JavaDStream
import scala.collection.JavaConversions._
import scala.reflect.ClassTag
/**
* This is the Java Wrapper over HBaseContext which is written in
* Scala. This class will be used by developers that want to
* work with Spark or Spark Streaming in Java
*
* @param jsc This is the JavaSparkContext that we will wrap
* @param config This is the config information to out HBase cluster
*/
@InterfaceAudience.Public
class JavaHBaseContext(@transient jsc: JavaSparkContext,
@transient config: Configuration) extends Serializable {
val hbaseContext = new HBaseContext(jsc.sc, config)
/**
* A simple enrichment of the traditional Spark javaRdd foreachPartition.
* This function differs from the original in that it offers the
* developer access to a already connected Connection object
*
* Note: Do not close the Connection object. All Connection
* management is handled outside this method
*
* @param javaRdd Original javaRdd with data to iterate over
* @param f Function to be given a iterator to iterate through
* the RDD values and a Connection object to interact
* with HBase
*/
def foreachPartition[T](javaRdd: JavaRDD[T],
f: VoidFunction[(java.util.Iterator[T], Connection)]) = {
hbaseContext.foreachPartition(javaRdd.rdd,
(it: Iterator[T], conn: Connection) => {
f.call((it, conn))
})
}
/**
* 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 Connection object
*
* Note: Do not close the Connection object. All Connection
* management is handled outside this method
*
* @param javaDstream Original DStream with data to iterate over
* @param f Function to be given a iterator to iterate through
* the JavaDStream values and a Connection object to
* interact with HBase
*/
def foreachPartition[T](javaDstream: JavaDStream[T],
f: VoidFunction[(Iterator[T], Connection)]) = {
hbaseContext.foreachPartition(javaDstream.dstream,
(it: Iterator[T], conn: Connection) => f.call(it, conn))
}
/**
* A simple enrichment of the traditional Spark JavaRDD mapPartition.
* This function differs from the original in that it offers the
* developer access to a already connected Connection object
*
* Note: Do not close the Connection object. All Connection
* management is handled outside this method
*
* Note: Make sure to partition correctly to avoid memory issue when
* getting data from HBase
*
* @param javaRdd Original JavaRdd with data to iterate over
* @param f Function to be given a iterator to iterate through
* the RDD values and a Connection object to interact
* with HBase
* @return Returns a new RDD generated by the user definition
* function just like normal mapPartition
*/
def mapPartitions[T, R](javaRdd: JavaRDD[T],
f: FlatMapFunction[(java.util.Iterator[T],
Connection), R]): JavaRDD[R] = {
def fn = (it: Iterator[T], conn: Connection) =>
asScalaIterator(
f.call((asJavaIterator(it), conn)).iterator()
)
JavaRDD.fromRDD(hbaseContext.mapPartitions(javaRdd.rdd,
(iterator: Iterator[T], connection: Connection) =>
fn(iterator, connection))(fakeClassTag[R]))(fakeClassTag[R])
}
/**
* A simple enrichment of the traditional Spark Streaming JavaDStream
* mapPartition.
*
* This function differs from the original in that it offers the
* developer access to a already connected Connection object
*
* Note: Do not close the Connection object. All Connection
* management is handled outside this method
*
* Note: Make sure to partition correctly to avoid memory issue when
* getting data from HBase
*
* @param javaDstream Original JavaDStream with data to iterate over
* @param mp Function to be given a iterator to iterate through
* the JavaDStream values and a Connection object to
* interact with HBase
* @return Returns a new JavaDStream generated by the user
* definition function just like normal mapPartition
*/
def streamMap[T, U](javaDstream: JavaDStream[T],
mp: Function[(Iterator[T], Connection), Iterator[U]]):
JavaDStream[U] = {
JavaDStream.fromDStream(hbaseContext.streamMapPartitions(javaDstream.dstream,
(it: Iterator[T], conn: Connection) =>
mp.call(it, conn))(fakeClassTag[U]))(fakeClassTag[U])
}
/**
* A simple abstraction over the HBaseContext.foreachPartition method.
*
* It allow addition support for a user to take JavaRDD
* and generate puts and send them to HBase.
* The complexity of managing the Connection is
* removed from the developer
*
* @param javaRdd Original JavaRDD with data to iterate over
* @param tableName The name of the table to put into
* @param f Function to convert a value in the JavaRDD
* to a HBase Put
*/
def bulkPut[T](javaRdd: JavaRDD[T],
tableName: TableName,
f: Function[(T), Put]) {
hbaseContext.bulkPut(javaRdd.rdd, tableName, (t: T) => f.call(t))
}
/**
* A simple abstraction over the HBaseContext.streamMapPartition method.
*
* It allow addition support for a user to take a JavaDStream and
* generate puts and send them to HBase.
*
* The complexity of managing the Connection is
* removed from the developer
*
* @param javaDstream Original DStream with data to iterate over
* @param tableName The name of the table to put into
* @param f Function to convert a value in
* the JavaDStream to a HBase Put
*/
def streamBulkPut[T](javaDstream: JavaDStream[T],
tableName: TableName,
f: Function[T, Put]) = {
hbaseContext.streamBulkPut(javaDstream.dstream,
tableName,
(t: T) => f.call(t))
}
/**
* A simple abstraction over the HBaseContext.foreachPartition method.
*
* It allow addition support for a user to take a JavaRDD and
* generate delete and send them to HBase.
*
* The complexity of managing the Connection is
* removed from the developer
*
* @param javaRdd Original JavaRDD with data to iterate over
* @param tableName The name of the table to delete from
* @param f Function to convert a value in the JavaRDD to a
* HBase Deletes
* @param batchSize The number of deletes to batch before sending to HBase
*/
def bulkDelete[T](javaRdd: JavaRDD[T], tableName: TableName,
f: Function[T, Delete], batchSize: Integer) {
hbaseContext.bulkDelete(javaRdd.rdd, tableName, (t: T) => f.call(t), batchSize)
}
/**
* A simple abstraction over the HBaseContext.streamBulkMutation method.
*
* It allow addition support for a user to take a JavaDStream and
* generate Delete and send them to HBase.
*
* The complexity of managing the Connection is
* removed from the developer
*
* @param javaDStream Original DStream with data to iterate over
* @param tableName The name of the table to delete from
* @param f Function to convert a value in the JavaDStream to a
* HBase Delete
* @param batchSize The number of deletes to be sent at once
*/
def streamBulkDelete[T](javaDStream: JavaDStream[T],
tableName: TableName,
f: Function[T, Delete],
batchSize: Integer) = {
hbaseContext.streamBulkDelete(javaDStream.dstream, tableName,
(t: T) => f.call(t),
batchSize)
}
/**
* A simple abstraction over the HBaseContext.mapPartition method.
*
* It allow addition support for a user to take a JavaRDD and generates a
* new RDD based on Gets and the results they bring back from HBase
*
* @param tableName The name of the table to get from
* @param batchSize batch size of how many gets to retrieve in a single fetch
* @param javaRdd Original JavaRDD with data to iterate over
* @param makeGet Function to convert a value in the JavaRDD to a
* HBase Get
* @param convertResult This will convert the HBase Result object to
* what ever the user wants to put in the resulting
* JavaRDD
* @return New JavaRDD that is created by the Get to HBase
*/
def bulkGet[T, U](tableName: TableName,
batchSize: Integer,
javaRdd: JavaRDD[T],
makeGet: Function[T, Get],
convertResult: Function[Result, U]): JavaRDD[U] = {
JavaRDD.fromRDD(hbaseContext.bulkGet[T, U](tableName,
batchSize,
javaRdd.rdd,
(t: T) => makeGet.call(t),
(r: Result) => {
convertResult.call(r)
})(fakeClassTag[U]))(fakeClassTag[U])
}
/**
* A simple abstraction over the HBaseContext.streamMap method.
*
* It allow addition support for a user to take a DStream and
* generates a new DStream based on Gets and the results
* they bring back from HBase
*
* @param tableName The name of the table to get from
* @param batchSize The number of gets to be batched together
* @param javaDStream Original DStream with data to iterate over
* @param makeGet Function to convert a value in the JavaDStream to a
* HBase Get
* @param convertResult This will convert the HBase Result object to
* what ever the user wants to put in the resulting
* JavaDStream
* @return New JavaDStream that is created by the Get to HBase
*/
def streamBulkGet[T, U](tableName: TableName,
batchSize: Integer,
javaDStream: JavaDStream[T],
makeGet: Function[T, Get],
convertResult: Function[Result, U]): JavaDStream[U] = {
JavaDStream.fromDStream(hbaseContext.streamBulkGet(tableName,
batchSize,
javaDStream.dstream,
(t: T) => makeGet.call(t),
(r: Result) => convertResult.call(r))(fakeClassTag[U]))(fakeClassTag[U])
}
/**
* A simple abstraction over the HBaseContext.bulkLoad method.
* It allow addition support for a user to take a JavaRDD and
* convert into new JavaRDD[Pair] based on MapFunction,
* and HFiles will be generated in stagingDir for bulk load
*
* @param javaRdd The javaRDD we are bulk loading from
* @param tableName The HBase table we are loading into
* @param mapFunc A Function that will convert a value in JavaRDD
* to Pair(KeyFamilyQualifier, Array[Byte])
* @param stagingDir The location on the FileSystem to bulk load into
* @param familyHFileWriteOptionsMap Options that will define how the HFile for a
* column family is written
* @param compactionExclude Compaction excluded for the HFiles
* @param maxSize Max size for the HFiles before they roll
*/
def bulkLoad[T](javaRdd: JavaRDD[T],
tableName: TableName,
mapFunc : Function[T, Pair[KeyFamilyQualifier, Array[Byte]]],
stagingDir: String,
familyHFileWriteOptionsMap: Map[Array[Byte], FamilyHFileWriteOptions],
compactionExclude: Boolean,
maxSize: Long):
Unit = {
hbaseContext.bulkLoad[Pair[KeyFamilyQualifier, Array[Byte]]](javaRdd.map(mapFunc).rdd, tableName, t => {
val keyFamilyQualifier = t.getFirst
val value = t.getSecond
Seq((keyFamilyQualifier, value)).iterator
}, stagingDir, familyHFileWriteOptionsMap, compactionExclude, maxSize)
}
/**
* A simple abstraction over the HBaseContext.bulkLoadThinRows method.
* It allow addition support for a user to take a JavaRDD and
* convert into new JavaRDD[Pair] based on MapFunction,
* and HFiles will be generated in stagingDir for bulk load
*
* @param javaRdd The javaRDD we are bulk loading from
* @param tableName The HBase table we are loading into
* @param mapFunc A Function that will convert a value in JavaRDD
* to Pair(ByteArrayWrapper, FamiliesQualifiersValues)
* @param stagingDir The location on the FileSystem to bulk load into
* @param familyHFileWriteOptionsMap Options that will define how the HFile for a
* column family is written
* @param compactionExclude Compaction excluded for the HFiles
* @param maxSize Max size for the HFiles before they roll
*/
def bulkLoadThinRows[T](javaRdd: JavaRDD[T],
tableName: TableName,
mapFunc : Function[T, Pair[ByteArrayWrapper, FamiliesQualifiersValues]],
stagingDir: String,
familyHFileWriteOptionsMap: Map[Array[Byte], FamilyHFileWriteOptions],
compactionExclude: Boolean,
maxSize: Long):
Unit = {
hbaseContext.bulkLoadThinRows[Pair[ByteArrayWrapper, FamiliesQualifiersValues]](javaRdd.map(mapFunc).rdd,
tableName, t => {
(t.getFirst, t.getSecond)
}, stagingDir, familyHFileWriteOptionsMap, compactionExclude, maxSize)
}
/**
* This function will use the native HBase TableInputFormat with the
* given scan object to generate a new JavaRDD
*
* @param tableName The name of the table to scan
* @param scans The HBase scan object to use to read data from HBase
* @param f Function to convert a Result object from HBase into
* What the user wants in the final generated JavaRDD
* @return New JavaRDD with results from scan
*/
def hbaseRDD[U](tableName: TableName,
scans: Scan,
f: Function[(ImmutableBytesWritable, Result), U]):
JavaRDD[U] = {
JavaRDD.fromRDD(
hbaseContext.hbaseRDD[U](tableName,
scans,
(v: (ImmutableBytesWritable, Result)) =>
f.call(v._1, v._2))(fakeClassTag[U]))(fakeClassTag[U])
}
/**
* A overloaded version of HBaseContext hbaseRDD that define the
* type of the resulting JavaRDD
*
* @param tableName The name of the table to scan
* @param scans The HBase scan object to use to read data from HBase
* @return New JavaRDD with results from scan
*/
def hbaseRDD(tableName: TableName,
scans: Scan):
JavaRDD[(ImmutableBytesWritable, Result)] = {
JavaRDD.fromRDD(hbaseContext.hbaseRDD(tableName, scans))
}
/**
* Produces a ClassTag[T], which is actually just a casted ClassTag[AnyRef].
*
* This method is used to keep ClassTags out of the external Java API, as the Java compiler
* cannot produce them automatically. While this ClassTag-faking does please the compiler,
* it can cause problems at runtime if the Scala API relies on ClassTags for correctness.
*
* Often, though, a ClassTag[AnyRef] will not lead to incorrect behavior,
* just worse performance or security issues.
* For instance, an Array[AnyRef] can hold any type T,
* but may lose primitive
* specialization.
*/
private[spark]
def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]]
}

View File

@ -1,48 +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.hadoop.hbase.spark
import java.io.Serializable
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes
/**
* This is the key to be used for sorting and shuffling.
*
* We will only partition on the rowKey but we will sort on all three
*
* @param rowKey Record RowKey
* @param family Record ColumnFamily
* @param qualifier Cell Qualifier
*/
@InterfaceAudience.Public
class KeyFamilyQualifier(val rowKey:Array[Byte], val family:Array[Byte], val qualifier:Array[Byte])
extends Comparable[KeyFamilyQualifier] with Serializable {
override def compareTo(o: KeyFamilyQualifier): Int = {
var result = Bytes.compareTo(rowKey, o.rowKey)
if (result == 0) {
result = Bytes.compareTo(family, o.family)
if (result == 0) result = Bytes.compareTo(qualifier, o.qualifier)
}
result
}
override def toString: String = {
Bytes.toString(rowKey) + ":" + Bytes.toString(family) + ":" + Bytes.toString(qualifier)
}
}

View File

@ -1,38 +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.hadoop.hbase.spark
import org.apache.hadoop.conf.Configuration
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.mapreduce.InputFormat
import org.apache.spark.rdd.NewHadoopRDD
import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskContext}
@InterfaceAudience.Public
class NewHBaseRDD[K,V](@transient sc : SparkContext,
@transient inputFormatClass: Class[_ <: InputFormat[K, V]],
@transient keyClass: Class[K],
@transient valueClass: Class[V],
@transient conf: Configuration,
val hBaseContext: HBaseContext) extends NewHadoopRDD(sc,inputFormatClass, keyClass, valueClass, conf) {
override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = {
hBaseContext.applyCreds()
super.compute(theSplit, context)
}
}

View File

@ -1,121 +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.hadoop.hbase.spark.datasources
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.spark.hbase._
/**
* The Bound represent the boudary for the scan
*
* @param b The byte array of the bound
* @param inc inclusive or not.
*/
@InterfaceAudience.Private
case class Bound(b: Array[Byte], inc: Boolean)
// The non-overlapping ranges we need to scan, if lower is equal to upper, it is a get request
@InterfaceAudience.Private
case class Range(lower: Option[Bound], upper: Option[Bound])
@InterfaceAudience.Private
object Range {
def apply(region: HBaseRegion): Range = {
Range(region.start.map(Bound(_, true)), if (region.end.get.size == 0) {
None
} else {
region.end.map((Bound(_, false)))
})
}
}
@InterfaceAudience.Private
object Ranges {
// We assume that
// 1. r.lower.inc is true, and r.upper.inc is false
// 2. for each range in rs, its upper.inc is false
def and(r: Range, rs: Seq[Range]): Seq[Range] = {
rs.flatMap{ s =>
val lower = s.lower.map { x =>
// the scan has lower bound
r.lower.map { y =>
// the region has lower bound
if (ord.compare(x.b, y.b) < 0) {
// scan lower bound is smaller than region server lower bound
Some(y)
} else {
// scan low bound is greater or equal to region server lower bound
Some(x)
}
}.getOrElse(Some(x))
}.getOrElse(r.lower)
val upper = s.upper.map { x =>
// the scan has upper bound
r.upper.map { y =>
// the region has upper bound
if (ord.compare(x.b, y.b) >= 0) {
// scan upper bound is larger than server upper bound
// but region server scan stop is exclusive. It is OK here.
Some(y)
} else {
// scan upper bound is less or equal to region server upper bound
Some(x)
}
}.getOrElse(Some(x))
}.getOrElse(r.upper)
val c = lower.map { case x =>
upper.map { case y =>
ord.compare(x.b, y.b)
}.getOrElse(-1)
}.getOrElse(-1)
if (c < 0) {
Some(Range(lower, upper))
} else {
None
}
}.seq
}
}
@InterfaceAudience.Private
object Points {
def and(r: Range, ps: Seq[Array[Byte]]): Seq[Array[Byte]] = {
ps.flatMap { p =>
if (ord.compare(r.lower.get.b, p) <= 0) {
// if region lower bound is less or equal to the point
if (r.upper.isDefined) {
// if region upper bound is defined
if (ord.compare(r.upper.get.b, p) > 0) {
// if the upper bound is greater than the point (because upper bound is exclusive)
Some(p)
} else {
None
}
} else {
// if the region upper bound is not defined (infinity)
Some(p)
}
} else {
None
}
}
}
}

View File

@ -1,171 +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.hadoop.hbase.spark.datasources
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.TableName
import org.apache.hadoop.hbase.client._
import org.apache.hadoop.hbase.spark.{HBaseConnectionKey, SmartConnection,
HBaseConnectionCache, HBaseRelation}
import scala.language.implicitConversions
// Resource and ReferencedResources are defined for extensibility,
// e.g., consolidate scan and bulkGet in the future work.
// User has to invoke release explicitly to release the resource,
// and potentially parent resources
@InterfaceAudience.Private
trait Resource {
def release(): Unit
}
@InterfaceAudience.Private
case class ScanResource(tbr: TableResource, rs: ResultScanner) extends Resource {
def release() {
rs.close()
tbr.release()
}
}
@InterfaceAudience.Private
case class GetResource(tbr: TableResource, rs: Array[Result]) extends Resource {
def release() {
tbr.release()
}
}
@InterfaceAudience.Private
trait ReferencedResource {
var count: Int = 0
def init(): Unit
def destroy(): Unit
def acquire() = synchronized {
try {
count += 1
if (count == 1) {
init()
}
} catch {
case e: Throwable =>
release()
throw e
}
}
def release() = synchronized {
count -= 1
if (count == 0) {
destroy()
}
}
def releaseOnException[T](func: => T): T = {
acquire()
val ret = {
try {
func
} catch {
case e: Throwable =>
release()
throw e
}
}
ret
}
}
@InterfaceAudience.Private
case class TableResource(relation: HBaseRelation) extends ReferencedResource {
var connection: SmartConnection = _
var table: Table = _
override def init(): Unit = {
connection = HBaseConnectionCache.getConnection(relation.hbaseConf)
table = connection.getTable(TableName.valueOf(relation.tableName))
}
override def destroy(): Unit = {
if (table != null) {
table.close()
table = null
}
if (connection != null) {
connection.close()
connection = null
}
}
def getScanner(scan: Scan): ScanResource = releaseOnException {
ScanResource(this, table.getScanner(scan))
}
def get(list: java.util.List[org.apache.hadoop.hbase.client.Get]) = releaseOnException {
GetResource(this, table.get(list))
}
}
@InterfaceAudience.Private
case class RegionResource(relation: HBaseRelation) extends ReferencedResource {
var connection: SmartConnection = _
var rl: RegionLocator = _
val regions = releaseOnException {
val keys = rl.getStartEndKeys
keys.getFirst.zip(keys.getSecond)
.zipWithIndex
.map(x =>
HBaseRegion(x._2,
Some(x._1._1),
Some(x._1._2),
Some(rl.getRegionLocation(x._1._1).getHostname)))
}
override def init(): Unit = {
connection = HBaseConnectionCache.getConnection(relation.hbaseConf)
rl = connection.getRegionLocator(TableName.valueOf(relation.tableName))
}
override def destroy(): Unit = {
if (rl != null) {
rl.close()
rl = null
}
if (connection != null) {
connection.close()
connection = null
}
}
}
@InterfaceAudience.Private
object HBaseResources{
implicit def ScanResToScan(sr: ScanResource): ResultScanner = {
sr.rs
}
implicit def GetResToResult(gr: GetResource): Array[Result] = {
gr.rs
}
implicit def TableResToTable(tr: TableResource): Table = {
tr.table
}
implicit def RegionResToRegions(rr: RegionResource): Seq[HBaseRegion] = {
rr.regions
}
}

View File

@ -1,62 +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.hadoop.hbase.spark.datasources
import org.apache.yetus.audience.InterfaceAudience;
/**
* This is the hbase configuration. User can either set them in SparkConf, which
* will take effect globally, or configure it per table, which will overwrite the value
* set in SparkConf. If not set, the default value will take effect.
*/
@InterfaceAudience.Public
object HBaseSparkConf{
/** Set to false to disable server-side caching of blocks for this scan,
* false by default, since full table scans generate too much BC churn.
*/
val QUERY_CACHEBLOCKS = "hbase.spark.query.cacheblocks"
val DEFAULT_QUERY_CACHEBLOCKS = false
/** The number of rows for caching that will be passed to scan. */
val QUERY_CACHEDROWS = "hbase.spark.query.cachedrows"
/** Set the maximum number of values to return for each call to next() in scan. */
val QUERY_BATCHSIZE = "hbase.spark.query.batchsize"
/** The number of BulkGets send to HBase. */
val BULKGET_SIZE = "hbase.spark.bulkget.size"
val DEFAULT_BULKGET_SIZE = 1000
/** Set to specify the location of hbase configuration file. */
val HBASE_CONFIG_LOCATION = "hbase.spark.config.location"
/** Set to specify whether create or use latest cached HBaseContext*/
val USE_HBASECONTEXT = "hbase.spark.use.hbasecontext"
val DEFAULT_USE_HBASECONTEXT = true
/** Pushdown the filter to data source engine to increase the performance of queries. */
val PUSHDOWN_COLUMN_FILTER = "hbase.spark.pushdown.columnfilter"
val DEFAULT_PUSHDOWN_COLUMN_FILTER= true
/** Class name of the encoder, which encode data types from Spark to HBase bytes. */
val QUERY_ENCODER = "hbase.spark.query.encoder"
val DEFAULT_QUERY_ENCODER = classOf[NaiveEncoder].getCanonicalName
/** The timestamp used to filter columns with a specific timestamp. */
val TIMESTAMP = "hbase.spark.query.timestamp"
/** The starting timestamp used to filter columns with a specific range of versions. */
val TIMERANGE_START = "hbase.spark.query.timerange.start"
/** The ending timestamp used to filter columns with a specific range of versions. */
val TIMERANGE_END = "hbase.spark.query.timerange.end"
/** The maximum number of version to return. */
val MAX_VERSIONS = "hbase.spark.query.maxVersions"
/** Delayed time to close hbase-spark connection when no reference to this connection, in milliseconds. */
val DEFAULT_CONNECTION_CLOSE_DELAY = 10 * 60 * 1000
}

View File

@ -1,308 +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.hadoop.hbase.spark.datasources
import java.util.ArrayList
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client._
import org.apache.hadoop.hbase.spark._
import org.apache.hadoop.hbase.spark.hbase._
import org.apache.hadoop.hbase.spark.datasources.HBaseResources._
import org.apache.hadoop.hbase.util.ShutdownHookManager
import org.apache.spark.sql.datasources.hbase.Field
import org.apache.spark.{SparkEnv, TaskContext, Logging, Partition}
import org.apache.spark.rdd.RDD
import scala.collection.mutable
@InterfaceAudience.Private
class HBaseTableScanRDD(relation: HBaseRelation,
val hbaseContext: HBaseContext,
@transient val filter: Option[SparkSQLPushDownFilter] = None,
val columns: Seq[Field] = Seq.empty
)extends RDD[Result](relation.sqlContext.sparkContext, Nil) with Logging {
private def sparkConf = SparkEnv.get.conf
@transient var ranges = Seq.empty[Range]
@transient var points = Seq.empty[Array[Byte]]
def addPoint(p: Array[Byte]) {
points :+= p
}
def addRange(r: ScanRange) = {
val lower = if (r.lowerBound != null && r.lowerBound.length > 0) {
Some(Bound(r.lowerBound, r.isLowerBoundEqualTo))
} else {
None
}
val upper = if (r.upperBound != null && r.upperBound.length > 0) {
if (!r.isUpperBoundEqualTo) {
Some(Bound(r.upperBound, false))
} else {
// HBase stopRow is exclusive: therefore it DOESN'T act like isUpperBoundEqualTo
// by default. So we need to add a new max byte to the stopRow key
val newArray = new Array[Byte](r.upperBound.length + 1)
System.arraycopy(r.upperBound, 0, newArray, 0, r.upperBound.length)
//New Max Bytes
newArray(r.upperBound.length) = ByteMin
Some(Bound(newArray, false))
}
} else {
None
}
ranges :+= Range(lower, upper)
}
override def getPartitions: Array[Partition] = {
val regions = RegionResource(relation)
var idx = 0
logDebug(s"There are ${regions.size} regions")
val ps = regions.flatMap { x =>
val rs = Ranges.and(Range(x), ranges)
val ps = Points.and(Range(x), points)
if (rs.size > 0 || ps.size > 0) {
if(log.isDebugEnabled) {
rs.foreach(x => logDebug(x.toString))
}
idx += 1
Some(HBaseScanPartition(idx - 1, x, rs, ps, SerializedFilter.toSerializedTypedFilter(filter)))
} else {
None
}
}.toArray
regions.release()
ShutdownHookManager.affixShutdownHook( new Thread() {
override def run() {
HBaseConnectionCache.close()
}
}, 0)
ps.asInstanceOf[Array[Partition]]
}
override def getPreferredLocations(split: Partition): Seq[String] = {
split.asInstanceOf[HBaseScanPartition].regions.server.map {
identity
}.toSeq
}
private def buildGets(
tbr: TableResource,
g: Seq[Array[Byte]],
filter: Option[SparkSQLPushDownFilter],
columns: Seq[Field],
hbaseContext: HBaseContext): Iterator[Result] = {
g.grouped(relation.bulkGetSize).flatMap{ x =>
val gets = new ArrayList[Get](x.size)
x.foreach{ y =>
val g = new Get(y)
handleTimeSemantics(g)
columns.foreach { d =>
if (!d.isRowKey) {
g.addColumn(d.cfBytes, d.colBytes)
}
}
filter.foreach(g.setFilter(_))
gets.add(g)
}
hbaseContext.applyCreds()
val tmp = tbr.get(gets)
rddResources.addResource(tmp)
toResultIterator(tmp)
}
}
private def toResultIterator(result: GetResource): Iterator[Result] = {
val iterator = new Iterator[Result] {
var idx = 0
var cur: Option[Result] = None
override def hasNext: Boolean = {
while(idx < result.length && cur.isEmpty) {
val r = result(idx)
idx += 1
if (!r.isEmpty) {
cur = Some(r)
}
}
if (cur.isEmpty) {
rddResources.release(result)
}
cur.isDefined
}
override def next(): Result = {
hasNext
val ret = cur.get
cur = None
ret
}
}
iterator
}
private def buildScan(range: Range,
filter: Option[SparkSQLPushDownFilter],
columns: Seq[Field]): Scan = {
val scan = (range.lower, range.upper) match {
case (Some(Bound(a, b)), Some(Bound(c, d))) => new Scan(a, c)
case (None, Some(Bound(c, d))) => new Scan(Array[Byte](), c)
case (Some(Bound(a, b)), None) => new Scan(a)
case (None, None) => new Scan()
}
handleTimeSemantics(scan)
columns.foreach { d =>
if (!d.isRowKey) {
scan.addColumn(d.cfBytes, d.colBytes)
}
}
scan.setCacheBlocks(relation.blockCacheEnable)
scan.setBatch(relation.batchNum)
scan.setCaching(relation.cacheSize)
filter.foreach(scan.setFilter(_))
scan
}
private def toResultIterator(scanner: ScanResource): Iterator[Result] = {
val iterator = new Iterator[Result] {
var cur: Option[Result] = None
override def hasNext: Boolean = {
if (cur.isEmpty) {
val r = scanner.next()
if (r == null) {
rddResources.release(scanner)
} else {
cur = Some(r)
}
}
cur.isDefined
}
override def next(): Result = {
hasNext
val ret = cur.get
cur = None
ret
}
}
iterator
}
lazy val rddResources = RDDResources(new mutable.HashSet[Resource]())
private def close() {
rddResources.release()
}
override def compute(split: Partition, context: TaskContext): Iterator[Result] = {
val partition = split.asInstanceOf[HBaseScanPartition]
val filter = SerializedFilter.fromSerializedFilter(partition.sf)
val scans = partition.scanRanges
.map(buildScan(_, filter, columns))
val tableResource = TableResource(relation)
context.addTaskCompletionListener(context => close())
val points = partition.points
val gIt: Iterator[Result] = {
if (points.isEmpty) {
Iterator.empty: Iterator[Result]
} else {
buildGets(tableResource, points, filter, columns, hbaseContext)
}
}
val rIts = scans.par
.map { scan =>
hbaseContext.applyCreds()
val scanner = tableResource.getScanner(scan)
rddResources.addResource(scanner)
scanner
}.map(toResultIterator(_))
.fold(Iterator.empty: Iterator[Result]){ case (x, y) =>
x ++ y
} ++ gIt
ShutdownHookManager.affixShutdownHook( new Thread() {
override def run() {
HBaseConnectionCache.close()
}
}, 0)
rIts
}
private def handleTimeSemantics(query: Query): Unit = {
// Set timestamp related values if present
(query, relation.timestamp, relation.minTimestamp, relation.maxTimestamp) match {
case (q: Scan, Some(ts), None, None) => q.setTimeStamp(ts)
case (q: Get, Some(ts), None, None) => q.setTimeStamp(ts)
case (q:Scan, None, Some(minStamp), Some(maxStamp)) => q.setTimeRange(minStamp, maxStamp)
case (q:Get, None, Some(minStamp), Some(maxStamp)) => q.setTimeRange(minStamp, maxStamp)
case (q, None, None, None) =>
case _ => throw new IllegalArgumentException(s"Invalid combination of query/timestamp/time range provided. " +
s"timeStamp is: ${relation.timestamp.get}, minTimeStamp is: ${relation.minTimestamp.get}, " +
s"maxTimeStamp is: ${relation.maxTimestamp.get}")
}
if (relation.maxVersions.isDefined) {
query match {
case q: Scan => q.setMaxVersions(relation.maxVersions.get)
case q: Get => q.setMaxVersions(relation.maxVersions.get)
case _ => throw new IllegalArgumentException("Invalid query provided with maxVersions")
}
}
}
}
case class SerializedFilter(b: Option[Array[Byte]])
object SerializedFilter {
def toSerializedTypedFilter(f: Option[SparkSQLPushDownFilter]): SerializedFilter = {
SerializedFilter(f.map(_.toByteArray))
}
def fromSerializedFilter(sf: SerializedFilter): Option[SparkSQLPushDownFilter] = {
sf.b.map(SparkSQLPushDownFilter.parseFrom(_))
}
}
private[hbase] case class HBaseRegion(
override val index: Int,
val start: Option[HBaseType] = None,
val end: Option[HBaseType] = None,
val server: Option[String] = None) extends Partition
private[hbase] case class HBaseScanPartition(
override val index: Int,
val regions: HBaseRegion,
val scanRanges: Seq[Range],
val points: Seq[Array[Byte]],
val sf: SerializedFilter) extends Partition
case class RDDResources(set: mutable.HashSet[Resource]) {
def addResource(s: Resource) {
set += s
}
def release() {
set.foreach(release(_))
}
def release(rs: Resource) {
try {
rs.release()
} finally {
set.remove(rs)
}
}
}

View File

@ -1,116 +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.hadoop.hbase.spark.datasources
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder.JavaBytesEncoder
import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.Logging
import org.apache.spark.sql.types._
/**
* The ranges for the data type whose size is known. Whether the bound is inclusive
* or exclusive is undefind, and upper to the caller to decide.
*
* @param low: the lower bound of the range.
* @param upper: the upper bound of the range.
*/
@InterfaceAudience.LimitedPrivate(Array(HBaseInterfaceAudience.SPARK))
@InterfaceStability.Evolving
case class BoundRange(low: Array[Byte],upper: Array[Byte])
/**
* The class identifies the ranges for a java primitive type. The caller needs
* to decide the bound is either inclusive or exclusive on its own.
* information
*
* @param less: the set of ranges for LessThan/LessOrEqualThan
* @param greater: the set of ranges for GreaterThan/GreaterThanOrEqualTo
* @param value: the byte array of the original value
*/
@InterfaceAudience.LimitedPrivate(Array(HBaseInterfaceAudience.SPARK))
@InterfaceStability.Evolving
case class BoundRanges(less: Array[BoundRange], greater: Array[BoundRange], value: Array[Byte])
/**
* The trait to support plugin architecture for different encoder/decoder.
* encode is used for serializing the data type to byte array and the filter is
* used to filter out the unnecessary records.
*/
@InterfaceAudience.LimitedPrivate(Array(HBaseInterfaceAudience.SPARK))
@InterfaceStability.Evolving
trait BytesEncoder {
def encode(dt: DataType, value: Any): Array[Byte]
/**
* The function performing real filtering operations. The format of filterBytes depends on the
* implementation of the BytesEncoder.
*
* @param input: the current input byte array that needs to be filtered out
* @param offset1: the starting offset of the input byte array.
* @param length1: the length of the input byte array.
* @param filterBytes: the byte array provided by query condition.
* @param offset2: the starting offset in the filterBytes.
* @param length2: the length of the bytes in the filterBytes
* @param ops: The operation of the filter operator.
* @return true: the record satisfies the predicates
* false: the record does not satisfy the predicates.
*/
def filter(input: Array[Byte], offset1: Int, length1: Int,
filterBytes: Array[Byte], offset2: Int, length2: Int,
ops: JavaBytesEncoder): Boolean
/**
* Currently, it is used for partition pruning.
* As for some codec, the order may be inconsistent between java primitive
* type and its byte array. We may have to split the predicates on some
* of the java primitive type into multiple predicates.
*
* For example in naive codec, some of the java primitive types have to be
* split into multiple predicates, and union these predicates together to
* make the predicates be performed correctly.
* For example, if we have "COLUMN < 2", we will transform it into
* "0 <= COLUMN < 2 OR Integer.MIN_VALUE <= COLUMN <= -1"
*/
def ranges(in: Any): Option[BoundRanges]
}
@InterfaceAudience.LimitedPrivate(Array(HBaseInterfaceAudience.SPARK))
@InterfaceStability.Evolving
object JavaBytesEncoder extends Enumeration with Logging{
type JavaBytesEncoder = Value
val Greater, GreaterEqual, Less, LessEqual, Equal, Unknown = Value
/**
* create the encoder/decoder
*
* @param clsName: the class name of the encoder/decoder class
* @return the instance of the encoder plugin.
*/
def create(clsName: String): BytesEncoder = {
try {
Class.forName(clsName).newInstance.asInstanceOf[BytesEncoder]
} catch {
case _: Throwable =>
logWarning(s"$clsName cannot be initiated, falling back to naive encoder")
new NaiveEncoder()
}
}
}

View File

@ -1,261 +0,0 @@
package org.apache.hadoop.hbase.spark.datasources
/*
* 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.
*/
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder.JavaBytesEncoder
import org.apache.hadoop.hbase.spark.hbase._
import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.Logging
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
/**
* This is the naive non-order preserving encoder/decoder.
* Due to the inconsistency of the order between java primitive types
* and their bytearray. The data type has to be passed in so that the filter
* can work correctly, which is done by wrapping the type into the first byte
* of the serialized array.
*/
@InterfaceAudience.Private
class NaiveEncoder extends BytesEncoder with Logging{
var code = 0
def nextCode: Byte = {
code += 1
(code - 1).asInstanceOf[Byte]
}
val BooleanEnc = nextCode
val ShortEnc = nextCode
val IntEnc = nextCode
val LongEnc = nextCode
val FloatEnc = nextCode
val DoubleEnc = nextCode
val StringEnc = nextCode
val BinaryEnc = nextCode
val TimestampEnc = nextCode
val UnknownEnc = nextCode
/**
* Evaluate the java primitive type and return the BoundRanges. For one value, it may have
* multiple output ranges because of the inconsistency of order between java primitive type
* and its byte array order.
*
* For short, integer, and long, the order of number is consistent with byte array order
* if two number has the same sign bit. But the negative number is larger than positive
* number in byte array.
*
* For double and float, the order of positive number is consistent with its byte array order.
* But the order of negative number is the reverse order of byte array. Please refer to IEEE-754
* and https://en.wikipedia.org/wiki/Single-precision_floating-point_format
*/
def ranges(in: Any): Option[BoundRanges] = in match {
case a: Integer =>
val b = Bytes.toBytes(a)
if (a >= 0) {
logDebug(s"range is 0 to $a and ${Integer.MIN_VALUE} to -1")
Some(BoundRanges(
Array(BoundRange(Bytes.toBytes(0: Int), b),
BoundRange(Bytes.toBytes(Integer.MIN_VALUE), Bytes.toBytes(-1: Int))),
Array(BoundRange(b, Bytes.toBytes(Integer.MAX_VALUE))), b))
} else {
Some(BoundRanges(
Array(BoundRange(Bytes.toBytes(Integer.MIN_VALUE), b)),
Array(BoundRange(b, Bytes.toBytes(-1: Integer)),
BoundRange(Bytes.toBytes(0: Int), Bytes.toBytes(Integer.MAX_VALUE))), b))
}
case a: Long =>
val b = Bytes.toBytes(a)
if (a >= 0) {
Some(BoundRanges(
Array(BoundRange(Bytes.toBytes(0: Long), b),
BoundRange(Bytes.toBytes(Long.MinValue), Bytes.toBytes(-1: Long))),
Array(BoundRange(b, Bytes.toBytes(Long.MaxValue))), b))
} else {
Some(BoundRanges(
Array(BoundRange(Bytes.toBytes(Long.MinValue), b)),
Array(BoundRange(b, Bytes.toBytes(-1: Long)),
BoundRange(Bytes.toBytes(0: Long), Bytes.toBytes(Long.MaxValue))), b))
}
case a: Short =>
val b = Bytes.toBytes(a)
if (a >= 0) {
Some(BoundRanges(
Array(BoundRange(Bytes.toBytes(0: Short), b),
BoundRange(Bytes.toBytes(Short.MinValue), Bytes.toBytes(-1: Short))),
Array(BoundRange(b, Bytes.toBytes(Short.MaxValue))), b))
} else {
Some(BoundRanges(
Array(BoundRange(Bytes.toBytes(Short.MinValue), b)),
Array(BoundRange(b, Bytes.toBytes(-1: Short)),
BoundRange(Bytes.toBytes(0: Short), Bytes.toBytes(Short.MaxValue))), b))
}
case a: Double =>
val b = Bytes.toBytes(a)
if (a >= 0.0f) {
Some(BoundRanges(
Array(BoundRange(Bytes.toBytes(0.0d), b),
BoundRange(Bytes.toBytes(-0.0d), Bytes.toBytes(Double.MinValue))),
Array(BoundRange(b, Bytes.toBytes(Double.MaxValue))), b))
} else {
Some(BoundRanges(
Array(BoundRange(b, Bytes.toBytes(Double.MinValue))),
Array(BoundRange(Bytes.toBytes(-0.0d), b),
BoundRange(Bytes.toBytes(0.0d), Bytes.toBytes(Double.MaxValue))), b))
}
case a: Float =>
val b = Bytes.toBytes(a)
if (a >= 0.0f) {
Some(BoundRanges(
Array(BoundRange(Bytes.toBytes(0.0f), b),
BoundRange(Bytes.toBytes(-0.0f), Bytes.toBytes(Float.MinValue))),
Array(BoundRange(b, Bytes.toBytes(Float.MaxValue))), b))
} else {
Some(BoundRanges(
Array(BoundRange(b, Bytes.toBytes(Float.MinValue))),
Array(BoundRange(Bytes.toBytes(-0.0f), b),
BoundRange(Bytes.toBytes(0.0f), Bytes.toBytes(Float.MaxValue))), b))
}
case a: Array[Byte] =>
Some(BoundRanges(
Array(BoundRange(bytesMin, a)),
Array(BoundRange(a, bytesMax)), a))
case a: Byte =>
val b = Array(a)
Some(BoundRanges(
Array(BoundRange(bytesMin, b)),
Array(BoundRange(b, bytesMax)), b))
case a: String =>
val b = Bytes.toBytes(a)
Some(BoundRanges(
Array(BoundRange(bytesMin, b)),
Array(BoundRange(b, bytesMax)), b))
case a: UTF8String =>
val b = a.getBytes
Some(BoundRanges(
Array(BoundRange(bytesMin, b)),
Array(BoundRange(b, bytesMax)), b))
case _ => None
}
def compare(c: Int, ops: JavaBytesEncoder): Boolean = {
ops match {
case JavaBytesEncoder.Greater => c > 0
case JavaBytesEncoder.GreaterEqual => c >= 0
case JavaBytesEncoder.Less => c < 0
case JavaBytesEncoder.LessEqual => c <= 0
}
}
/**
* encode the data type into byte array. Note that it is a naive implementation with the
* data type byte appending to the head of the serialized byte array.
*
* @param dt: The data type of the input
* @param value: the value of the input
* @return the byte array with the first byte indicating the data type.
*/
override def encode(dt: DataType,
value: Any): Array[Byte] = {
dt match {
case BooleanType =>
val result = new Array[Byte](Bytes.SIZEOF_BOOLEAN + 1)
result(0) = BooleanEnc
value.asInstanceOf[Boolean] match {
case true => result(1) = -1: Byte
case false => result(1) = 0: Byte
}
result
case ShortType =>
val result = new Array[Byte](Bytes.SIZEOF_SHORT + 1)
result(0) = ShortEnc
Bytes.putShort(result, 1, value.asInstanceOf[Short])
result
case IntegerType =>
val result = new Array[Byte](Bytes.SIZEOF_INT + 1)
result(0) = IntEnc
Bytes.putInt(result, 1, value.asInstanceOf[Int])
result
case LongType|TimestampType =>
val result = new Array[Byte](Bytes.SIZEOF_LONG + 1)
result(0) = LongEnc
Bytes.putLong(result, 1, value.asInstanceOf[Long])
result
case FloatType =>
val result = new Array[Byte](Bytes.SIZEOF_FLOAT + 1)
result(0) = FloatEnc
Bytes.putFloat(result, 1, value.asInstanceOf[Float])
result
case DoubleType =>
val result = new Array[Byte](Bytes.SIZEOF_DOUBLE + 1)
result(0) = DoubleEnc
Bytes.putDouble(result, 1, value.asInstanceOf[Double])
result
case BinaryType =>
val v = value.asInstanceOf[Array[Bytes]]
val result = new Array[Byte](v.length + 1)
result(0) = BinaryEnc
System.arraycopy(v, 0, result, 1, v.length)
result
case StringType =>
val bytes = Bytes.toBytes(value.asInstanceOf[String])
val result = new Array[Byte](bytes.length + 1)
result(0) = StringEnc
System.arraycopy(bytes, 0, result, 1, bytes.length)
result
case _ =>
val bytes = Bytes.toBytes(value.toString)
val result = new Array[Byte](bytes.length + 1)
result(0) = UnknownEnc
System.arraycopy(bytes, 0, result, 1, bytes.length)
result
}
}
override def filter(input: Array[Byte], offset1: Int, length1: Int,
filterBytes: Array[Byte], offset2: Int, length2: Int,
ops: JavaBytesEncoder): Boolean = {
filterBytes(offset2) match {
case ShortEnc =>
val in = Bytes.toShort(input, offset1)
val value = Bytes.toShort(filterBytes, offset2 + 1)
compare(in.compareTo(value), ops)
case IntEnc =>
val in = Bytes.toInt(input, offset1)
val value = Bytes.toInt(filterBytes, offset2 + 1)
compare(in.compareTo(value), ops)
case LongEnc | TimestampEnc =>
val in = Bytes.toInt(input, offset1)
val value = Bytes.toInt(filterBytes, offset2 + 1)
compare(in.compareTo(value), ops)
case FloatEnc =>
val in = Bytes.toFloat(input, offset1)
val value = Bytes.toFloat(filterBytes, offset2 + 1)
compare(in.compareTo(value), ops)
case DoubleEnc =>
val in = Bytes.toDouble(input, offset1)
val value = Bytes.toDouble(filterBytes, offset2 + 1)
compare(in.compareTo(value), ops)
case _ =>
// for String, Byte, Binary, Boolean and other types
// we can use the order of byte array directly.
compare(
Bytes.compareTo(input, offset1, length1, filterBytes, offset2 + 1, length2 - 1), ops)
}
}
}

View File

@ -1,430 +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.hadoop.hbase.spark
import java.io.ByteArrayInputStream
import java.nio.ByteBuffer
import java.sql.Timestamp
import java.util
import java.util.HashMap
import org.apache.avro.SchemaBuilder.BaseFieldTypeBuilder
import org.apache.avro.SchemaBuilder.BaseTypeBuilder
import org.apache.avro.SchemaBuilder.FieldAssembler
import org.apache.avro.SchemaBuilder.FieldDefault
import org.apache.avro.SchemaBuilder.RecordBuilder
import org.apache.avro.io._
import org.apache.commons.io.output.ByteArrayOutputStream
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes
import scala.collection.JavaConversions._
import org.apache.avro.{SchemaBuilder, Schema}
import org.apache.avro.Schema.Type._
import org.apache.avro.generic.GenericData.{Record, Fixed}
import org.apache.avro.generic.{GenericDatumReader, GenericDatumWriter, GenericData, GenericRecord}
import org.apache.spark.sql.Row
import org.apache.spark.sql.types._
import scala.collection.immutable.Map
@InterfaceAudience.Private
abstract class AvroException(msg: String) extends Exception(msg)
@InterfaceAudience.Private
case class SchemaConversionException(msg: String) extends AvroException(msg)
/***
* On top level, the converters provide three high level interface.
* 1. toSqlType: This function takes an avro schema and returns a sql schema.
* 2. createConverterToSQL: Returns a function that is used to convert avro types to their
* corresponding sparkSQL representations.
* 3. convertTypeToAvro: This function constructs converter function for a given sparkSQL
* datatype. This is used in writing Avro records out to disk
*/
@InterfaceAudience.Private
object SchemaConverters {
case class SchemaType(dataType: DataType, nullable: Boolean)
/**
* This function takes an avro schema and returns a sql schema.
*/
def toSqlType(avroSchema: Schema): SchemaType = {
avroSchema.getType match {
case INT => SchemaType(IntegerType, nullable = false)
case STRING => SchemaType(StringType, nullable = false)
case BOOLEAN => SchemaType(BooleanType, nullable = false)
case BYTES => SchemaType(BinaryType, nullable = false)
case DOUBLE => SchemaType(DoubleType, nullable = false)
case FLOAT => SchemaType(FloatType, nullable = false)
case LONG => SchemaType(LongType, nullable = false)
case FIXED => SchemaType(BinaryType, nullable = false)
case ENUM => SchemaType(StringType, nullable = false)
case RECORD =>
val fields = avroSchema.getFields.map { f =>
val schemaType = toSqlType(f.schema())
StructField(f.name, schemaType.dataType, schemaType.nullable)
}
SchemaType(StructType(fields), nullable = false)
case ARRAY =>
val schemaType = toSqlType(avroSchema.getElementType)
SchemaType(
ArrayType(schemaType.dataType, containsNull = schemaType.nullable),
nullable = false)
case MAP =>
val schemaType = toSqlType(avroSchema.getValueType)
SchemaType(
MapType(StringType, schemaType.dataType, valueContainsNull = schemaType.nullable),
nullable = false)
case UNION =>
if (avroSchema.getTypes.exists(_.getType == NULL)) {
// In case of a union with null, eliminate it and make a recursive call
val remainingUnionTypes = avroSchema.getTypes.filterNot(_.getType == NULL)
if (remainingUnionTypes.size == 1) {
toSqlType(remainingUnionTypes.get(0)).copy(nullable = true)
} else {
toSqlType(Schema.createUnion(remainingUnionTypes)).copy(nullable = true)
}
} else avroSchema.getTypes.map(_.getType) match {
case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) =>
SchemaType(LongType, nullable = false)
case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) =>
SchemaType(DoubleType, nullable = false)
case other => throw new SchemaConversionException(
s"This mix of union types is not supported: $other")
}
case other => throw new SchemaConversionException(s"Unsupported type $other")
}
}
/**
* This function converts sparkSQL StructType into avro schema. This method uses two other
* converter methods in order to do the conversion.
*/
private def convertStructToAvro[T](
structType: StructType,
schemaBuilder: RecordBuilder[T],
recordNamespace: String): T = {
val fieldsAssembler: FieldAssembler[T] = schemaBuilder.fields()
structType.fields.foreach { field =>
val newField = fieldsAssembler.name(field.name).`type`()
if (field.nullable) {
convertFieldTypeToAvro(field.dataType, newField.nullable(), field.name, recordNamespace)
.noDefault
} else {
convertFieldTypeToAvro(field.dataType, newField, field.name, recordNamespace)
.noDefault
}
}
fieldsAssembler.endRecord()
}
/**
* Returns a function that is used to convert avro types to their
* corresponding sparkSQL representations.
*/
def createConverterToSQL(schema: Schema): Any => Any = {
schema.getType match {
// Avro strings are in Utf8, so we have to call toString on them
case STRING | ENUM => (item: Any) => if (item == null) null else item.toString
case INT | BOOLEAN | DOUBLE | FLOAT | LONG => identity
// Byte arrays are reused by avro, so we have to make a copy of them.
case FIXED => (item: Any) => if (item == null) {
null
} else {
item.asInstanceOf[Fixed].bytes().clone()
}
case BYTES => (item: Any) => if (item == null) {
null
} else {
val bytes = item.asInstanceOf[ByteBuffer]
val javaBytes = new Array[Byte](bytes.remaining)
bytes.get(javaBytes)
javaBytes
}
case RECORD =>
val fieldConverters = schema.getFields.map(f => createConverterToSQL(f.schema))
(item: Any) => if (item == null) {
null
} else {
val record = item.asInstanceOf[GenericRecord]
val converted = new Array[Any](fieldConverters.size)
var idx = 0
while (idx < fieldConverters.size) {
converted(idx) = fieldConverters.apply(idx)(record.get(idx))
idx += 1
}
Row.fromSeq(converted.toSeq)
}
case ARRAY =>
val elementConverter = createConverterToSQL(schema.getElementType)
(item: Any) => if (item == null) {
null
} else {
try {
item.asInstanceOf[GenericData.Array[Any]].map(elementConverter)
} catch {
case e: Throwable =>
item.asInstanceOf[util.ArrayList[Any]].map(elementConverter)
}
}
case MAP =>
val valueConverter = createConverterToSQL(schema.getValueType)
(item: Any) => if (item == null) {
null
} else {
item.asInstanceOf[HashMap[Any, Any]].map(x => (x._1.toString, valueConverter(x._2))).toMap
}
case UNION =>
if (schema.getTypes.exists(_.getType == NULL)) {
val remainingUnionTypes = schema.getTypes.filterNot(_.getType == NULL)
if (remainingUnionTypes.size == 1) {
createConverterToSQL(remainingUnionTypes.get(0))
} else {
createConverterToSQL(Schema.createUnion(remainingUnionTypes))
}
} else schema.getTypes.map(_.getType) match {
case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) =>
(item: Any) => {
item match {
case l: Long => l
case i: Int => i.toLong
case null => null
}
}
case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) =>
(item: Any) => {
item match {
case d: Double => d
case f: Float => f.toDouble
case null => null
}
}
case other => throw new SchemaConversionException(
s"This mix of union types is not supported (see README): $other")
}
case other => throw new SchemaConversionException(s"invalid avro type: $other")
}
}
/**
* This function is used to convert some sparkSQL type to avro type. Note that this function won't
* be used to construct fields of avro record (convertFieldTypeToAvro is used for that).
*/
private def convertTypeToAvro[T](
dataType: DataType,
schemaBuilder: BaseTypeBuilder[T],
structName: String,
recordNamespace: String): T = {
dataType match {
case ByteType => schemaBuilder.intType()
case ShortType => schemaBuilder.intType()
case IntegerType => schemaBuilder.intType()
case LongType => schemaBuilder.longType()
case FloatType => schemaBuilder.floatType()
case DoubleType => schemaBuilder.doubleType()
case _: DecimalType => schemaBuilder.stringType()
case StringType => schemaBuilder.stringType()
case BinaryType => schemaBuilder.bytesType()
case BooleanType => schemaBuilder.booleanType()
case TimestampType => schemaBuilder.longType()
case ArrayType(elementType, _) =>
val builder = getSchemaBuilder(dataType.asInstanceOf[ArrayType].containsNull)
val elementSchema = convertTypeToAvro(elementType, builder, structName, recordNamespace)
schemaBuilder.array().items(elementSchema)
case MapType(StringType, valueType, _) =>
val builder = getSchemaBuilder(dataType.asInstanceOf[MapType].valueContainsNull)
val valueSchema = convertTypeToAvro(valueType, builder, structName, recordNamespace)
schemaBuilder.map().values(valueSchema)
case structType: StructType =>
convertStructToAvro(
structType,
schemaBuilder.record(structName).namespace(recordNamespace),
recordNamespace)
case other => throw new IllegalArgumentException(s"Unexpected type $dataType.")
}
}
/**
* This function is used to construct fields of the avro record, where schema of the field is
* specified by avro representation of dataType. Since builders for record fields are different
* from those for everything else, we have to use a separate method.
*/
private def convertFieldTypeToAvro[T](
dataType: DataType,
newFieldBuilder: BaseFieldTypeBuilder[T],
structName: String,
recordNamespace: String): FieldDefault[T, _] = {
dataType match {
case ByteType => newFieldBuilder.intType()
case ShortType => newFieldBuilder.intType()
case IntegerType => newFieldBuilder.intType()
case LongType => newFieldBuilder.longType()
case FloatType => newFieldBuilder.floatType()
case DoubleType => newFieldBuilder.doubleType()
case _: DecimalType => newFieldBuilder.stringType()
case StringType => newFieldBuilder.stringType()
case BinaryType => newFieldBuilder.bytesType()
case BooleanType => newFieldBuilder.booleanType()
case TimestampType => newFieldBuilder.longType()
case ArrayType(elementType, _) =>
val builder = getSchemaBuilder(dataType.asInstanceOf[ArrayType].containsNull)
val elementSchema = convertTypeToAvro(elementType, builder, structName, recordNamespace)
newFieldBuilder.array().items(elementSchema)
case MapType(StringType, valueType, _) =>
val builder = getSchemaBuilder(dataType.asInstanceOf[MapType].valueContainsNull)
val valueSchema = convertTypeToAvro(valueType, builder, structName, recordNamespace)
newFieldBuilder.map().values(valueSchema)
case structType: StructType =>
convertStructToAvro(
structType,
newFieldBuilder.record(structName).namespace(recordNamespace),
recordNamespace)
case other => throw new IllegalArgumentException(s"Unexpected type $dataType.")
}
}
private def getSchemaBuilder(isNullable: Boolean): BaseTypeBuilder[Schema] = {
if (isNullable) {
SchemaBuilder.builder().nullable()
} else {
SchemaBuilder.builder()
}
}
/**
* This function constructs converter function for a given sparkSQL datatype. This is used in
* writing Avro records out to disk
*/
def createConverterToAvro(
dataType: DataType,
structName: String,
recordNamespace: String): (Any) => Any = {
dataType match {
case BinaryType => (item: Any) => item match {
case null => null
case bytes: Array[Byte] => ByteBuffer.wrap(bytes)
}
case ByteType | ShortType | IntegerType | LongType |
FloatType | DoubleType | StringType | BooleanType => identity
case _: DecimalType => (item: Any) => if (item == null) null else item.toString
case TimestampType => (item: Any) =>
if (item == null) null else item.asInstanceOf[Timestamp].getTime
case ArrayType(elementType, _) =>
val elementConverter = createConverterToAvro(elementType, structName, recordNamespace)
(item: Any) => {
if (item == null) {
null
} else {
val sourceArray = item.asInstanceOf[Seq[Any]]
val sourceArraySize = sourceArray.size
val targetArray = new util.ArrayList[Any](sourceArraySize)
var idx = 0
while (idx < sourceArraySize) {
targetArray.add(elementConverter(sourceArray(idx)))
idx += 1
}
targetArray
}
}
case MapType(StringType, valueType, _) =>
val valueConverter = createConverterToAvro(valueType, structName, recordNamespace)
(item: Any) => {
if (item == null) {
null
} else {
val javaMap = new HashMap[String, Any]()
item.asInstanceOf[Map[String, Any]].foreach { case (key, value) =>
javaMap.put(key, valueConverter(value))
}
javaMap
}
}
case structType: StructType =>
val builder = SchemaBuilder.record(structName).namespace(recordNamespace)
val schema: Schema = SchemaConverters.convertStructToAvro(
structType, builder, recordNamespace)
val fieldConverters = structType.fields.map(field =>
createConverterToAvro(field.dataType, field.name, recordNamespace))
(item: Any) => {
if (item == null) {
null
} else {
val record = new Record(schema)
val convertersIterator = fieldConverters.iterator
val fieldNamesIterator = dataType.asInstanceOf[StructType].fieldNames.iterator
val rowIterator = item.asInstanceOf[Row].toSeq.iterator
while (convertersIterator.hasNext) {
val converter = convertersIterator.next()
record.put(fieldNamesIterator.next(), converter(rowIterator.next()))
}
record
}
}
}
}
}
@InterfaceAudience.Private
object AvroSerdes {
// We only handle top level is record or primary type now
def serialize(input: Any, schema: Schema): Array[Byte]= {
schema.getType match {
case BOOLEAN => Bytes.toBytes(input.asInstanceOf[Boolean])
case BYTES | FIXED=> input.asInstanceOf[Array[Byte]]
case DOUBLE => Bytes.toBytes(input.asInstanceOf[Double])
case FLOAT => Bytes.toBytes(input.asInstanceOf[Float])
case INT => Bytes.toBytes(input.asInstanceOf[Int])
case LONG => Bytes.toBytes(input.asInstanceOf[Long])
case STRING => Bytes.toBytes(input.asInstanceOf[String])
case RECORD =>
val gr = input.asInstanceOf[GenericRecord]
val writer2 = new GenericDatumWriter[GenericRecord](schema)
val bao2 = new ByteArrayOutputStream()
val encoder2: BinaryEncoder = EncoderFactory.get().directBinaryEncoder(bao2, null)
writer2.write(gr, encoder2)
bao2.toByteArray()
case _ => throw new Exception(s"unsupported data type ${schema.getType}") //TODO
}
}
def deserialize(input: Array[Byte], schema: Schema): GenericRecord = {
val reader2: DatumReader[GenericRecord] = new GenericDatumReader[GenericRecord](schema)
val bai2 = new ByteArrayInputStream(input)
val decoder2: BinaryDecoder = DecoderFactory.get().directBinaryDecoder(bai2, null)
val gr2: GenericRecord = reader2.read(null, decoder2)
gr2
}
}

View File

@ -1,47 +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.hadoop.hbase.spark.datasources
import java.io.ByteArrayInputStream
import org.apache.avro.Schema
import org.apache.avro.Schema.Type._
import org.apache.avro.generic.GenericDatumReader
import org.apache.avro.generic.GenericDatumWriter
import org.apache.avro.generic.GenericRecord
import org.apache.avro.generic.{GenericDatumReader, GenericDatumWriter, GenericRecord}
import org.apache.avro.io._
import org.apache.commons.io.output.ByteArrayOutputStream
import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.sql.types._
// TODO: This is not really used in code.
trait SerDes {
def serialize(value: Any): Array[Byte]
def deserialize(bytes: Array[Byte], start: Int, end: Int): Any
}
// TODO: This is not really used in code.
class DoubleSerDes extends SerDes {
override def serialize(value: Any): Array[Byte] = Bytes.toBytes(value.asInstanceOf[Double])
override def deserialize(bytes: Array[Byte], start: Int, end: Int): Any = {
Bytes.toDouble(bytes, start)
}
}

View File

@ -1,47 +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.hadoop.hbase.spark.datasources
import java.io.{IOException, ObjectInputStream, ObjectOutputStream}
import org.apache.hadoop.conf.Configuration
import org.apache.yetus.audience.InterfaceAudience;
import scala.util.control.NonFatal
@InterfaceAudience.Private
class SerializableConfiguration(@transient var value: Configuration) extends Serializable {
private def writeObject(out: ObjectOutputStream): Unit = tryOrIOException {
out.defaultWriteObject()
value.write(out)
}
private def readObject(in: ObjectInputStream): Unit = tryOrIOException {
value = new Configuration(false)
value.readFields(in)
}
def tryOrIOException(block: => Unit) {
try {
block
} catch {
case e: IOException => throw e
case NonFatal(t) => throw new IOException(t)
}
}
}

View File

@ -1,38 +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.hadoop.hbase.spark
import org.apache.hadoop.hbase.util.Bytes
import scala.math.Ordering
package object hbase {
type HBaseType = Array[Byte]
def bytesMin = new Array[Byte](0)
def bytesMax = null
val ByteMax = -1.asInstanceOf[Byte]
val ByteMin = 0.asInstanceOf[Byte]
val ord: Ordering[HBaseType] = new Ordering[HBaseType] {
def compare(x: Array[Byte], y: Array[Byte]): Int = {
return Bytes.compareTo(x, y)
}
}
//Do not use BinaryType.ordering
implicit val order: Ordering[HBaseType] = ord
}

View File

@ -1,158 +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.hadoop.hbase.spark.example.datasources
import org.apache.avro.Schema
import org.apache.avro.generic.GenericData
import org.apache.hadoop.hbase.spark.AvroSerdes
import org.apache.spark.sql.datasources.hbase.HBaseTableCatalog
import org.apache.spark.sql.{DataFrame, SQLContext}
import org.apache.spark.{SparkConf, SparkContext}
/**
* @param col0 Column #0, Type is String
* @param col1 Column #1, Type is Array[Byte]
*/
case class AvroHBaseRecord(col0: String,
col1: Array[Byte])
object AvroHBaseRecord {
val schemaString =
s"""{"namespace": "example.avro",
| "type": "record", "name": "User",
| "fields": [
| {"name": "name", "type": "string"},
| {"name": "favorite_number", "type": ["int", "null"]},
| {"name": "favorite_color", "type": ["string", "null"]},
| {"name": "favorite_array", "type": {"type": "array", "items": "string"}},
| {"name": "favorite_map", "type": {"type": "map", "values": "int"}}
| ] }""".stripMargin
val avroSchema: Schema = {
val p = new Schema.Parser
p.parse(schemaString)
}
def apply(i: Int): AvroHBaseRecord = {
val user = new GenericData.Record(avroSchema);
user.put("name", s"name${"%03d".format(i)}")
user.put("favorite_number", i)
user.put("favorite_color", s"color${"%03d".format(i)}")
val favoriteArray = new GenericData.Array[String](2, avroSchema.getField("favorite_array").schema())
favoriteArray.add(s"number${i}")
favoriteArray.add(s"number${i+1}")
user.put("favorite_array", favoriteArray)
import collection.JavaConverters._
val favoriteMap = Map[String, Int](("key1" -> i), ("key2" -> (i+1))).asJava
user.put("favorite_map", favoriteMap)
val avroByte = AvroSerdes.serialize(user, avroSchema)
AvroHBaseRecord(s"name${"%03d".format(i)}", avroByte)
}
}
object AvroSource {
def catalog = s"""{
|"table":{"namespace":"default", "name":"ExampleAvrotable"},
|"rowkey":"key",
|"columns":{
|"col0":{"cf":"rowkey", "col":"key", "type":"string"},
|"col1":{"cf":"cf1", "col":"col1", "type":"binary"}
|}
|}""".stripMargin
def avroCatalog = s"""{
|"table":{"namespace":"default", "name":"ExampleAvrotable"},
|"rowkey":"key",
|"columns":{
|"col0":{"cf":"rowkey", "col":"key", "type":"string"},
|"col1":{"cf":"cf1", "col":"col1", "avro":"avroSchema"}
|}
|}""".stripMargin
def avroCatalogInsert = s"""{
|"table":{"namespace":"default", "name":"ExampleAvrotableInsert"},
|"rowkey":"key",
|"columns":{
|"col0":{"cf":"rowkey", "col":"key", "type":"string"},
|"col1":{"cf":"cf1", "col":"col1", "avro":"avroSchema"}
|}
|}""".stripMargin
def main(args: Array[String]) {
val sparkConf = new SparkConf().setAppName("AvroSourceExample")
val sc = new SparkContext(sparkConf)
val sqlContext = new SQLContext(sc)
import sqlContext.implicits._
def withCatalog(cat: String): DataFrame = {
sqlContext
.read
.options(Map("avroSchema" -> AvroHBaseRecord.schemaString, HBaseTableCatalog.tableCatalog -> avroCatalog))
.format("org.apache.hadoop.hbase.spark")
.load()
}
val data = (0 to 255).map { i =>
AvroHBaseRecord(i)
}
sc.parallelize(data).toDF.write.options(
Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
.format("org.apache.hadoop.hbase.spark")
.save()
val df = withCatalog(catalog)
df.show()
df.printSchema()
df.registerTempTable("ExampleAvrotable")
val c = sqlContext.sql("select count(1) from ExampleAvrotable")
c.show()
val filtered = df.select($"col0", $"col1.favorite_array").where($"col0" === "name001")
filtered.show()
val collected = filtered.collect()
if (collected(0).getSeq[String](1)(0) != "number1") {
throw new UserCustomizedSampleException("value invalid")
}
if (collected(0).getSeq[String](1)(1) != "number2") {
throw new UserCustomizedSampleException("value invalid")
}
df.write.options(
Map("avroSchema"->AvroHBaseRecord.schemaString, HBaseTableCatalog.tableCatalog->avroCatalogInsert,
HBaseTableCatalog.newTable -> "5"))
.format("org.apache.hadoop.hbase.spark")
.save()
val newDF = withCatalog(avroCatalogInsert)
newDF.show()
newDF.printSchema()
if(newDF.count() != 256) {
throw new UserCustomizedSampleException("value invalid")
}
df.filter($"col1.name" === "name005" || $"col1.name" <= "name005")
.select("col0", "col1.favorite_color", "col1.favorite_number")
.show()
df.filter($"col1.name" <= "name005" || $"col1.name".contains("name007"))
.select("col0", "col1.favorite_color", "col1.favorite_number")
.show()
}
}

View File

@ -1,165 +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.hadoop.hbase.spark.example.datasources
import org.apache.spark.sql.{DataFrame, SQLContext}
import org.apache.spark.{SparkContext, SparkConf}
import org.apache.spark.sql.datasources.hbase.HBaseTableCatalog
class UserCustomizedSampleException(message: String = null, cause: Throwable = null) extends
RuntimeException(UserCustomizedSampleException.message(message, cause), cause)
object UserCustomizedSampleException {
def message(message: String, cause: Throwable) =
if (message != null) message
else if (cause != null) cause.toString()
else null
}
case class IntKeyRecord(
col0: Integer,
col1: Boolean,
col2: Double,
col3: Float,
col4: Int,
col5: Long,
col6: Short,
col7: String,
col8: Byte)
object IntKeyRecord {
def apply(i: Int): IntKeyRecord = {
IntKeyRecord(if (i % 2 == 0) i else -i,
i % 2 == 0,
i.toDouble,
i.toFloat,
i,
i.toLong,
i.toShort,
s"String$i extra",
i.toByte)
}
}
object DataType {
val cat = s"""{
|"table":{"namespace":"default", "name":"DataTypeExampleTable"},
|"rowkey":"key",
|"columns":{
|"col0":{"cf":"rowkey", "col":"key", "type":"int"},
|"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
|"col2":{"cf":"cf2", "col":"col2", "type":"double"},
|"col3":{"cf":"cf3", "col":"col3", "type":"float"},
|"col4":{"cf":"cf4", "col":"col4", "type":"int"},
|"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
|"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
|"col7":{"cf":"cf7", "col":"col7", "type":"string"},
|"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
|}
|}""".stripMargin
def main(args: Array[String]){
val sparkConf = new SparkConf().setAppName("DataTypeExample")
val sc = new SparkContext(sparkConf)
val sqlContext = new SQLContext(sc)
import sqlContext.implicits._
def withCatalog(cat: String): DataFrame = {
sqlContext
.read
.options(Map(HBaseTableCatalog.tableCatalog->cat))
.format("org.apache.hadoop.hbase.spark")
.load()
}
// test populate table
val data = (0 until 32).map { i =>
IntKeyRecord(i)
}
sc.parallelize(data).toDF.write.options(
Map(HBaseTableCatalog.tableCatalog -> cat, HBaseTableCatalog.newTable -> "5"))
.format("org.apache.hadoop.hbase.spark")
.save()
// test less than 0
val df = withCatalog(cat)
val s = df.filter($"col0" < 0)
s.show()
if(s.count() != 16){
throw new UserCustomizedSampleException("value invalid")
}
//test less or equal than -10. The number of results is 11
val num1 = df.filter($"col0" <= -10)
num1.show()
val c1 = num1.count()
println(s"test result count should be 11: $c1")
//test less or equal than -9. The number of results is 12
val num2 = df.filter($"col0" <= -9)
num2.show()
val c2 = num2.count()
println(s"test result count should be 12: $c2")
//test greater or equal than -9". The number of results is 21
val num3 = df.filter($"col0" >= -9)
num3.show()
val c3 = num3.count()
println(s"test result count should be 21: $c3")
//test greater or equal than 0. The number of results is 16
val num4 = df.filter($"col0" >= 0)
num4.show()
val c4 = num4.count()
println(s"test result count should be 16: $c4")
//test greater than 10. The number of results is 10
val num5 = df.filter($"col0" > 10)
num5.show()
val c5 = num5.count()
println(s"test result count should be 10: $c5")
// test "and". The number of results is 11
val num6 = df.filter($"col0" > -10 && $"col0" <= 10)
num6.show()
val c6 = num6.count()
println(s"test result count should be 11: $c6")
//test "or". The number of results is 21
val num7 = df.filter($"col0" <= -10 || $"col0" > 10)
num7.show()
val c7 = num7.count()
println(s"test result count should be 21: $c7")
//test "all". The number of results is 32
val num8 = df.filter($"col0" >= -100)
num8.show()
val c8 = num8.count()
println(s"test result count should be 32: $c8")
//test "full query"
val df1 = withCatalog(cat)
df1.show()
val c_df = df1.count()
println(s"df count should be 32: $c_df")
if(c_df != 32){
throw new UserCustomizedSampleException("value invalid")
}
}
}

View File

@ -1,103 +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.hadoop.hbase.spark.example.datasources
import org.apache.spark.sql.{DataFrame, SQLContext}
import org.apache.spark.{SparkContext, SparkConf}
import org.apache.spark.sql.datasources.hbase.HBaseTableCatalog
case class HBaseRecord(
col0: String,
col1: Boolean,
col2: Double,
col3: Float,
col4: Int,
col5: Long,
col6: Short,
col7: String,
col8: Byte)
object HBaseRecord {
def apply(i: Int): HBaseRecord = {
val s = s"""row${"%03d".format(i)}"""
HBaseRecord(s,
i % 2 == 0,
i.toDouble,
i.toFloat,
i,
i.toLong,
i.toShort,
s"String$i extra",
i.toByte)
}
}
object HBaseSource {
val cat = s"""{
|"table":{"namespace":"default", "name":"HBaseSourceExampleTable"},
|"rowkey":"key",
|"columns":{
|"col0":{"cf":"rowkey", "col":"key", "type":"string"},
|"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
|"col2":{"cf":"cf2", "col":"col2", "type":"double"},
|"col3":{"cf":"cf3", "col":"col3", "type":"float"},
|"col4":{"cf":"cf4", "col":"col4", "type":"int"},
|"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
|"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
|"col7":{"cf":"cf7", "col":"col7", "type":"string"},
|"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
|}
|}""".stripMargin
def main(args: Array[String]) {
val sparkConf = new SparkConf().setAppName("HBaseSourceExample")
val sc = new SparkContext(sparkConf)
val sqlContext = new SQLContext(sc)
import sqlContext.implicits._
def withCatalog(cat: String): DataFrame = {
sqlContext
.read
.options(Map(HBaseTableCatalog.tableCatalog->cat))
.format("org.apache.hadoop.hbase.spark")
.load()
}
val data = (0 to 255).map { i =>
HBaseRecord(i)
}
sc.parallelize(data).toDF.write.options(
Map(HBaseTableCatalog.tableCatalog -> cat, HBaseTableCatalog.newTable -> "5"))
.format("org.apache.hadoop.hbase.spark")
.save()
val df = withCatalog(cat)
df.show()
df.filter($"col0" <= "row005")
.select($"col0", $"col1").show
df.filter($"col0" === "row005" || $"col0" <= "row005")
.select($"col0", $"col1").show
df.filter($"col0" > "row250")
.select($"col0", $"col1").show
df.registerTempTable("table1")
val c = sqlContext.sql("select count(col1) from table1 where col0 < 'row050'")
c.show()
}
}

View File

@ -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.hadoop.hbase.spark.example.hbasecontext
import org.apache.hadoop.hbase.spark.HBaseContext
import org.apache.spark.SparkContext
import org.apache.hadoop.hbase.{TableName, HBaseConfiguration}
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.client.Delete
import org.apache.spark.SparkConf
/**
* This is a simple example of deleting records in HBase
* with the bulkDelete function.
*/
object HBaseBulkDeleteExample {
def main(args: Array[String]) {
if (args.length < 1) {
println("HBaseBulkDeleteExample {tableName} missing an argument")
return
}
val tableName = args(0)
val sparkConf = new SparkConf().setAppName("HBaseBulkDeleteExample " + tableName)
val sc = new SparkContext(sparkConf)
try {
//[Array[Byte]]
val rdd = sc.parallelize(Array(
Bytes.toBytes("1"),
Bytes.toBytes("2"),
Bytes.toBytes("3"),
Bytes.toBytes("4"),
Bytes.toBytes("5")
))
val conf = HBaseConfiguration.create()
val hbaseContext = new HBaseContext(sc, conf)
hbaseContext.bulkDelete[Array[Byte]](rdd,
TableName.valueOf(tableName),
putRecord => new Delete(putRecord),
4)
} finally {
sc.stop()
}
}
}

View File

@ -1,93 +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.hadoop.hbase.spark.example.hbasecontext
import org.apache.hadoop.hbase.spark.HBaseContext
import org.apache.spark.SparkContext
import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseConfiguration}
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.client.Get
import org.apache.hadoop.hbase.client.Result
import org.apache.spark.SparkConf
/**
* This is a simple example of getting records from HBase
* with the bulkGet function.
*/
object HBaseBulkGetExample {
def main(args: Array[String]) {
if (args.length < 1) {
println("HBaseBulkGetExample {tableName} missing an argument")
return
}
val tableName = args(0)
val sparkConf = new SparkConf().setAppName("HBaseBulkGetExample " + tableName)
val sc = new SparkContext(sparkConf)
try {
//[(Array[Byte])]
val rdd = sc.parallelize(Array(
Bytes.toBytes("1"),
Bytes.toBytes("2"),
Bytes.toBytes("3"),
Bytes.toBytes("4"),
Bytes.toBytes("5"),
Bytes.toBytes("6"),
Bytes.toBytes("7")))
val conf = HBaseConfiguration.create()
val hbaseContext = new HBaseContext(sc, conf)
val getRdd = hbaseContext.bulkGet[Array[Byte], String](
TableName.valueOf(tableName),
2,
rdd,
record => {
System.out.println("making Get")
new Get(record)
},
(result: Result) => {
val it = result.listCells().iterator()
val b = new StringBuilder
b.append(Bytes.toString(result.getRow) + ":")
while (it.hasNext) {
val cell = it.next()
val q = Bytes.toString(CellUtil.cloneQualifier(cell))
if (q.equals("counter")) {
b.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")")
} else {
b.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")")
}
}
b.toString()
})
getRdd.collect().foreach(v => println(v))
} finally {
sc.stop()
}
}
}

View File

@ -1,75 +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.hadoop.hbase.spark.example.hbasecontext
import org.apache.hadoop.hbase.spark.HBaseContext
import org.apache.spark.SparkContext
import org.apache.hadoop.hbase.{TableName, HBaseConfiguration}
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.client.Put
import org.apache.spark.SparkConf
/**
* This is a simple example of putting records in HBase
* with the bulkPut function.
*/
object HBaseBulkPutExample {
def main(args: Array[String]) {
if (args.length < 2) {
println("HBaseBulkPutExample {tableName} {columnFamily} are missing an arguments")
return
}
val tableName = args(0)
val columnFamily = args(1)
val sparkConf = new SparkConf().setAppName("HBaseBulkPutExample " +
tableName + " " + columnFamily)
val sc = new SparkContext(sparkConf)
try {
//[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])]
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))),
(Bytes.toBytes("2"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))),
(Bytes.toBytes("3"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))),
(Bytes.toBytes("4"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))),
(Bytes.toBytes("5"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5"))))
))
val conf = HBaseConfiguration.create()
val hbaseContext = new HBaseContext(sc, conf)
hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])](rdd,
TableName.valueOf(tableName),
(putRecord) => {
val put = new Put(putRecord._1)
putRecord._2.foreach((putValue) =>
put.addColumn(putValue._1, putValue._2, putValue._3))
put
});
} finally {
sc.stop()
}
}
}

View File

@ -1,76 +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.hadoop.hbase.spark.example.hbasecontext
import org.apache.hadoop.hbase.spark.HBaseContext
import org.apache.spark.SparkContext
import org.apache.hadoop.hbase.{TableName, HBaseConfiguration}
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.client.Put
import org.apache.hadoop.mapred.TextInputFormat
import org.apache.hadoop.io.LongWritable
import org.apache.hadoop.io.Text
import org.apache.spark.SparkConf
/**
* This is a simple example of putting records in HBase
* with the bulkPut function. In this example we are
* getting the put information from a file
*/
object HBaseBulkPutExampleFromFile {
def main(args: Array[String]) {
if (args.length < 3) {
println("HBaseBulkPutExampleFromFile {tableName} {columnFamily} {inputFile} are missing an argument")
return
}
val tableName = args(0)
val columnFamily = args(1)
val inputFile = args(2)
val sparkConf = new SparkConf().setAppName("HBaseBulkPutExampleFromFile " +
tableName + " " + columnFamily + " " + inputFile)
val sc = new SparkContext(sparkConf)
try {
var rdd = sc.hadoopFile(
inputFile,
classOf[TextInputFormat],
classOf[LongWritable],
classOf[Text]).map(v => {
System.out.println("reading-" + v._2.toString)
v._2.toString
})
val conf = HBaseConfiguration.create()
val hbaseContext = new HBaseContext(sc, conf)
hbaseContext.bulkPut[String](rdd,
TableName.valueOf(tableName),
(putRecord) => {
System.out.println("hbase-" + putRecord)
val put = new Put(Bytes.toBytes("Value- " + putRecord))
put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("1"),
Bytes.toBytes(putRecord.length()))
put
});
} finally {
sc.stop()
}
}
}

View File

@ -1,77 +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.hadoop.hbase.spark.example.hbasecontext
import org.apache.hadoop.hbase.spark.HBaseContext
import org.apache.spark.SparkContext
import org.apache.hadoop.hbase.{TableName, HBaseConfiguration}
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.client.Put
import org.apache.spark.SparkConf
/**
* This is a simple example of putting records in HBase
* with the bulkPut function. In this example we are
* also setting the timestamp in the put
*/
object HBaseBulkPutTimestampExample {
def main(args: Array[String]) {
if (args.length < 2) {
System.out.println("HBaseBulkPutTimestampExample {tableName} {columnFamily} are missing an argument")
return
}
val tableName = args(0)
val columnFamily = args(1)
val sparkConf = new SparkConf().setAppName("HBaseBulkPutTimestampExample " +
tableName + " " + columnFamily)
val sc = new SparkContext(sparkConf)
try {
val rdd = sc.parallelize(Array(
(Bytes.toBytes("6"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))),
(Bytes.toBytes("7"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))),
(Bytes.toBytes("8"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))),
(Bytes.toBytes("9"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))),
(Bytes.toBytes("10"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5"))))))
val conf = HBaseConfiguration.create()
val timeStamp = System.currentTimeMillis()
val hbaseContext = new HBaseContext(sc, conf)
hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])](rdd,
TableName.valueOf(tableName),
(putRecord) => {
val put = new Put(putRecord._1)
putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2,
timeStamp, putValue._3))
put
})
} finally {
sc.stop()
}
}
}

View File

@ -1,59 +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.hadoop.hbase.spark.example.hbasecontext
import org.apache.hadoop.hbase.spark.HBaseContext
import org.apache.spark.SparkContext
import org.apache.hadoop.hbase.{TableName, HBaseConfiguration}
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.client.Scan
import org.apache.spark.SparkConf
/**
* This is a simple example of scanning records from HBase
* with the hbaseRDD function in Distributed fashion.
*/
object HBaseDistributedScanExample {
def main(args: Array[String]) {
if (args.length < 1) {
println("HBaseDistributedScanExample {tableName} missing an argument")
return
}
val tableName = args(0)
val sparkConf = new SparkConf().setAppName("HBaseDistributedScanExample " + tableName )
val sc = new SparkContext(sparkConf)
try {
val conf = HBaseConfiguration.create()
val hbaseContext = new HBaseContext(sc, conf)
val scan = new Scan()
scan.setCaching(100)
val getRdd = hbaseContext.hbaseRDD(TableName.valueOf(tableName), scan)
getRdd.foreach(v => println(Bytes.toString(v._1.get())))
println("Length: " + getRdd.map(r => r._1.copyBytes()).collect().length);
} finally {
sc.stop()
}
}
}

View File

@ -1,74 +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.hadoop.hbase.spark.example.hbasecontext
import org.apache.hadoop.hbase.spark.HBaseContext
import org.apache.spark.SparkContext
import org.apache.hadoop.hbase.{TableName, HBaseConfiguration}
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.client.Put
import org.apache.spark.streaming.StreamingContext
import org.apache.spark.streaming.Seconds
import org.apache.spark.SparkConf
/**
* This is a simple example of BulkPut with Spark Streaming
*/
object HBaseStreamingBulkPutExample {
def main(args: Array[String]) {
if (args.length < 4) {
println("HBaseStreamingBulkPutExample " +
"{host} {port} {tableName} {columnFamily} are missing an argument")
return
}
val host = args(0)
val port = args(1)
val tableName = args(2)
val columnFamily = args(3)
val sparkConf = new SparkConf().setAppName("HBaseStreamingBulkPutExample " +
tableName + " " + columnFamily)
val sc = new SparkContext(sparkConf)
try {
val ssc = new StreamingContext(sc, Seconds(1))
val lines = ssc.socketTextStream(host, port.toInt)
val conf = HBaseConfiguration.create()
val hbaseContext = new HBaseContext(sc, conf)
hbaseContext.streamBulkPut[String](lines,
TableName.valueOf(tableName),
(putRecord) => {
if (putRecord.length() > 0) {
val put = new Put(Bytes.toBytes(putRecord))
put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("foo"), Bytes.toBytes("bar"))
put
} else {
null
}
})
ssc.start()
ssc.awaitTerminationOrTimeout(60000)
} finally {
sc.stop()
}
}
}

View File

@ -1,64 +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.hadoop.hbase.spark.example.rdd
import org.apache.hadoop.hbase.client.Delete
import org.apache.hadoop.hbase.{TableName, HBaseConfiguration}
import org.apache.hadoop.hbase.spark.HBaseContext
import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._
import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.{SparkContext, SparkConf}
/**
* This is a simple example of deleting records in HBase
* with the bulkDelete function.
*/
object HBaseBulkDeleteExample {
def main(args: Array[String]) {
if (args.length < 1) {
println("HBaseBulkDeleteExample {tableName} are missing an argument")
return
}
val tableName = args(0)
val sparkConf = new SparkConf().setAppName("HBaseBulkDeleteExample " + tableName)
val sc = new SparkContext(sparkConf)
try {
//[Array[Byte]]
val rdd = sc.parallelize(Array(
Bytes.toBytes("1"),
Bytes.toBytes("2"),
Bytes.toBytes("3"),
Bytes.toBytes("4"),
Bytes.toBytes("5")
))
val conf = HBaseConfiguration.create()
val hbaseContext = new HBaseContext(sc, conf)
rdd.hbaseBulkDelete(hbaseContext, TableName.valueOf(tableName),
putRecord => new Delete(putRecord),
4)
} finally {
sc.stop()
}
}
}

View File

@ -1,88 +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.hadoop.hbase.spark.example.rdd
import org.apache.hadoop.hbase.client.{Result, Get}
import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseConfiguration}
import org.apache.hadoop.hbase.spark.HBaseContext
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._
import org.apache.spark.{SparkContext, SparkConf}
/**
* This is a simple example of getting records from HBase
* with the bulkGet function.
*/
object HBaseBulkGetExample {
def main(args: Array[String]) {
if (args.length < 1) {
println("HBaseBulkGetExample {tableName} is missing an argument")
return
}
val tableName = args(0)
val sparkConf = new SparkConf().setAppName("HBaseBulkGetExample " + tableName)
val sc = new SparkContext(sparkConf)
try {
//[(Array[Byte])]
val rdd = sc.parallelize(Array(
Bytes.toBytes("1"),
Bytes.toBytes("2"),
Bytes.toBytes("3"),
Bytes.toBytes("4"),
Bytes.toBytes("5"),
Bytes.toBytes("6"),
Bytes.toBytes("7")))
val conf = HBaseConfiguration.create()
val hbaseContext = new HBaseContext(sc, conf)
val getRdd = rdd.hbaseBulkGet[String](hbaseContext, TableName.valueOf(tableName), 2,
record => {
System.out.println("making Get")
new Get(record)
},
(result: Result) => {
val it = result.listCells().iterator()
val b = new StringBuilder
b.append(Bytes.toString(result.getRow) + ":")
while (it.hasNext) {
val cell = it.next()
val q = Bytes.toString(CellUtil.cloneQualifier(cell))
if (q.equals("counter")) {
b.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")")
} else {
b.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")")
}
}
b.toString()
})
getRdd.collect().foreach(v => println(v))
} finally {
sc.stop()
}
}
}

View File

@ -1,76 +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.hadoop.hbase.spark.example.rdd
import org.apache.hadoop.hbase.client.Put
import org.apache.hadoop.hbase.spark.HBaseContext
import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.{HBaseConfiguration, TableName}
import org.apache.spark.{SparkConf, SparkContext}
/**
* This is a simple example of putting records in HBase
* with the bulkPut function.
*/
object HBaseBulkPutExample {
def main(args: Array[String]) {
if (args.length < 2) {
println("HBaseBulkPutExample {tableName} {columnFamily} are missing an arguments")
return
}
val tableName = args(0)
val columnFamily = args(1)
val sparkConf = new SparkConf().setAppName("HBaseBulkPutExample " +
tableName + " " + columnFamily)
val sc = new SparkContext(sparkConf)
try {
//[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])]
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))),
(Bytes.toBytes("2"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))),
(Bytes.toBytes("3"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))),
(Bytes.toBytes("4"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))),
(Bytes.toBytes("5"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5"))))
))
val conf = HBaseConfiguration.create()
val hbaseContext = new HBaseContext(sc, conf)
rdd.hbaseBulkPut(hbaseContext, TableName.valueOf(tableName),
(putRecord) => {
val put = new Put(putRecord._1)
putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2,
putValue._3))
put
})
} finally {
sc.stop()
}
}
}

View File

@ -1,83 +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.hadoop.hbase.spark.example.rdd
import org.apache.hadoop.hbase.client.Put
import org.apache.hadoop.hbase.{TableName, HBaseConfiguration}
import org.apache.hadoop.hbase.spark.HBaseContext
import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._
import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.{SparkContext, SparkConf}
/**
* This is a simple example of using the foreachPartition
* method with a HBase connection
*/
object HBaseForeachPartitionExample {
def main(args: Array[String]) {
if (args.length < 2) {
println("HBaseForeachPartitionExample {tableName} {columnFamily} are missing an arguments")
return
}
val tableName = args(0)
val columnFamily = args(1)
val sparkConf = new SparkConf().setAppName("HBaseForeachPartitionExample " +
tableName + " " + columnFamily)
val sc = new SparkContext(sparkConf)
try {
//[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])]
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))),
(Bytes.toBytes("2"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))),
(Bytes.toBytes("3"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))),
(Bytes.toBytes("4"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))),
(Bytes.toBytes("5"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5"))))
))
val conf = HBaseConfiguration.create()
val hbaseContext = new HBaseContext(sc, conf)
rdd.hbaseForeachPartition(hbaseContext,
(it, connection) => {
val m = connection.getBufferedMutator(TableName.valueOf(tableName))
it.foreach(r => {
val put = new Put(r._1)
r._2.foreach((putValue) =>
put.addColumn(putValue._1, putValue._2, putValue._3))
m.mutate(put)
})
m.flush()
m.close()
})
} finally {
sc.stop()
}
}
}

View File

@ -1,89 +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.hadoop.hbase.spark.example.rdd
import org.apache.hadoop.hbase.client.Get
import org.apache.hadoop.hbase.{TableName, HBaseConfiguration}
import org.apache.hadoop.hbase.spark.HBaseContext
import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._
import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.{SparkContext, SparkConf}
/**
* This is a simple example of using the mapPartitions
* method with a HBase connection
*/
object HBaseMapPartitionExample {
def main(args: Array[String]) {
if (args.length < 1) {
println("HBaseMapPartitionExample {tableName} is missing an argument")
return
}
val tableName = args(0)
val sparkConf = new SparkConf().setAppName("HBaseMapPartitionExample " + tableName)
val sc = new SparkContext(sparkConf)
try {
//[(Array[Byte])]
val rdd = sc.parallelize(Array(
Bytes.toBytes("1"),
Bytes.toBytes("2"),
Bytes.toBytes("3"),
Bytes.toBytes("4"),
Bytes.toBytes("5"),
Bytes.toBytes("6"),
Bytes.toBytes("7")))
val conf = HBaseConfiguration.create()
val hbaseContext = new HBaseContext(sc, conf)
val getRdd = rdd.hbaseMapPartitions[String](hbaseContext, (it, connection) => {
val table = connection.getTable(TableName.valueOf(tableName))
it.map{r =>
//batching would be faster. This is just an example
val result = table.get(new Get(r))
val it = result.listCells().iterator()
val b = new StringBuilder
b.append(Bytes.toString(result.getRow) + ":")
while (it.hasNext) {
val cell = it.next()
val q = Bytes.toString(cell.getQualifierArray)
if (q.equals("counter")) {
b.append("(" + q + "," + Bytes.toLong(cell.getValueArray) + ")")
} else {
b.append("(" + q + "," + Bytes.toString(cell.getValueArray) + ")")
}
}
b.toString()
}
})
getRdd.collect().foreach(v => println(v))
} finally {
sc.stop()
}
}
}

View File

@ -1,31 +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.spark.sql.datasources.hbase
import org.apache.spark.sql.catalyst.SqlLexical
import org.apache.spark.sql.catalyst.util.DataTypeParser
import org.apache.spark.sql.types.DataType
// TODO: Only used in test suite.
object DataTypeParserWrapper {
lazy val dataTypeParser = new DataTypeParser {
override val lexical = new SqlLexical
}
def parse(dataTypeString: String): DataType = dataTypeParser.toDataType(dataTypeString)
}

View File

@ -1,377 +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.spark.sql.datasources.hbase
import org.apache.avro.Schema
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.spark.SchemaConverters
import org.apache.hadoop.hbase.spark.datasources._
import org.apache.hadoop.hbase.spark.hbase._
import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.util.DataTypeParser
import org.apache.spark.sql.types._
import org.json4s.jackson.JsonMethods._
import scala.collection.mutable
// Due the access issue defined in spark, we have to locate the file in this package.
// The definition of each column cell, which may be composite type
// TODO: add avro support
@InterfaceAudience.Private
case class Field(
colName: String,
cf: String,
col: String,
sType: Option[String] = None,
avroSchema: Option[String] = None,
serdes: Option[SerDes]= None,
len: Int = -1) extends Logging {
override def toString = s"$colName $cf $col"
val isRowKey = cf == HBaseTableCatalog.rowKey
var start: Int = _
def schema: Option[Schema] = avroSchema.map { x =>
logDebug(s"avro: $x")
val p = new Schema.Parser
p.parse(x)
}
lazy val exeSchema = schema
// converter from avro to catalyst structure
lazy val avroToCatalyst: Option[Any => Any] = {
schema.map(SchemaConverters.createConverterToSQL(_))
}
// converter from catalyst to avro
lazy val catalystToAvro: (Any) => Any ={
SchemaConverters.createConverterToAvro(dt, colName, "recordNamespace")
}
def cfBytes: Array[Byte] = {
if (isRowKey) {
Bytes.toBytes("")
} else {
Bytes.toBytes(cf)
}
}
def colBytes: Array[Byte] = {
if (isRowKey) {
Bytes.toBytes("key")
} else {
Bytes.toBytes(col)
}
}
val dt = {
sType.map(DataTypeParser.parse(_)).getOrElse{
schema.map{ x=>
SchemaConverters.toSqlType(x).dataType
}.get
}
}
var length: Int = {
if (len == -1) {
dt match {
case BinaryType | StringType => -1
case BooleanType => Bytes.SIZEOF_BOOLEAN
case ByteType => 1
case DoubleType => Bytes.SIZEOF_DOUBLE
case FloatType => Bytes.SIZEOF_FLOAT
case IntegerType => Bytes.SIZEOF_INT
case LongType => Bytes.SIZEOF_LONG
case ShortType => Bytes.SIZEOF_SHORT
case _ => -1
}
} else {
len
}
}
override def equals(other: Any): Boolean = other match {
case that: Field =>
colName == that.colName && cf == that.cf && col == that.col
case _ => false
}
}
// The row key definition, with each key refer to the col defined in Field, e.g.,
// key1:key2:key3
@InterfaceAudience.Private
case class RowKey(k: String) {
val keys = k.split(":")
var fields: Seq[Field] = _
var varLength = false
def length = {
if (varLength) {
-1
} else {
fields.foldLeft(0){case (x, y) =>
x + y.length
}
}
}
}
// The map between the column presented to Spark and the HBase field
@InterfaceAudience.Private
case class SchemaMap(map: mutable.HashMap[String, Field]) {
def toFields = map.map { case (name, field) =>
StructField(name, field.dt)
}.toSeq
def fields = map.values
def getField(name: String) = map(name)
}
// The definition of HBase and Relation relation schema
@InterfaceAudience.Private
case class HBaseTableCatalog(
namespace: String,
name: String,
row: RowKey,
sMap: SchemaMap,
@transient params: Map[String, String]) extends Logging {
def toDataType = StructType(sMap.toFields)
def getField(name: String) = sMap.getField(name)
def getRowKey: Seq[Field] = row.fields
def getPrimaryKey= row.keys(0)
def getColumnFamilies = {
sMap.fields.map(_.cf).filter(_ != HBaseTableCatalog.rowKey).toSeq.distinct
}
def get(key: String) = params.get(key)
// Setup the start and length for each dimension of row key at runtime.
def dynSetupRowKey(rowKey: Array[Byte]) {
logDebug(s"length: ${rowKey.length}")
if(row.varLength) {
var start = 0
row.fields.foreach { f =>
logDebug(s"start: $start")
f.start = start
f.length = {
// If the length is not defined
if (f.length == -1) {
f.dt match {
case StringType =>
var pos = rowKey.indexOf(HBaseTableCatalog.delimiter, start)
if (pos == -1 || pos > rowKey.length) {
// this is at the last dimension
pos = rowKey.length
}
pos - start
// We don't know the length, assume it extend to the end of the rowkey.
case _ => rowKey.length - start
}
} else {
f.length
}
}
start += f.length
}
}
}
def initRowKey = {
val fields = sMap.fields.filter(_.cf == HBaseTableCatalog.rowKey)
row.fields = row.keys.flatMap(n => fields.find(_.col == n))
// The length is determined at run time if it is string or binary and the length is undefined.
if (row.fields.filter(_.length == -1).isEmpty) {
var start = 0
row.fields.foreach { f =>
f.start = start
start += f.length
}
} else {
row.varLength = true
}
}
initRowKey
}
@InterfaceAudience.Public
object HBaseTableCatalog {
// If defined and larger than 3, a new table will be created with the nubmer of region specified.
val newTable = "newtable"
// The json string specifying hbase catalog information
val regionStart = "regionStart"
val defaultRegionStart = "aaaaaaa"
val regionEnd = "regionEnd"
val defaultRegionEnd = "zzzzzzz"
val tableCatalog = "catalog"
// The row key with format key1:key2 specifying table row key
val rowKey = "rowkey"
// The key for hbase table whose value specify namespace and table name
val table = "table"
// The namespace of hbase table
val nameSpace = "namespace"
// The name of hbase table
val tableName = "name"
// The name of columns in hbase catalog
val columns = "columns"
val cf = "cf"
val col = "col"
val `type` = "type"
// the name of avro schema json string
val avro = "avro"
val delimiter: Byte = 0
val serdes = "serdes"
val length = "length"
/**
* User provide table schema definition
* {"tablename":"name", "rowkey":"key1:key2",
* "columns":{"col1":{"cf":"cf1", "col":"col1", "type":"type1"},
* "col2":{"cf":"cf2", "col":"col2", "type":"type2"}}}
* Note that any col in the rowKey, there has to be one corresponding col defined in columns
*/
def apply(params: Map[String, String]): HBaseTableCatalog = {
val parameters = convert(params)
// println(jString)
val jString = parameters(tableCatalog)
val map = parse(jString).values.asInstanceOf[Map[String, _]]
val tableMeta = map.get(table).get.asInstanceOf[Map[String, _]]
val nSpace = tableMeta.get(nameSpace).getOrElse("default").asInstanceOf[String]
val tName = tableMeta.get(tableName).get.asInstanceOf[String]
val cIter = map.get(columns).get.asInstanceOf[Map[String, Map[String, String]]].toIterator
val schemaMap = mutable.HashMap.empty[String, Field]
cIter.foreach { case (name, column) =>
val sd = {
column.get(serdes).asInstanceOf[Option[String]].map(n =>
Class.forName(n).newInstance().asInstanceOf[SerDes]
)
}
val len = column.get(length).map(_.toInt).getOrElse(-1)
val sAvro = column.get(avro).map(parameters(_))
val f = Field(name, column.getOrElse(cf, rowKey),
column.get(col).get,
column.get(`type`),
sAvro, sd, len)
schemaMap.+=((name, f))
}
val rKey = RowKey(map.get(rowKey).get.asInstanceOf[String])
HBaseTableCatalog(nSpace, tName, rKey, SchemaMap(schemaMap), parameters)
}
val TABLE_KEY: String = "hbase.table"
val SCHEMA_COLUMNS_MAPPING_KEY: String = "hbase.columns.mapping"
/* for backward compatibility. Convert the old definition to new json based definition formated as below
val catalog = s"""{
|"table":{"namespace":"default", "name":"htable"},
|"rowkey":"key1:key2",
|"columns":{
|"col1":{"cf":"rowkey", "col":"key1", "type":"string"},
|"col2":{"cf":"rowkey", "col":"key2", "type":"double"},
|"col3":{"cf":"cf1", "col":"col2", "type":"binary"},
|"col4":{"cf":"cf1", "col":"col3", "type":"timestamp"},
|"col5":{"cf":"cf1", "col":"col4", "type":"double", "serdes":"${classOf[DoubleSerDes].getName}"},
|"col6":{"cf":"cf1", "col":"col5", "type":"$map"},
|"col7":{"cf":"cf1", "col":"col6", "type":"$array"},
|"col8":{"cf":"cf1", "col":"col7", "type":"$arrayMap"}
|}
|}""".stripMargin
*/
@deprecated("Please use new json format to define HBaseCatalog")
// TODO: There is no need to deprecate since this is the first release.
def convert(parameters: Map[String, String]): Map[String, String] = {
val tableName = parameters.get(TABLE_KEY).getOrElse(null)
// if the hbase.table is not defined, we assume it is json format already.
if (tableName == null) return parameters
val schemaMappingString = parameters.getOrElse(SCHEMA_COLUMNS_MAPPING_KEY, "")
import scala.collection.JavaConverters._
val schemaMap = generateSchemaMappingMap(schemaMappingString).asScala.map(_._2.asInstanceOf[SchemaQualifierDefinition])
val rowkey = schemaMap.filter {
_.columnFamily == "rowkey"
}.map(_.columnName)
val cols = schemaMap.map { x =>
s""""${x.columnName}":{"cf":"${x.columnFamily}", "col":"${x.qualifier}", "type":"${x.colType}"}""".stripMargin
}
val jsonCatalog =
s"""{
|"table":{"namespace":"default", "name":"${tableName}"},
|"rowkey":"${rowkey.mkString(":")}",
|"columns":{
|${cols.mkString(",")}
|}
|}
""".stripMargin
parameters ++ Map(HBaseTableCatalog.tableCatalog->jsonCatalog)
}
/**
* Reads the SCHEMA_COLUMNS_MAPPING_KEY and converts it to a map of
* SchemaQualifierDefinitions with the original sql column name as the key
*
* @param schemaMappingString The schema mapping string from the SparkSQL map
* @return A map of definitions keyed by the SparkSQL column name
*/
@InterfaceAudience.Private
def generateSchemaMappingMap(schemaMappingString:String):
java.util.HashMap[String, SchemaQualifierDefinition] = {
println(schemaMappingString)
try {
val columnDefinitions = schemaMappingString.split(',')
val resultingMap = new java.util.HashMap[String, SchemaQualifierDefinition]()
columnDefinitions.map(cd => {
val parts = cd.trim.split(' ')
//Make sure we get three parts
//<ColumnName> <ColumnType> <ColumnFamily:Qualifier>
if (parts.length == 3) {
val hbaseDefinitionParts = if (parts(2).charAt(0) == ':') {
Array[String]("rowkey", parts(0))
} else {
parts(2).split(':')
}
resultingMap.put(parts(0), new SchemaQualifierDefinition(parts(0),
parts(1), hbaseDefinitionParts(0), hbaseDefinitionParts(1)))
} else {
throw new IllegalArgumentException("Invalid value for schema mapping '" + cd +
"' should be '<columnName> <columnType> <columnFamily>:<qualifier>' " +
"for columns and '<columnName> <columnType> :<qualifier>' for rowKeys")
}
})
resultingMap
} catch {
case e:Exception => throw
new IllegalArgumentException("Invalid value for " + SCHEMA_COLUMNS_MAPPING_KEY +
" '" +
schemaMappingString + "'", e )
}
}
}
/**
* Construct to contains column data that spend SparkSQL and HBase
*
* @param columnName SparkSQL column name
* @param colType SparkSQL column type
* @param columnFamily HBase column family
* @param qualifier HBase qualifier name
*/
@InterfaceAudience.Private
case class SchemaQualifierDefinition(columnName:String,
colType:String,
columnFamily:String,
qualifier:String)

View File

@ -1,100 +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.spark.sql.datasources.hbase
import org.apache.hadoop.hbase.spark.AvroSerdes
import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.sql.execution.SparkSqlSerializer
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
object Utils {
/**
* Parses the hbase field to it's corresponding
* scala type which can then be put into a Spark GenericRow
* which is then automatically converted by Spark.
*/
def hbaseFieldToScalaType(
f: Field,
src: Array[Byte],
offset: Int,
length: Int): Any = {
if (f.exeSchema.isDefined) {
// If we have avro schema defined, use it to get record, and then convert them to catalyst data type
val m = AvroSerdes.deserialize(src, f.exeSchema.get)
val n = f.avroToCatalyst.map(_(m))
n.get
} else {
// Fall back to atomic type
f.dt match {
case BooleanType => toBoolean(src, offset)
case ByteType => src(offset)
case DoubleType => Bytes.toDouble(src, offset)
case FloatType => Bytes.toFloat(src, offset)
case IntegerType => Bytes.toInt(src, offset)
case LongType|TimestampType => Bytes.toLong(src, offset)
case ShortType => Bytes.toShort(src, offset)
case StringType => toUTF8String(src, offset, length)
case BinaryType =>
val newArray = new Array[Byte](length)
System.arraycopy(src, offset, newArray, 0, length)
newArray
// TODO: add more data type support
case _ => SparkSqlSerializer.deserialize[Any](src)
}
}
}
// convert input to data type
def toBytes(input: Any, field: Field): Array[Byte] = {
if (field.schema.isDefined) {
// Here we assume the top level type is structType
val record = field.catalystToAvro(input)
AvroSerdes.serialize(record, field.schema.get)
} else {
input match {
case data: Boolean => Bytes.toBytes(data)
case data: Byte => Array(data)
case data: Array[Byte] => data
case data: Double => Bytes.toBytes(data)
case data: Float => Bytes.toBytes(data)
case data: Int => Bytes.toBytes(data)
case data: Long => Bytes.toBytes(data)
case data: Short => Bytes.toBytes(data)
case data: UTF8String => data.getBytes
case data: String => Bytes.toBytes(data)
// TODO: add more data type support
case _ => throw new Exception(s"unsupported data type ${field.dt}")
}
}
}
def toBoolean(input: Array[Byte], offset: Int): Boolean = {
input(offset) != 0
}
def toUTF8String(input: Array[Byte], offset: Int, length: Int): UTF8String = {
UTF8String.fromBytes(input.slice(offset, offset + length))
}
}

View File

@ -1,520 +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.hadoop.hbase.spark;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.HashMap;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
import org.apache.hadoop.hbase.spark.example.hbasecontext.JavaHBaseBulkDeleteExample;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import scala.Tuple2;
import org.apache.hadoop.hbase.shaded.com.google.common.io.Files;
@Category({MiscTests.class, MediumTests.class})
public class TestJavaHBaseContext implements Serializable {
private transient JavaSparkContext jsc;
HBaseTestingUtility htu;
protected static final Log LOG = LogFactory.getLog(TestJavaHBaseContext.class);
byte[] tableName = Bytes.toBytes("t1");
byte[] columnFamily = Bytes.toBytes("c");
byte[] columnFamily1 = Bytes.toBytes("d");
String columnFamilyStr = Bytes.toString(columnFamily);
String columnFamilyStr1 = Bytes.toString(columnFamily1);
@Before
public void setUp() {
jsc = new JavaSparkContext("local", "JavaHBaseContextSuite");
File tempDir = Files.createTempDir();
tempDir.deleteOnExit();
htu = new HBaseTestingUtility();
try {
LOG.info("cleaning up test dir");
htu.cleanupTestDir();
LOG.info("starting minicluster");
htu.startMiniZKCluster();
htu.startMiniHBaseCluster(1, 1);
LOG.info(" - minicluster started");
try {
htu.deleteTable(TableName.valueOf(tableName));
} catch (Exception e) {
LOG.info(" - no table " + Bytes.toString(tableName) + " found");
}
LOG.info(" - creating table " + Bytes.toString(tableName));
htu.createTable(TableName.valueOf(tableName),
new byte[][]{columnFamily, columnFamily1});
LOG.info(" - created table");
} catch (Exception e1) {
throw new RuntimeException(e1);
}
}
@After
public void tearDown() {
try {
htu.deleteTable(TableName.valueOf(tableName));
LOG.info("shuting down minicluster");
htu.shutdownMiniHBaseCluster();
htu.shutdownMiniZKCluster();
LOG.info(" - minicluster shut down");
htu.cleanupTestDir();
} catch (Exception e) {
throw new RuntimeException(e);
}
jsc.stop();
jsc = null;
}
@Test
public void testBulkPut() throws IOException {
List<String> list = new ArrayList<>(5);
list.add("1," + columnFamilyStr + ",a,1");
list.add("2," + columnFamilyStr + ",a,2");
list.add("3," + columnFamilyStr + ",a,3");
list.add("4," + columnFamilyStr + ",a,4");
list.add("5," + columnFamilyStr + ",a,5");
JavaRDD<String> rdd = jsc.parallelize(list);
Configuration conf = htu.getConfiguration();
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
Connection conn = ConnectionFactory.createConnection(conf);
Table table = conn.getTable(TableName.valueOf(tableName));
try {
List<Delete> deletes = new ArrayList<>(5);
for (int i = 1; i < 6; i++) {
deletes.add(new Delete(Bytes.toBytes(Integer.toString(i))));
}
table.delete(deletes);
} finally {
table.close();
}
hbaseContext.bulkPut(rdd,
TableName.valueOf(tableName),
new PutFunction());
table = conn.getTable(TableName.valueOf(tableName));
try {
Result result1 = table.get(new Get(Bytes.toBytes("1")));
Assert.assertNotNull("Row 1 should had been deleted", result1.getRow());
Result result2 = table.get(new Get(Bytes.toBytes("2")));
Assert.assertNotNull("Row 2 should had been deleted", result2.getRow());
Result result3 = table.get(new Get(Bytes.toBytes("3")));
Assert.assertNotNull("Row 3 should had been deleted", result3.getRow());
Result result4 = table.get(new Get(Bytes.toBytes("4")));
Assert.assertNotNull("Row 4 should had been deleted", result4.getRow());
Result result5 = table.get(new Get(Bytes.toBytes("5")));
Assert.assertNotNull("Row 5 should had been deleted", result5.getRow());
} finally {
table.close();
conn.close();
}
}
public static class PutFunction implements Function<String, Put> {
private static final long serialVersionUID = 1L;
public Put call(String v) throws Exception {
String[] cells = v.split(",");
Put put = new Put(Bytes.toBytes(cells[0]));
put.addColumn(Bytes.toBytes(cells[1]), Bytes.toBytes(cells[2]),
Bytes.toBytes(cells[3]));
return put;
}
}
@Test
public void testBulkDelete() throws IOException {
List<byte[]> list = new ArrayList<>(3);
list.add(Bytes.toBytes("1"));
list.add(Bytes.toBytes("2"));
list.add(Bytes.toBytes("3"));
JavaRDD<byte[]> rdd = jsc.parallelize(list);
Configuration conf = htu.getConfiguration();
populateTableWithMockData(conf, TableName.valueOf(tableName));
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
hbaseContext.bulkDelete(rdd, TableName.valueOf(tableName),
new JavaHBaseBulkDeleteExample.DeleteFunction(), 2);
try (
Connection conn = ConnectionFactory.createConnection(conf);
Table table = conn.getTable(TableName.valueOf(tableName))
){
Result result1 = table.get(new Get(Bytes.toBytes("1")));
Assert.assertNull("Row 1 should had been deleted", result1.getRow());
Result result2 = table.get(new Get(Bytes.toBytes("2")));
Assert.assertNull("Row 2 should had been deleted", result2.getRow());
Result result3 = table.get(new Get(Bytes.toBytes("3")));
Assert.assertNull("Row 3 should had been deleted", result3.getRow());
Result result4 = table.get(new Get(Bytes.toBytes("4")));
Assert.assertNotNull("Row 4 should had been deleted", result4.getRow());
Result result5 = table.get(new Get(Bytes.toBytes("5")));
Assert.assertNotNull("Row 5 should had been deleted", result5.getRow());
}
}
@Test
public void testDistributedScan() throws IOException {
Configuration conf = htu.getConfiguration();
populateTableWithMockData(conf, TableName.valueOf(tableName));
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
Scan scan = new Scan();
scan.setCaching(100);
JavaRDD<String> javaRdd =
hbaseContext.hbaseRDD(TableName.valueOf(tableName), scan)
.map(new ScanConvertFunction());
List<String> results = javaRdd.collect();
Assert.assertEquals(results.size(), 5);
}
private static class ScanConvertFunction implements
Function<Tuple2<ImmutableBytesWritable, Result>, String> {
@Override
public String call(Tuple2<ImmutableBytesWritable, Result> v1) throws Exception {
return Bytes.toString(v1._1().copyBytes());
}
}
@Test
public void testBulkGet() throws IOException {
List<byte[]> list = new ArrayList<>(5);
list.add(Bytes.toBytes("1"));
list.add(Bytes.toBytes("2"));
list.add(Bytes.toBytes("3"));
list.add(Bytes.toBytes("4"));
list.add(Bytes.toBytes("5"));
JavaRDD<byte[]> rdd = jsc.parallelize(list);
Configuration conf = htu.getConfiguration();
populateTableWithMockData(conf, TableName.valueOf(tableName));
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
final JavaRDD<String> stringJavaRDD =
hbaseContext.bulkGet(TableName.valueOf(tableName), 2, rdd,
new GetFunction(),
new ResultFunction());
Assert.assertEquals(stringJavaRDD.count(), 5);
}
@Test
public void testBulkLoad() throws Exception {
Path output = htu.getDataTestDir("testBulkLoad");
// Add cell as String: "row,falmily,qualifier,value"
List<String> list= new ArrayList<String>();
// row1
list.add("1," + columnFamilyStr + ",b,1");
// row3
list.add("3," + columnFamilyStr + ",a,2");
list.add("3," + columnFamilyStr + ",b,1");
list.add("3," + columnFamilyStr1 + ",a,1");
//row2
list.add("2," + columnFamilyStr + ",a,3");
list.add("2," + columnFamilyStr + ",b,3");
JavaRDD<String> rdd = jsc.parallelize(list);
Configuration conf = htu.getConfiguration();
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
hbaseContext.bulkLoad(rdd, TableName.valueOf(tableName), new BulkLoadFunction(), output.toUri().getPath(),
new HashMap<byte[], FamilyHFileWriteOptions>(), false, HConstants.DEFAULT_MAX_FILE_SIZE);
try (Connection conn = ConnectionFactory.createConnection(conf); Admin admin = conn.getAdmin()) {
Table table = conn.getTable(TableName.valueOf(tableName));
// Do bulk load
LoadIncrementalHFiles load = new LoadIncrementalHFiles(conf);
load.doBulkLoad(output, admin, table, conn.getRegionLocator(TableName.valueOf(tableName)));
// Check row1
List<Cell> cell1 = table.get(new Get(Bytes.toBytes("1"))).listCells();
Assert.assertEquals(cell1.size(), 1);
Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell1.get(0))), columnFamilyStr);
Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell1.get(0))), "b");
Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell1.get(0))), "1");
// Check row3
List<Cell> cell3 = table.get(new Get(Bytes.toBytes("3"))).listCells();
Assert.assertEquals(cell3.size(), 3);
Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell3.get(0))), columnFamilyStr);
Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell3.get(0))), "a");
Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell3.get(0))), "2");
Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell3.get(1))), columnFamilyStr);
Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell3.get(1))), "b");
Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell3.get(1))), "1");
Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell3.get(2))), columnFamilyStr1);
Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell3.get(2))), "a");
Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell3.get(2))), "1");
// Check row2
List<Cell> cell2 = table.get(new Get(Bytes.toBytes("2"))).listCells();
Assert.assertEquals(cell2.size(), 2);
Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell2.get(0))), columnFamilyStr);
Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell2.get(0))), "a");
Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell2.get(0))), "3");
Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell2.get(1))), columnFamilyStr);
Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell2.get(1))), "b");
Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell2.get(1))), "3");
}
}
@Test
public void testBulkLoadThinRows() throws Exception {
Path output = htu.getDataTestDir("testBulkLoadThinRows");
// because of the limitation of scala bulkLoadThinRows API
// we need to provide data as <row, all cells in that row>
List<List<String>> list= new ArrayList<List<String>>();
// row1
List<String> list1 = new ArrayList<String>();
list1.add("1," + columnFamilyStr + ",b,1");
list.add(list1);
// row3
List<String> list3 = new ArrayList<String>();
list3.add("3," + columnFamilyStr + ",a,2");
list3.add("3," + columnFamilyStr + ",b,1");
list3.add("3," + columnFamilyStr1 + ",a,1");
list.add(list3);
//row2
List<String> list2 = new ArrayList<String>();
list2.add("2," + columnFamilyStr + ",a,3");
list2.add("2," + columnFamilyStr + ",b,3");
list.add(list2);
JavaRDD<List<String>> rdd = jsc.parallelize(list);
Configuration conf = htu.getConfiguration();
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
hbaseContext.bulkLoadThinRows(rdd, TableName.valueOf(tableName), new BulkLoadThinRowsFunction(), output.toString(),
new HashMap<byte[], FamilyHFileWriteOptions>(), false, HConstants.DEFAULT_MAX_FILE_SIZE);
try (Connection conn = ConnectionFactory.createConnection(conf); Admin admin = conn.getAdmin()) {
Table table = conn.getTable(TableName.valueOf(tableName));
// Do bulk load
LoadIncrementalHFiles load = new LoadIncrementalHFiles(conf);
load.doBulkLoad(output, admin, table, conn.getRegionLocator(TableName.valueOf(tableName)));
// Check row1
List<Cell> cell1 = table.get(new Get(Bytes.toBytes("1"))).listCells();
Assert.assertEquals(cell1.size(), 1);
Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell1.get(0))), columnFamilyStr);
Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell1.get(0))), "b");
Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell1.get(0))), "1");
// Check row3
List<Cell> cell3 = table.get(new Get(Bytes.toBytes("3"))).listCells();
Assert.assertEquals(cell3.size(), 3);
Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell3.get(0))), columnFamilyStr);
Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell3.get(0))), "a");
Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell3.get(0))), "2");
Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell3.get(1))), columnFamilyStr);
Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell3.get(1))), "b");
Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell3.get(1))), "1");
Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell3.get(2))), columnFamilyStr1);
Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell3.get(2))), "a");
Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell3.get(2))), "1");
// Check row2
List<Cell> cell2 = table.get(new Get(Bytes.toBytes("2"))).listCells();
Assert.assertEquals(cell2.size(), 2);
Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell2.get(0))), columnFamilyStr);
Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell2.get(0))), "a");
Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell2.get(0))), "3");
Assert.assertEquals(Bytes.toString(CellUtil.cloneFamily(cell2.get(1))), columnFamilyStr);
Assert.assertEquals(Bytes.toString(CellUtil.cloneQualifier(cell2.get(1))), "b");
Assert.assertEquals(Bytes.toString(CellUtil.cloneValue(cell2.get(1))), "3");
}
}
public static class BulkLoadFunction implements Function<String, Pair<KeyFamilyQualifier, byte[]>> {
@Override public Pair<KeyFamilyQualifier, byte[]> call(String v1) throws Exception {
if (v1 == null)
return null;
String[] strs = v1.split(",");
if(strs.length != 4)
return null;
KeyFamilyQualifier kfq = new KeyFamilyQualifier(Bytes.toBytes(strs[0]), Bytes.toBytes(strs[1]),
Bytes.toBytes(strs[2]));
return new Pair(kfq, Bytes.toBytes(strs[3]));
}
}
public static class BulkLoadThinRowsFunction implements Function<List<String>, Pair<ByteArrayWrapper, FamiliesQualifiersValues>> {
@Override public Pair<ByteArrayWrapper, FamiliesQualifiersValues> call(List<String> list) throws Exception {
if (list == null)
return null;
ByteArrayWrapper rowKey = null;
FamiliesQualifiersValues fqv = new FamiliesQualifiersValues();
for (String cell : list) {
String[] strs = cell.split(",");
if (rowKey == null) {
rowKey = new ByteArrayWrapper(Bytes.toBytes(strs[0]));
}
fqv.add(Bytes.toBytes(strs[1]), Bytes.toBytes(strs[2]), Bytes.toBytes(strs[3]));
}
return new Pair(rowKey, fqv);
}
}
public static class GetFunction implements Function<byte[], Get> {
private static final long serialVersionUID = 1L;
public Get call(byte[] v) throws Exception {
return new Get(v);
}
}
public static class ResultFunction implements Function<Result, String> {
private static final long serialVersionUID = 1L;
public String call(Result result) throws Exception {
Iterator<Cell> it = result.listCells().iterator();
StringBuilder b = new StringBuilder();
b.append(Bytes.toString(result.getRow())).append(":");
while (it.hasNext()) {
Cell cell = it.next();
String q = Bytes.toString(CellUtil.cloneQualifier(cell));
if ("counter".equals(q)) {
b.append("(")
.append(q)
.append(",")
.append(Bytes.toLong(CellUtil.cloneValue(cell)))
.append(")");
} else {
b.append("(")
.append(q)
.append(",")
.append(Bytes.toString(CellUtil.cloneValue(cell)))
.append(")");
}
}
return b.toString();
}
}
private void populateTableWithMockData(Configuration conf, TableName tableName)
throws IOException {
try (
Connection conn = ConnectionFactory.createConnection(conf);
Table table = conn.getTable(tableName)) {
List<Put> puts = new ArrayList<>(5);
for (int i = 1; i < 6; i++) {
Put put = new Put(Bytes.toBytes(Integer.toString(i)));
put.addColumn(columnFamily, columnFamily, columnFamily);
puts.add(put);
}
table.put(puts);
}
}
}

View File

@ -1,157 +0,0 @@
<?xml version="1.0"?>
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!--
/**
*
* 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.
*/
-->
<configuration>
<property>
<name>hbase.regionserver.msginterval</name>
<value>1000</value>
<description>Interval between messages from the RegionServer to HMaster
in milliseconds. Default is 15. Set this value low if you want unit
tests to be responsive.
</description>
</property>
<property>
<name>hbase.defaults.for.version.skip</name>
<value>true</value>
</property>
<property>
<name>hbase.server.thread.wakefrequency</name>
<value>1000</value>
<description>Time to sleep in between searches for work (in milliseconds).
Used as sleep interval by service threads such as hbase:meta scanner and log roller.
</description>
</property>
<property>
<name>hbase.master.event.waiting.time</name>
<value>50</value>
<description>Time to sleep between checks to see if a table event took place.
</description>
</property>
<property>
<name>hbase.regionserver.handler.count</name>
<value>5</value>
</property>
<property>
<name>hbase.regionserver.metahandler.count</name>
<value>5</value>
</property>
<property>
<name>hbase.ipc.server.read.threadpool.size</name>
<value>3</value>
</property>
<property>
<name>hbase.master.info.port</name>
<value>-1</value>
<description>The port for the hbase master web UI
Set to -1 if you do not want the info server to run.
</description>
</property>
<property>
<name>hbase.master.port</name>
<value>0</value>
<description>Always have masters and regionservers come up on port '0' so we don't clash over
default ports.
</description>
</property>
<property>
<name>hbase.regionserver.port</name>
<value>0</value>
<description>Always have masters and regionservers come up on port '0' so we don't clash over
default ports.
</description>
</property>
<property>
<name>hbase.ipc.client.fallback-to-simple-auth-allowed</name>
<value>true</value>
</property>
<property>
<name>hbase.regionserver.info.port</name>
<value>-1</value>
<description>The port for the hbase regionserver web UI
Set to -1 if you do not want the info server to run.
</description>
</property>
<property>
<name>hbase.regionserver.info.port.auto</name>
<value>true</value>
<description>Info server auto port bind. Enables automatic port
search if hbase.regionserver.info.port is already in use.
Enabled for testing to run multiple tests on one machine.
</description>
</property>
<property>
<name>hbase.regionserver.safemode</name>
<value>false</value>
<description>
Turn on/off safe mode in region server. Always on for production, always off
for tests.
</description>
</property>
<property>
<name>hbase.hregion.max.filesize</name>
<value>67108864</value>
<description>
Maximum desired file size for an HRegion. If filesize exceeds
value + (value / 2), the HRegion is split in two. Default: 256M.
Keep the maximum filesize small so we split more often in tests.
</description>
</property>
<property>
<name>hadoop.log.dir</name>
<value>${user.dir}/../logs</value>
</property>
<property>
<name>hbase.zookeeper.property.clientPort</name>
<value>21818</value>
<description>Property from ZooKeeper's config zoo.cfg.
The port at which the clients will connect.
</description>
</property>
<property>
<name>hbase.defaults.for.version.skip</name>
<value>true</value>
<description>
Set to true to skip the 'hbase.defaults.for.version'.
Setting this to true can be useful in contexts other than
the other side of a maven generation; i.e. running in an
ide. You'll want to set this boolean to true to avoid
seeing the RuntimeException complaint: "hbase-default.xml file
seems to be for and old version of HBase (@@@VERSION@@@), this
version is X.X.X-SNAPSHOT"
</description>
</property>
<property>
<name>hbase.table.sanity.checks</name>
<value>false</value>
<description>Skip sanity checks in tests
</description>
</property>
<property>
<name>hbase.procedure.fail.on.corruption</name>
<value>true</value>
<description>
Enable replay sanity checks on procedure tests.
</description>
</property>
</configuration>

View File

@ -1,76 +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.
# Define some default values that can be overridden by system properties
hbase.root.logger=INFO,FA
hbase.log.dir=.
hbase.log.file=hbase.log
# Define the root logger to the system property "hbase.root.logger".
log4j.rootLogger=${hbase.root.logger}
# Logging Threshold
log4j.threshold=ALL
#
# Daily Rolling File Appender
#
log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
log4j.appender.DRFA.File=${hbase.log.dir}/${hbase.log.file}
# Rollver at midnight
log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
# 30-day backup
#log4j.appender.DRFA.MaxBackupIndex=30
log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
# Debugging Pattern format
log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %C{2}(%L): %m%n
#
# console
# Add "console" to rootlogger above if you want to use this
#
log4j.appender.console=org.apache.log4j.ConsoleAppender
log4j.appender.console.target=System.err
log4j.appender.console.layout=org.apache.log4j.PatternLayout
log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %C{2}(%L): %m%n
#File Appender
log4j.appender.FA=org.apache.log4j.FileAppender
log4j.appender.FA.append=false
log4j.appender.FA.file=target/log-output.txt
log4j.appender.FA.layout=org.apache.log4j.PatternLayout
log4j.appender.FA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %C{2}(%L): %m%n
log4j.appender.FA.Threshold = INFO
# Custom Logging levels
#log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
log4j.logger.org.apache.hadoop=WARN
log4j.logger.org.apache.zookeeper=ERROR
log4j.logger.org.apache.hadoop.hbase=DEBUG
#These settings are workarounds against spurious logs from the minicluster.
#See HBASE-4709
log4j.logger.org.apache.hadoop.metrics2.impl.MetricsConfig=WARN
log4j.logger.org.apache.hadoop.metrics2.impl.MetricsSinkAdapter=WARN
log4j.logger.org.apache.hadoop.metrics2.impl.MetricsSystemImpl=WARN
log4j.logger.org.apache.hadoop.metrics2.util.MBeans=WARN
# Enable this to get detailed connection error/retry logging.
# log4j.logger.org.apache.hadoop.hbase.client.ConnectionImplementation=TRACE

View File

@ -1,956 +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.hadoop.hbase.spark
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.hbase.client.{Get, ConnectionFactory}
import org.apache.hadoop.hbase.io.hfile.{CacheConfig, HFile}
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles
import org.apache.hadoop.hbase.{HConstants, CellUtil, HBaseTestingUtility, TableName}
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._
import org.apache.spark.{SparkContext, Logging}
import org.junit.rules.TemporaryFolder
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
class BulkLoadSuite extends FunSuite with
BeforeAndAfterEach with BeforeAndAfterAll with Logging {
@transient var sc: SparkContext = null
var TEST_UTIL = new HBaseTestingUtility
val tableName = "t1"
val columnFamily1 = "f1"
val columnFamily2 = "f2"
val testFolder = new TemporaryFolder()
override def beforeAll() {
TEST_UTIL.startMiniCluster()
logInfo(" - minicluster started")
try {
TEST_UTIL.deleteTable(TableName.valueOf(tableName))
} catch {
case e: Exception =>
logInfo(" - no table " + tableName + " found")
}
logInfo(" - created table")
val envMap = Map[String,String](("Xmx", "512m"))
sc = new SparkContext("local", "test", null, Nil, envMap)
}
override def afterAll() {
logInfo("shuting down minicluster")
TEST_UTIL.shutdownMiniCluster()
logInfo(" - minicluster shut down")
TEST_UTIL.cleanupTestDir()
sc.stop()
}
test("Wide Row Bulk Load: Test multi family and multi column tests " +
"with all default HFile Configs.") {
val config = TEST_UTIL.getConfiguration
logInfo(" - creating table " + tableName)
TEST_UTIL.createTable(TableName.valueOf(tableName),
Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2)))
//There are a number of tests in here.
// 1. Row keys are not in order
// 2. Qualifiers are not in order
// 3. Column Families are not in order
// 4. There are tests for records in one column family and some in two column families
// 5. There are records will a single qualifier and some with two
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
(Bytes.toBytes("3"),
(Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo2.a"))),
(Bytes.toBytes("3"),
(Bytes.toBytes(columnFamily2), Bytes.toBytes("a"), Bytes.toBytes("foo2.b"))),
(Bytes.toBytes("3"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.c"))),
(Bytes.toBytes("5"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))),
(Bytes.toBytes("4"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))),
(Bytes.toBytes("4"),
(Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))),
(Bytes.toBytes("2"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))),
(Bytes.toBytes("2"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2")))))
val hbaseContext = new HBaseContext(sc, config)
testFolder.create()
val stagingFolder = testFolder.newFolder()
hbaseContext.bulkLoad[(Array[Byte], (Array[Byte], Array[Byte], Array[Byte]))](rdd,
TableName.valueOf(tableName),
t => {
val rowKey = t._1
val family:Array[Byte] = t._2._1
val qualifier = t._2._2
val value:Array[Byte] = t._2._3
val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier)
Seq((keyFamilyQualifier, value)).iterator
},
stagingFolder.getPath)
val fs = FileSystem.get(config)
assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 2)
val conn = ConnectionFactory.createConnection(config)
val load = new LoadIncrementalHFiles(config)
val table = conn.getTable(TableName.valueOf(tableName))
try {
load.doBulkLoad(new Path(stagingFolder.getPath), conn.getAdmin, table,
conn.getRegionLocator(TableName.valueOf(tableName)))
val cells5 = table.get(new Get(Bytes.toBytes("5"))).listCells()
assert(cells5.size == 1)
assert(Bytes.toString(CellUtil.cloneValue(cells5.get(0))).equals("foo3"))
assert(Bytes.toString(CellUtil.cloneFamily(cells5.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells5.get(0))).equals("a"))
val cells4 = table.get(new Get(Bytes.toBytes("4"))).listCells()
assert(cells4.size == 2)
assert(Bytes.toString(CellUtil.cloneValue(cells4.get(0))).equals("foo.1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells4.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells4.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells4.get(1))).equals("foo.2"))
assert(Bytes.toString(CellUtil.cloneFamily(cells4.get(1))).equals("f2"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells4.get(1))).equals("b"))
val cells3 = table.get(new Get(Bytes.toBytes("3"))).listCells()
assert(cells3.size == 3)
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(0))).equals("foo2.c"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(1))).equals("foo2.b"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(1))).equals("f2"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(1))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(2))).equals("foo2.a"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(2))).equals("f2"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(2))).equals("b"))
val cells2 = table.get(new Get(Bytes.toBytes("2"))).listCells()
assert(cells2.size == 2)
assert(Bytes.toString(CellUtil.cloneValue(cells2.get(0))).equals("bar.1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells2.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells2.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells2.get(1))).equals("bar.2"))
assert(Bytes.toString(CellUtil.cloneFamily(cells2.get(1))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells2.get(1))).equals("b"))
val cells1 = table.get(new Get(Bytes.toBytes("1"))).listCells()
assert(cells1.size == 1)
assert(Bytes.toString(CellUtil.cloneValue(cells1.get(0))).equals("foo1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells1.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells1.get(0))).equals("a"))
} finally {
table.close()
val admin = ConnectionFactory.createConnection(config).getAdmin
try {
admin.disableTable(TableName.valueOf(tableName))
admin.deleteTable(TableName.valueOf(tableName))
} finally {
admin.close()
}
fs.delete(new Path(stagingFolder.getPath), true)
testFolder.delete()
}
}
test("Wide Row Bulk Load: Test HBase client: Test Roll Over and " +
"using an implicit call to bulk load") {
val config = TEST_UTIL.getConfiguration
logInfo(" - creating table " + tableName)
TEST_UTIL.createTable(TableName.valueOf(tableName),
Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2)))
//There are a number of tests in here.
// 1. Row keys are not in order
// 2. Qualifiers are not in order
// 3. Column Families are not in order
// 4. There are tests for records in one column family and some in two column families
// 5. There are records will a single qualifier and some with two
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
(Bytes.toBytes("3"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))),
(Bytes.toBytes("3"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.a"))),
(Bytes.toBytes("3"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("c"), Bytes.toBytes("foo2.c"))),
(Bytes.toBytes("5"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))),
(Bytes.toBytes("4"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))),
(Bytes.toBytes("4"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))),
(Bytes.toBytes("2"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))),
(Bytes.toBytes("2"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2")))))
val hbaseContext = new HBaseContext(sc, config)
testFolder.create()
val stagingFolder = testFolder.newFolder()
rdd.hbaseBulkLoad(hbaseContext,
TableName.valueOf(tableName),
t => {
val rowKey = t._1
val family:Array[Byte] = t._2._1
val qualifier = t._2._2
val value = t._2._3
val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier)
Seq((keyFamilyQualifier, value)).iterator
},
stagingFolder.getPath,
new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions],
compactionExclude = false,
20)
val fs = FileSystem.get(config)
assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 1)
assert(fs.listStatus(new Path(stagingFolder.getPath+ "/f1")).length == 5)
val conn = ConnectionFactory.createConnection(config)
val load = new LoadIncrementalHFiles(config)
val table = conn.getTable(TableName.valueOf(tableName))
try {
load.doBulkLoad(new Path(stagingFolder.getPath),
conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))
val cells5 = table.get(new Get(Bytes.toBytes("5"))).listCells()
assert(cells5.size == 1)
assert(Bytes.toString(CellUtil.cloneValue(cells5.get(0))).equals("foo3"))
assert(Bytes.toString(CellUtil.cloneFamily(cells5.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells5.get(0))).equals("a"))
val cells4 = table.get(new Get(Bytes.toBytes("4"))).listCells()
assert(cells4.size == 2)
assert(Bytes.toString(CellUtil.cloneValue(cells4.get(0))).equals("foo.1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells4.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells4.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells4.get(1))).equals("foo.2"))
assert(Bytes.toString(CellUtil.cloneFamily(cells4.get(1))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells4.get(1))).equals("b"))
val cells3 = table.get(new Get(Bytes.toBytes("3"))).listCells()
assert(cells3.size == 3)
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(0))).equals("foo2.a"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(1))).equals("foo2.b"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(1))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(1))).equals("b"))
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(2))).equals("foo2.c"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(2))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(2))).equals("c"))
val cells2 = table.get(new Get(Bytes.toBytes("2"))).listCells()
assert(cells2.size == 2)
assert(Bytes.toString(CellUtil.cloneValue(cells2.get(0))).equals("bar.1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells2.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells2.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells2.get(1))).equals("bar.2"))
assert(Bytes.toString(CellUtil.cloneFamily(cells2.get(1))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells2.get(1))).equals("b"))
val cells1 = table.get(new Get(Bytes.toBytes("1"))).listCells()
assert(cells1.size == 1)
assert(Bytes.toString(CellUtil.cloneValue(cells1.get(0))).equals("foo1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells1.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells1.get(0))).equals("a"))
} finally {
table.close()
val admin = ConnectionFactory.createConnection(config).getAdmin
try {
admin.disableTable(TableName.valueOf(tableName))
admin.deleteTable(TableName.valueOf(tableName))
} finally {
admin.close()
}
fs.delete(new Path(stagingFolder.getPath), true)
testFolder.delete()
}
}
test("Wide Row Bulk Load: Test multi family and multi column tests" +
" with one column family with custom configs plus multi region") {
val config = TEST_UTIL.getConfiguration
val splitKeys:Array[Array[Byte]] = new Array[Array[Byte]](2)
splitKeys(0) = Bytes.toBytes("2")
splitKeys(1) = Bytes.toBytes("4")
logInfo(" - creating table " + tableName)
TEST_UTIL.createTable(TableName.valueOf(tableName),
Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2)),
splitKeys)
//There are a number of tests in here.
// 1. Row keys are not in order
// 2. Qualifiers are not in order
// 3. Column Families are not in order
// 4. There are tests for records in one column family and some in two column families
// 5. There are records will a single qualifier and some with two
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
(Bytes.toBytes("3"),
(Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo2.a"))),
(Bytes.toBytes("3"),
(Bytes.toBytes(columnFamily2), Bytes.toBytes("a"), Bytes.toBytes("foo2.b"))),
(Bytes.toBytes("3"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.c"))),
(Bytes.toBytes("5"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))),
(Bytes.toBytes("4"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))),
(Bytes.toBytes("4"),
(Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))),
(Bytes.toBytes("2"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))),
(Bytes.toBytes("2"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2")))))
val hbaseContext = new HBaseContext(sc, config)
testFolder.create()
val stagingFolder = testFolder.newFolder()
val familyHBaseWriterOptions = new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions]
val f1Options = new FamilyHFileWriteOptions("GZ", "ROW", 128,
"PREFIX")
familyHBaseWriterOptions.put(Bytes.toBytes(columnFamily1), f1Options)
hbaseContext.bulkLoad[(Array[Byte], (Array[Byte], Array[Byte], Array[Byte]))](rdd,
TableName.valueOf(tableName),
t => {
val rowKey = t._1
val family:Array[Byte] = t._2._1
val qualifier = t._2._2
val value = t._2._3
val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier)
Seq((keyFamilyQualifier, value)).iterator
},
stagingFolder.getPath,
familyHBaseWriterOptions,
compactionExclude = false,
HConstants.DEFAULT_MAX_FILE_SIZE)
val fs = FileSystem.get(config)
assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 2)
val f1FileList = fs.listStatus(new Path(stagingFolder.getPath +"/f1"))
for ( i <- 0 until f1FileList.length) {
val reader = HFile.createReader(fs, f1FileList(i).getPath,
new CacheConfig(config), true, config)
assert(reader.getCompressionAlgorithm.getName.equals("gz"))
assert(reader.getDataBlockEncoding.name().equals("PREFIX"))
}
assert( 3 == f1FileList.length)
val f2FileList = fs.listStatus(new Path(stagingFolder.getPath +"/f2"))
for ( i <- 0 until f2FileList.length) {
val reader = HFile.createReader(fs, f2FileList(i).getPath,
new CacheConfig(config), true, config)
assert(reader.getCompressionAlgorithm.getName.equals("none"))
assert(reader.getDataBlockEncoding.name().equals("NONE"))
}
assert( 2 == f2FileList.length)
val conn = ConnectionFactory.createConnection(config)
val load = new LoadIncrementalHFiles(config)
val table = conn.getTable(TableName.valueOf(tableName))
try {
load.doBulkLoad(new Path(stagingFolder.getPath),
conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))
val cells5 = table.get(new Get(Bytes.toBytes("5"))).listCells()
assert(cells5.size == 1)
assert(Bytes.toString(CellUtil.cloneValue(cells5.get(0))).equals("foo3"))
assert(Bytes.toString(CellUtil.cloneFamily(cells5.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells5.get(0))).equals("a"))
val cells4 = table.get(new Get(Bytes.toBytes("4"))).listCells()
assert(cells4.size == 2)
assert(Bytes.toString(CellUtil.cloneValue(cells4.get(0))).equals("foo.1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells4.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells4.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells4.get(1))).equals("foo.2"))
assert(Bytes.toString(CellUtil.cloneFamily(cells4.get(1))).equals("f2"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells4.get(1))).equals("b"))
val cells3 = table.get(new Get(Bytes.toBytes("3"))).listCells()
assert(cells3.size == 3)
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(0))).equals("foo2.c"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(1))).equals("foo2.b"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(1))).equals("f2"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(1))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(2))).equals("foo2.a"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(2))).equals("f2"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(2))).equals("b"))
val cells2 = table.get(new Get(Bytes.toBytes("2"))).listCells()
assert(cells2.size == 2)
assert(Bytes.toString(CellUtil.cloneValue(cells2.get(0))).equals("bar.1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells2.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells2.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells2.get(1))).equals("bar.2"))
assert(Bytes.toString(CellUtil.cloneFamily(cells2.get(1))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells2.get(1))).equals("b"))
val cells1 = table.get(new Get(Bytes.toBytes("1"))).listCells()
assert(cells1.size == 1)
assert(Bytes.toString(CellUtil.cloneValue(cells1.get(0))).equals("foo1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells1.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells1.get(0))).equals("a"))
} finally {
table.close()
val admin = ConnectionFactory.createConnection(config).getAdmin
try {
admin.disableTable(TableName.valueOf(tableName))
admin.deleteTable(TableName.valueOf(tableName))
} finally {
admin.close()
}
fs.delete(new Path(stagingFolder.getPath), true)
testFolder.delete()
}
}
test("Test partitioner") {
var splitKeys:Array[Array[Byte]] = new Array[Array[Byte]](3)
splitKeys(0) = Bytes.toBytes("")
splitKeys(1) = Bytes.toBytes("3")
splitKeys(2) = Bytes.toBytes("7")
var partitioner = new BulkLoadPartitioner(splitKeys)
assert(0 == partitioner.getPartition(Bytes.toBytes("")))
assert(0 == partitioner.getPartition(Bytes.toBytes("1")))
assert(0 == partitioner.getPartition(Bytes.toBytes("2")))
assert(1 == partitioner.getPartition(Bytes.toBytes("3")))
assert(1 == partitioner.getPartition(Bytes.toBytes("4")))
assert(1 == partitioner.getPartition(Bytes.toBytes("6")))
assert(2 == partitioner.getPartition(Bytes.toBytes("7")))
assert(2 == partitioner.getPartition(Bytes.toBytes("8")))
splitKeys = new Array[Array[Byte]](1)
splitKeys(0) = Bytes.toBytes("")
partitioner = new BulkLoadPartitioner(splitKeys)
assert(0 == partitioner.getPartition(Bytes.toBytes("")))
assert(0 == partitioner.getPartition(Bytes.toBytes("1")))
assert(0 == partitioner.getPartition(Bytes.toBytes("2")))
assert(0 == partitioner.getPartition(Bytes.toBytes("3")))
assert(0 == partitioner.getPartition(Bytes.toBytes("4")))
assert(0 == partitioner.getPartition(Bytes.toBytes("6")))
assert(0 == partitioner.getPartition(Bytes.toBytes("7")))
splitKeys = new Array[Array[Byte]](7)
splitKeys(0) = Bytes.toBytes("")
splitKeys(1) = Bytes.toBytes("02")
splitKeys(2) = Bytes.toBytes("04")
splitKeys(3) = Bytes.toBytes("06")
splitKeys(4) = Bytes.toBytes("08")
splitKeys(5) = Bytes.toBytes("10")
splitKeys(6) = Bytes.toBytes("12")
partitioner = new BulkLoadPartitioner(splitKeys)
assert(0 == partitioner.getPartition(Bytes.toBytes("")))
assert(0 == partitioner.getPartition(Bytes.toBytes("01")))
assert(1 == partitioner.getPartition(Bytes.toBytes("02")))
assert(1 == partitioner.getPartition(Bytes.toBytes("03")))
assert(2 == partitioner.getPartition(Bytes.toBytes("04")))
assert(2 == partitioner.getPartition(Bytes.toBytes("05")))
assert(3 == partitioner.getPartition(Bytes.toBytes("06")))
assert(3 == partitioner.getPartition(Bytes.toBytes("07")))
assert(4 == partitioner.getPartition(Bytes.toBytes("08")))
assert(4 == partitioner.getPartition(Bytes.toBytes("09")))
assert(5 == partitioner.getPartition(Bytes.toBytes("10")))
assert(5 == partitioner.getPartition(Bytes.toBytes("11")))
assert(6 == partitioner.getPartition(Bytes.toBytes("12")))
assert(6 == partitioner.getPartition(Bytes.toBytes("13")))
}
test("Thin Row Bulk Load: Test multi family and multi column tests " +
"with all default HFile Configs") {
val config = TEST_UTIL.getConfiguration
logInfo(" - creating table " + tableName)
TEST_UTIL.createTable(TableName.valueOf(tableName),
Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2)))
//There are a number of tests in here.
// 1. Row keys are not in order
// 2. Qualifiers are not in order
// 3. Column Families are not in order
// 4. There are tests for records in one column family and some in two column families
// 5. There are records will a single qualifier and some with two
val rdd = sc.parallelize(Array(
("1",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
("3",
(Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo2.a"))),
("3",
(Bytes.toBytes(columnFamily2), Bytes.toBytes("a"), Bytes.toBytes("foo2.b"))),
("3",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.c"))),
("5",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))),
("4",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))),
("4",
(Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))),
("2",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))),
("2",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))).
groupByKey()
val hbaseContext = new HBaseContext(sc, config)
testFolder.create()
val stagingFolder = testFolder.newFolder()
hbaseContext.bulkLoadThinRows[(String, Iterable[(Array[Byte], Array[Byte], Array[Byte])])](rdd,
TableName.valueOf(tableName),
t => {
val rowKey = Bytes.toBytes(t._1)
val familyQualifiersValues = new FamiliesQualifiersValues
t._2.foreach(f => {
val family:Array[Byte] = f._1
val qualifier = f._2
val value:Array[Byte] = f._3
familyQualifiersValues +=(family, qualifier, value)
})
(new ByteArrayWrapper(rowKey), familyQualifiersValues)
},
stagingFolder.getPath)
val fs = FileSystem.get(config)
assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 2)
val conn = ConnectionFactory.createConnection(config)
val load = new LoadIncrementalHFiles(config)
val table = conn.getTable(TableName.valueOf(tableName))
try {
load.doBulkLoad(new Path(stagingFolder.getPath), conn.getAdmin, table,
conn.getRegionLocator(TableName.valueOf(tableName)))
val cells5 = table.get(new Get(Bytes.toBytes("5"))).listCells()
assert(cells5.size == 1)
assert(Bytes.toString(CellUtil.cloneValue(cells5.get(0))).equals("foo3"))
assert(Bytes.toString(CellUtil.cloneFamily(cells5.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells5.get(0))).equals("a"))
val cells4 = table.get(new Get(Bytes.toBytes("4"))).listCells()
assert(cells4.size == 2)
assert(Bytes.toString(CellUtil.cloneValue(cells4.get(0))).equals("foo.1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells4.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells4.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells4.get(1))).equals("foo.2"))
assert(Bytes.toString(CellUtil.cloneFamily(cells4.get(1))).equals("f2"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells4.get(1))).equals("b"))
val cells3 = table.get(new Get(Bytes.toBytes("3"))).listCells()
assert(cells3.size == 3)
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(0))).equals("foo2.c"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(1))).equals("foo2.b"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(1))).equals("f2"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(1))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(2))).equals("foo2.a"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(2))).equals("f2"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(2))).equals("b"))
val cells2 = table.get(new Get(Bytes.toBytes("2"))).listCells()
assert(cells2.size == 2)
assert(Bytes.toString(CellUtil.cloneValue(cells2.get(0))).equals("bar.1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells2.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells2.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells2.get(1))).equals("bar.2"))
assert(Bytes.toString(CellUtil.cloneFamily(cells2.get(1))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells2.get(1))).equals("b"))
val cells1 = table.get(new Get(Bytes.toBytes("1"))).listCells()
assert(cells1.size == 1)
assert(Bytes.toString(CellUtil.cloneValue(cells1.get(0))).equals("foo1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells1.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells1.get(0))).equals("a"))
} finally {
table.close()
val admin = ConnectionFactory.createConnection(config).getAdmin
try {
admin.disableTable(TableName.valueOf(tableName))
admin.deleteTable(TableName.valueOf(tableName))
} finally {
admin.close()
}
fs.delete(new Path(stagingFolder.getPath), true)
testFolder.delete()
}
}
test("Thin Row Bulk Load: Test HBase client: Test Roll Over and " +
"using an implicit call to bulk load") {
val config = TEST_UTIL.getConfiguration
logInfo(" - creating table " + tableName)
TEST_UTIL.createTable(TableName.valueOf(tableName),
Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2)))
//There are a number of tests in here.
// 1. Row keys are not in order
// 2. Qualifiers are not in order
// 3. Column Families are not in order
// 4. There are tests for records in one column family and some in two column families
// 5. There are records will a single qualifier and some with two
val rdd = sc.parallelize(Array(
("1",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
("3",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))),
("3",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.a"))),
("3",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("c"), Bytes.toBytes("foo2.c"))),
("5",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))),
("4",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))),
("4",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))),
("2",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))),
("2",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))).
groupByKey()
val hbaseContext = new HBaseContext(sc, config)
testFolder.create()
val stagingFolder = testFolder.newFolder()
rdd.hbaseBulkLoadThinRows(hbaseContext,
TableName.valueOf(tableName),
t => {
val rowKey = t._1
val familyQualifiersValues = new FamiliesQualifiersValues
t._2.foreach(f => {
val family:Array[Byte] = f._1
val qualifier = f._2
val value:Array[Byte] = f._3
familyQualifiersValues +=(family, qualifier, value)
})
(new ByteArrayWrapper(Bytes.toBytes(rowKey)), familyQualifiersValues)
},
stagingFolder.getPath,
new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions],
compactionExclude = false,
20)
val fs = FileSystem.get(config)
assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 1)
assert(fs.listStatus(new Path(stagingFolder.getPath+ "/f1")).length == 5)
val conn = ConnectionFactory.createConnection(config)
val load = new LoadIncrementalHFiles(config)
val table = conn.getTable(TableName.valueOf(tableName))
try {
load.doBulkLoad(new Path(stagingFolder.getPath),
conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))
val cells5 = table.get(new Get(Bytes.toBytes("5"))).listCells()
assert(cells5.size == 1)
assert(Bytes.toString(CellUtil.cloneValue(cells5.get(0))).equals("foo3"))
assert(Bytes.toString(CellUtil.cloneFamily(cells5.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells5.get(0))).equals("a"))
val cells4 = table.get(new Get(Bytes.toBytes("4"))).listCells()
assert(cells4.size == 2)
assert(Bytes.toString(CellUtil.cloneValue(cells4.get(0))).equals("foo.1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells4.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells4.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells4.get(1))).equals("foo.2"))
assert(Bytes.toString(CellUtil.cloneFamily(cells4.get(1))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells4.get(1))).equals("b"))
val cells3 = table.get(new Get(Bytes.toBytes("3"))).listCells()
assert(cells3.size == 3)
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(0))).equals("foo2.a"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(1))).equals("foo2.b"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(1))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(1))).equals("b"))
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(2))).equals("foo2.c"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(2))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(2))).equals("c"))
val cells2 = table.get(new Get(Bytes.toBytes("2"))).listCells()
assert(cells2.size == 2)
assert(Bytes.toString(CellUtil.cloneValue(cells2.get(0))).equals("bar.1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells2.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells2.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells2.get(1))).equals("bar.2"))
assert(Bytes.toString(CellUtil.cloneFamily(cells2.get(1))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells2.get(1))).equals("b"))
val cells1 = table.get(new Get(Bytes.toBytes("1"))).listCells()
assert(cells1.size == 1)
assert(Bytes.toString(CellUtil.cloneValue(cells1.get(0))).equals("foo1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells1.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells1.get(0))).equals("a"))
} finally {
table.close()
val admin = ConnectionFactory.createConnection(config).getAdmin
try {
admin.disableTable(TableName.valueOf(tableName))
admin.deleteTable(TableName.valueOf(tableName))
} finally {
admin.close()
}
fs.delete(new Path(stagingFolder.getPath), true)
testFolder.delete()
}
}
test("Thin Row Bulk Load: Test multi family and multi column tests" +
" with one column family with custom configs plus multi region") {
val config = TEST_UTIL.getConfiguration
val splitKeys:Array[Array[Byte]] = new Array[Array[Byte]](2)
splitKeys(0) = Bytes.toBytes("2")
splitKeys(1) = Bytes.toBytes("4")
logInfo(" - creating table " + tableName)
TEST_UTIL.createTable(TableName.valueOf(tableName),
Array(Bytes.toBytes(columnFamily1), Bytes.toBytes(columnFamily2)),
splitKeys)
//There are a number of tests in here.
// 1. Row keys are not in order
// 2. Qualifiers are not in order
// 3. Column Families are not in order
// 4. There are tests for records in one column family and some in two column families
// 5. There are records will a single qualifier and some with two
val rdd = sc.parallelize(Array(
("1",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
("3",
(Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo2.a"))),
("3",
(Bytes.toBytes(columnFamily2), Bytes.toBytes("a"), Bytes.toBytes("foo2.b"))),
("3",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo2.c"))),
("5",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo3"))),
("4",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo.1"))),
("4",
(Bytes.toBytes(columnFamily2), Bytes.toBytes("b"), Bytes.toBytes("foo.2"))),
("2",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("bar.1"))),
("2",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("bar.2"))))).
groupByKey()
val hbaseContext = new HBaseContext(sc, config)
testFolder.create()
val stagingFolder = testFolder.newFolder()
val familyHBaseWriterOptions = new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions]
val f1Options = new FamilyHFileWriteOptions("GZ", "ROW", 128,
"PREFIX")
familyHBaseWriterOptions.put(Bytes.toBytes(columnFamily1), f1Options)
hbaseContext.bulkLoadThinRows[(String, Iterable[(Array[Byte], Array[Byte], Array[Byte])])](rdd,
TableName.valueOf(tableName),
t => {
val rowKey = t._1
val familyQualifiersValues = new FamiliesQualifiersValues
t._2.foreach(f => {
val family:Array[Byte] = f._1
val qualifier = f._2
val value:Array[Byte] = f._3
familyQualifiersValues +=(family, qualifier, value)
})
(new ByteArrayWrapper(Bytes.toBytes(rowKey)), familyQualifiersValues)
},
stagingFolder.getPath,
familyHBaseWriterOptions,
compactionExclude = false,
HConstants.DEFAULT_MAX_FILE_SIZE)
val fs = FileSystem.get(config)
assert(fs.listStatus(new Path(stagingFolder.getPath)).length == 2)
val f1FileList = fs.listStatus(new Path(stagingFolder.getPath +"/f1"))
for ( i <- 0 until f1FileList.length) {
val reader = HFile.createReader(fs, f1FileList(i).getPath,
new CacheConfig(config), true, config)
assert(reader.getCompressionAlgorithm.getName.equals("gz"))
assert(reader.getDataBlockEncoding.name().equals("PREFIX"))
}
assert( 3 == f1FileList.length)
val f2FileList = fs.listStatus(new Path(stagingFolder.getPath +"/f2"))
for ( i <- 0 until f2FileList.length) {
val reader = HFile.createReader(fs, f2FileList(i).getPath,
new CacheConfig(config), true, config)
assert(reader.getCompressionAlgorithm.getName.equals("none"))
assert(reader.getDataBlockEncoding.name().equals("NONE"))
}
assert( 2 == f2FileList.length)
val conn = ConnectionFactory.createConnection(config)
val load = new LoadIncrementalHFiles(config)
val table = conn.getTable(TableName.valueOf(tableName))
try {
load.doBulkLoad(new Path(stagingFolder.getPath),
conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))
val cells5 = table.get(new Get(Bytes.toBytes("5"))).listCells()
assert(cells5.size == 1)
assert(Bytes.toString(CellUtil.cloneValue(cells5.get(0))).equals("foo3"))
assert(Bytes.toString(CellUtil.cloneFamily(cells5.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells5.get(0))).equals("a"))
val cells4 = table.get(new Get(Bytes.toBytes("4"))).listCells()
assert(cells4.size == 2)
assert(Bytes.toString(CellUtil.cloneValue(cells4.get(0))).equals("foo.1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells4.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells4.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells4.get(1))).equals("foo.2"))
assert(Bytes.toString(CellUtil.cloneFamily(cells4.get(1))).equals("f2"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells4.get(1))).equals("b"))
val cells3 = table.get(new Get(Bytes.toBytes("3"))).listCells()
assert(cells3.size == 3)
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(0))).equals("foo2.c"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(1))).equals("foo2.b"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(1))).equals("f2"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(1))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells3.get(2))).equals("foo2.a"))
assert(Bytes.toString(CellUtil.cloneFamily(cells3.get(2))).equals("f2"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells3.get(2))).equals("b"))
val cells2 = table.get(new Get(Bytes.toBytes("2"))).listCells()
assert(cells2.size == 2)
assert(Bytes.toString(CellUtil.cloneValue(cells2.get(0))).equals("bar.1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells2.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells2.get(0))).equals("a"))
assert(Bytes.toString(CellUtil.cloneValue(cells2.get(1))).equals("bar.2"))
assert(Bytes.toString(CellUtil.cloneFamily(cells2.get(1))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells2.get(1))).equals("b"))
val cells1 = table.get(new Get(Bytes.toBytes("1"))).listCells()
assert(cells1.size == 1)
assert(Bytes.toString(CellUtil.cloneValue(cells1.get(0))).equals("foo1"))
assert(Bytes.toString(CellUtil.cloneFamily(cells1.get(0))).equals("f1"))
assert(Bytes.toString(CellUtil.cloneQualifier(cells1.get(0))).equals("a"))
} finally {
table.close()
val admin = ConnectionFactory.createConnection(config).getAdmin
try {
admin.disableTable(TableName.valueOf(tableName))
admin.deleteTable(TableName.valueOf(tableName))
} finally {
admin.close()
}
fs.delete(new Path(stagingFolder.getPath), true)
testFolder.delete()
}
}
}

View File

@ -1,339 +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.hadoop.hbase.spark
import java.util
import org.apache.hadoop.hbase.spark.datasources.{HBaseSparkConf, JavaBytesEncoder}
import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.Logging
import org.apache.spark.sql.types._
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
class DynamicLogicExpressionSuite extends FunSuite with
BeforeAndAfterEach with BeforeAndAfterAll with Logging {
val encoder = JavaBytesEncoder.create(HBaseSparkConf.DEFAULT_QUERY_ENCODER)
test("Basic And Test") {
val leftLogic = new LessThanLogicExpression("Col1", 0)
leftLogic.setEncoder(encoder)
val rightLogic = new GreaterThanLogicExpression("Col1", 1)
rightLogic.setEncoder(encoder)
val andLogic = new AndLogicExpression(leftLogic, rightLogic)
val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(10)))
val valueFromQueryValueArray = new Array[Array[Byte]](2)
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
assert(andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10)
assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
val expressionString = andLogic.toExpressionString
assert(expressionString.equals("( Col1 < 0 AND Col1 > 1 )"))
val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder)
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10)
assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
}
test("Basic OR Test") {
val leftLogic = new LessThanLogicExpression("Col1", 0)
leftLogic.setEncoder(encoder)
val rightLogic = new GreaterThanLogicExpression("Col1", 1)
rightLogic.setEncoder(encoder)
val OrLogic = new OrLogicExpression(leftLogic, rightLogic)
val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(10)))
val valueFromQueryValueArray = new Array[Array[Byte]](2)
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
assert(OrLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
assert(OrLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10)
assert(OrLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10)
assert(!OrLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
val expressionString = OrLogic.toExpressionString
assert(expressionString.equals("( Col1 < 0 OR Col1 > 1 )"))
val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder)
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5)
assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10)
assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10)
assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
}
test("Basic Command Test") {
val greaterLogic = new GreaterThanLogicExpression("Col1", 0)
greaterLogic.setEncoder(encoder)
val greaterAndEqualLogic = new GreaterThanOrEqualLogicExpression("Col1", 0)
greaterAndEqualLogic.setEncoder(encoder)
val lessLogic = new LessThanLogicExpression("Col1", 0)
lessLogic.setEncoder(encoder)
val lessAndEqualLogic = new LessThanOrEqualLogicExpression("Col1", 0)
lessAndEqualLogic.setEncoder(encoder)
val equalLogic = new EqualLogicExpression("Col1", 0, false)
val notEqualLogic = new EqualLogicExpression("Col1", 0, true)
val passThrough = new PassThroughLogicExpression
val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(10)))
val valueFromQueryValueArray = new Array[Array[Byte]](1)
//great than
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
assert(!greaterLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20)
assert(!greaterLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
//great than and equal
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 5)
assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap,
valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap,
valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20)
assert(!greaterAndEqualLogic.execute(columnToCurrentRowValueMap,
valueFromQueryValueArray))
//less than
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
assert(!lessLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 5)
assert(!lessLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
//less than and equal
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20)
assert(lessAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20)
assert(lessAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10)
assert(lessAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
//equal too
valueFromQueryValueArray(0) = Bytes.toBytes(10)
assert(equalLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = Bytes.toBytes(5)
assert(!equalLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
//not equal too
valueFromQueryValueArray(0) = Bytes.toBytes(10)
assert(!notEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = Bytes.toBytes(5)
assert(notEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
//pass through
valueFromQueryValueArray(0) = Bytes.toBytes(10)
assert(passThrough.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = Bytes.toBytes(5)
assert(passThrough.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
}
test("Double Type") {
val leftLogic = new LessThanLogicExpression("Col1", 0)
leftLogic.setEncoder(encoder)
val rightLogic = new GreaterThanLogicExpression("Col1", 1)
rightLogic.setEncoder(encoder)
val andLogic = new AndLogicExpression(leftLogic, rightLogic)
val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(-4.0d)))
val valueFromQueryValueArray = new Array[Array[Byte]](2)
valueFromQueryValueArray(0) = encoder.encode(DoubleType, 15.0d)
valueFromQueryValueArray(1) = encoder.encode(DoubleType, -5.0d)
assert(andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(DoubleType, 10.0d)
valueFromQueryValueArray(1) = encoder.encode(DoubleType, -1.0d)
assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(DoubleType, -10.0d)
valueFromQueryValueArray(1) = encoder.encode(DoubleType, -20.0d)
assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
val expressionString = andLogic.toExpressionString
// Note that here 0 and 1 is index, instead of value.
assert(expressionString.equals("( Col1 < 0 AND Col1 > 1 )"))
val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder)
valueFromQueryValueArray(0) = encoder.encode(DoubleType, 15.0d)
valueFromQueryValueArray(1) = encoder.encode(DoubleType, -5.0d)
assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(DoubleType, 10.0d)
valueFromQueryValueArray(1) = encoder.encode(DoubleType, -1.0d)
assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(DoubleType, -10.0d)
valueFromQueryValueArray(1) = encoder.encode(DoubleType, -20.0d)
assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
}
test("Float Type") {
val leftLogic = new LessThanLogicExpression("Col1", 0)
leftLogic.setEncoder(encoder)
val rightLogic = new GreaterThanLogicExpression("Col1", 1)
rightLogic.setEncoder(encoder)
val andLogic = new AndLogicExpression(leftLogic, rightLogic)
val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(-4.0f)))
val valueFromQueryValueArray = new Array[Array[Byte]](2)
valueFromQueryValueArray(0) = encoder.encode(FloatType, 15.0f)
valueFromQueryValueArray(1) = encoder.encode(FloatType, -5.0f)
assert(andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(FloatType, 10.0f)
valueFromQueryValueArray(1) = encoder.encode(FloatType, -1.0f)
assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(FloatType, -10.0f)
valueFromQueryValueArray(1) = encoder.encode(FloatType, -20.0f)
assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
val expressionString = andLogic.toExpressionString
// Note that here 0 and 1 is index, instead of value.
assert(expressionString.equals("( Col1 < 0 AND Col1 > 1 )"))
val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder)
valueFromQueryValueArray(0) = encoder.encode(FloatType, 15.0f)
valueFromQueryValueArray(1) = encoder.encode(FloatType, -5.0f)
assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(FloatType, 10.0f)
valueFromQueryValueArray(1) = encoder.encode(FloatType, -1.0f)
assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(FloatType, -10.0f)
valueFromQueryValueArray(1) = encoder.encode(FloatType, -20.0f)
assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
}
test("String Type") {
val leftLogic = new LessThanLogicExpression("Col1", 0)
leftLogic.setEncoder(encoder)
val rightLogic = new GreaterThanLogicExpression("Col1", 1)
rightLogic.setEncoder(encoder)
val andLogic = new AndLogicExpression(leftLogic, rightLogic)
val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes("row005")))
val valueFromQueryValueArray = new Array[Array[Byte]](2)
valueFromQueryValueArray(0) = encoder.encode(StringType, "row015")
valueFromQueryValueArray(1) = encoder.encode(StringType, "row000")
assert(andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(StringType, "row004")
valueFromQueryValueArray(1) = encoder.encode(StringType, "row000")
assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(StringType, "row020")
valueFromQueryValueArray(1) = encoder.encode(StringType, "row010")
assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
val expressionString = andLogic.toExpressionString
// Note that here 0 and 1 is index, instead of value.
assert(expressionString.equals("( Col1 < 0 AND Col1 > 1 )"))
val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder)
valueFromQueryValueArray(0) = encoder.encode(StringType, "row015")
valueFromQueryValueArray(1) = encoder.encode(StringType, "row000")
assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(StringType, "row004")
valueFromQueryValueArray(1) = encoder.encode(StringType, "row000")
assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
valueFromQueryValueArray(0) = encoder.encode(StringType, "row020")
valueFromQueryValueArray(1) = encoder.encode(StringType, "row010")
assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
}
test("Boolean Type") {
val leftLogic = new LessThanLogicExpression("Col1", 0)
leftLogic.setEncoder(encoder)
val rightLogic = new GreaterThanLogicExpression("Col1", 1)
rightLogic.setEncoder(encoder)
val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]()
columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(false)))
val valueFromQueryValueArray = new Array[Array[Byte]](2)
valueFromQueryValueArray(0) = encoder.encode(BooleanType, true)
valueFromQueryValueArray(1) = encoder.encode(BooleanType, false)
assert(leftLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
assert(!rightLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray))
}
}

View File

@ -1,111 +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.hadoop.hbase.spark
import org.apache.hadoop.hbase.spark.datasources.{DoubleSerDes, SerDes}
import org.apache.hadoop.hbase.util.Bytes
import org.apache.spark.Logging
import org.apache.spark.sql.datasources.hbase.{DataTypeParserWrapper, HBaseTableCatalog}
import org.apache.spark.sql.types._
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
class HBaseCatalogSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll with Logging {
val map = s"""MAP<int, struct<varchar:string>>"""
val array = s"""array<struct<tinYint:tinyint>>"""
val arrayMap = s"""MAp<int, ARRAY<double>>"""
val catalog = s"""{
|"table":{"namespace":"default", "name":"htable"},
|"rowkey":"key1:key2",
|"columns":{
|"col1":{"cf":"rowkey", "col":"key1", "type":"string"},
|"col2":{"cf":"rowkey", "col":"key2", "type":"double"},
|"col3":{"cf":"cf1", "col":"col2", "type":"binary"},
|"col4":{"cf":"cf1", "col":"col3", "type":"timestamp"},
|"col5":{"cf":"cf1", "col":"col4", "type":"double", "serdes":"${classOf[DoubleSerDes].getName}"},
|"col6":{"cf":"cf1", "col":"col5", "type":"$map"},
|"col7":{"cf":"cf1", "col":"col6", "type":"$array"},
|"col8":{"cf":"cf1", "col":"col7", "type":"$arrayMap"}
|}
|}""".stripMargin
val parameters = Map(HBaseTableCatalog.tableCatalog->catalog)
val t = HBaseTableCatalog(parameters)
def checkDataType(dataTypeString: String, expectedDataType: DataType): Unit = {
test(s"parse ${dataTypeString.replace("\n", "")}") {
assert(DataTypeParserWrapper.parse(dataTypeString) === expectedDataType)
}
}
test("basic") {
assert(t.getField("col1").isRowKey == true)
assert(t.getPrimaryKey == "key1")
assert(t.getField("col3").dt == BinaryType)
assert(t.getField("col4").dt == TimestampType)
assert(t.getField("col5").dt == DoubleType)
assert(t.getField("col5").serdes != None)
assert(t.getField("col4").serdes == None)
assert(t.getField("col1").isRowKey)
assert(t.getField("col2").isRowKey)
assert(!t.getField("col3").isRowKey)
assert(t.getField("col2").length == Bytes.SIZEOF_DOUBLE)
assert(t.getField("col1").length == -1)
assert(t.getField("col8").length == -1)
}
checkDataType(
map,
t.getField("col6").dt
)
checkDataType(
array,
t.getField("col7").dt
)
checkDataType(
arrayMap,
t.getField("col8").dt
)
test("convert") {
val m = Map("hbase.columns.mapping" ->
"KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD DOUBLE c:b, C_FIELD BINARY c:c,",
"hbase.table" -> "t1")
val map = HBaseTableCatalog.convert(m)
val json = map.get(HBaseTableCatalog.tableCatalog).get
val parameters = Map(HBaseTableCatalog.tableCatalog->json)
val t = HBaseTableCatalog(parameters)
assert(t.getField("KEY_FIELD").isRowKey)
assert(DataTypeParserWrapper.parse("STRING") === t.getField("A_FIELD").dt)
assert(!t.getField("A_FIELD").isRowKey)
assert(DataTypeParserWrapper.parse("DOUBLE") === t.getField("B_FIELD").dt)
assert(DataTypeParserWrapper.parse("BINARY") === t.getField("C_FIELD").dt)
}
test("compatiblity") {
val m = Map("hbase.columns.mapping" ->
"KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD DOUBLE c:b, C_FIELD BINARY c:c,",
"hbase.table" -> "t1")
val t = HBaseTableCatalog(m)
assert(t.getField("KEY_FIELD").isRowKey)
assert(DataTypeParserWrapper.parse("STRING") === t.getField("A_FIELD").dt)
assert(!t.getField("A_FIELD").isRowKey)
assert(DataTypeParserWrapper.parse("DOUBLE") === t.getField("B_FIELD").dt)
assert(DataTypeParserWrapper.parse("BINARY") === t.getField("C_FIELD").dt)
}
}

View File

@ -1,237 +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.hadoop.hbase.spark
import java.util.concurrent.ExecutorService
import scala.util.Random
import org.apache.hadoop.hbase.client.{BufferedMutator, Table, RegionLocator,
Connection, BufferedMutatorParams, Admin, TableBuilder}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.hbase.TableName
import org.apache.spark.Logging
import org.scalatest.FunSuite
case class HBaseConnectionKeyMocker (confId: Int) extends HBaseConnectionKey (null) {
override def hashCode: Int = {
confId
}
override def equals(obj: Any): Boolean = {
if(!obj.isInstanceOf[HBaseConnectionKeyMocker])
false
else
confId == obj.asInstanceOf[HBaseConnectionKeyMocker].confId
}
}
class ConnectionMocker extends Connection {
var isClosed: Boolean = false
def getRegionLocator (tableName: TableName): RegionLocator = null
def getConfiguration: Configuration = null
def getTable (tableName: TableName): Table = null
def getTable(tableName: TableName, pool: ExecutorService): Table = null
def getBufferedMutator (params: BufferedMutatorParams): BufferedMutator = null
def getBufferedMutator (tableName: TableName): BufferedMutator = null
def getAdmin: Admin = null
def getTableBuilder(tableName: TableName, pool: ExecutorService): TableBuilder = null
def close(): Unit = {
if (isClosed)
throw new IllegalStateException()
isClosed = true
}
def isAborted: Boolean = true
def abort(why: String, e: Throwable) = {}
}
class HBaseConnectionCacheSuite extends FunSuite with Logging {
/*
* These tests must be performed sequentially as they operate with an
* unique running thread and resource.
*
* It looks there's no way to tell FunSuite to do so, so making those
* test cases normal functions which are called sequentially in a single
* test case.
*/
test("all test cases") {
testBasic()
testWithPressureWithoutClose()
testWithPressureWithClose()
}
def cleanEnv() {
HBaseConnectionCache.connectionMap.synchronized {
HBaseConnectionCache.connectionMap.clear()
HBaseConnectionCache.cacheStat.numActiveConnections = 0
HBaseConnectionCache.cacheStat.numActualConnectionsCreated = 0
HBaseConnectionCache.cacheStat.numTotalRequests = 0
}
}
def testBasic() {
cleanEnv()
HBaseConnectionCache.setTimeout(1 * 1000)
val connKeyMocker1 = new HBaseConnectionKeyMocker(1)
val connKeyMocker1a = new HBaseConnectionKeyMocker(1)
val connKeyMocker2 = new HBaseConnectionKeyMocker(2)
val c1 = HBaseConnectionCache
.getConnection(connKeyMocker1, new ConnectionMocker)
assert(HBaseConnectionCache.connectionMap.size === 1)
assert(HBaseConnectionCache.getStat.numTotalRequests === 1)
assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 1)
assert(HBaseConnectionCache.getStat.numActiveConnections === 1)
val c1a = HBaseConnectionCache
.getConnection(connKeyMocker1a, new ConnectionMocker)
HBaseConnectionCache.connectionMap.synchronized {
assert(HBaseConnectionCache.connectionMap.size === 1)
assert(HBaseConnectionCache.getStat.numTotalRequests === 2)
assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 1)
assert(HBaseConnectionCache.getStat.numActiveConnections === 1)
}
val c2 = HBaseConnectionCache
.getConnection(connKeyMocker2, new ConnectionMocker)
HBaseConnectionCache.connectionMap.synchronized {
assert(HBaseConnectionCache.connectionMap.size === 2)
assert(HBaseConnectionCache.getStat.numTotalRequests === 3)
assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 2)
assert(HBaseConnectionCache.getStat.numActiveConnections === 2)
}
c1.close()
HBaseConnectionCache.connectionMap.synchronized {
assert(HBaseConnectionCache.connectionMap.size === 2)
assert(HBaseConnectionCache.getStat.numActiveConnections === 2)
}
c1a.close()
HBaseConnectionCache.connectionMap.synchronized {
assert(HBaseConnectionCache.connectionMap.size === 2)
assert(HBaseConnectionCache.getStat.numActiveConnections === 2)
}
Thread.sleep(3 * 1000) // Leave housekeeping thread enough time
HBaseConnectionCache.connectionMap.synchronized {
assert(HBaseConnectionCache.connectionMap.size === 1)
assert(HBaseConnectionCache.connectionMap.iterator.next()._1
.asInstanceOf[HBaseConnectionKeyMocker].confId === 2)
assert(HBaseConnectionCache.getStat.numActiveConnections === 1)
}
c2.close()
}
def testWithPressureWithoutClose() {
cleanEnv()
class TestThread extends Runnable {
override def run() {
for (i <- 0 to 999) {
val c = HBaseConnectionCache.getConnection(
new HBaseConnectionKeyMocker(Random.nextInt(10)), new ConnectionMocker)
}
}
}
HBaseConnectionCache.setTimeout(500)
val threads: Array[Thread] = new Array[Thread](100)
for (i <- 0 to 99) {
threads.update(i, new Thread(new TestThread()))
threads(i).run()
}
try {
threads.foreach { x => x.join() }
} catch {
case e: InterruptedException => println(e.getMessage)
}
Thread.sleep(1000)
HBaseConnectionCache.connectionMap.synchronized {
assert(HBaseConnectionCache.connectionMap.size === 10)
assert(HBaseConnectionCache.getStat.numTotalRequests === 100 * 1000)
assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 10)
assert(HBaseConnectionCache.getStat.numActiveConnections === 10)
var totalRc : Int = 0
HBaseConnectionCache.connectionMap.foreach {
x => totalRc += x._2.refCount
}
assert(totalRc === 100 * 1000)
HBaseConnectionCache.connectionMap.foreach {
x => {
x._2.refCount = 0
x._2.timestamp = System.currentTimeMillis() - 1000
}
}
}
Thread.sleep(1000)
assert(HBaseConnectionCache.connectionMap.size === 0)
assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 10)
assert(HBaseConnectionCache.getStat.numActiveConnections === 0)
}
def testWithPressureWithClose() {
cleanEnv()
class TestThread extends Runnable {
override def run() {
for (i <- 0 to 999) {
val c = HBaseConnectionCache.getConnection(
new HBaseConnectionKeyMocker(Random.nextInt(10)), new ConnectionMocker)
Thread.`yield`()
c.close()
}
}
}
HBaseConnectionCache.setTimeout(3 * 1000)
val threads: Array[Thread] = new Array[Thread](100)
for (i <- threads.indices) {
threads.update(i, new Thread(new TestThread()))
threads(i).run()
}
try {
threads.foreach { x => x.join() }
} catch {
case e: InterruptedException => println(e.getMessage)
}
HBaseConnectionCache.connectionMap.synchronized {
assert(HBaseConnectionCache.connectionMap.size === 10)
assert(HBaseConnectionCache.getStat.numTotalRequests === 100 * 1000)
assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 10)
assert(HBaseConnectionCache.getStat.numActiveConnections === 10)
}
Thread.sleep(6 * 1000)
HBaseConnectionCache.connectionMap.synchronized {
assert(HBaseConnectionCache.connectionMap.size === 0)
assert(HBaseConnectionCache.getStat.numActualConnectionsCreated === 10)
assert(HBaseConnectionCache.getStat.numActiveConnections === 0)
}
}
}

View File

@ -1,356 +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.hadoop.hbase.spark
import org.apache.hadoop.hbase.client._
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.{ CellUtil, TableName, HBaseTestingUtility}
import org.apache.spark.{SparkException, Logging, SparkContext}
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
class HBaseContextSuite extends FunSuite with
BeforeAndAfterEach with BeforeAndAfterAll with Logging {
@transient var sc: SparkContext = null
var TEST_UTIL = new HBaseTestingUtility
val tableName = "t1"
val columnFamily = "c"
override def beforeAll() {
TEST_UTIL.startMiniCluster()
logInfo(" - minicluster started")
try {
TEST_UTIL.deleteTable(TableName.valueOf(tableName))
} catch {
case e: Exception =>
logInfo(" - no table " + tableName + " found")
}
logInfo(" - creating table " + tableName)
TEST_UTIL.createTable(TableName.valueOf(tableName), Bytes.toBytes(columnFamily))
logInfo(" - created table")
val envMap = Map[String,String](("Xmx", "512m"))
sc = new SparkContext("local", "test", null, Nil, envMap)
}
override def afterAll() {
logInfo("shuting down minicluster")
TEST_UTIL.shutdownMiniCluster()
logInfo(" - minicluster shut down")
TEST_UTIL.cleanupTestDir()
sc.stop()
}
test("bulkput to test HBase client") {
val config = TEST_UTIL.getConfiguration
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))),
(Bytes.toBytes("2"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))),
(Bytes.toBytes("3"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))),
(Bytes.toBytes("4"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))),
(Bytes.toBytes("5"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar"))))))
val hbaseContext = new HBaseContext(sc, config)
hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])](rdd,
TableName.valueOf(tableName),
(putRecord) => {
val put = new Put(putRecord._1)
putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
put
})
val connection = ConnectionFactory.createConnection(config)
val table = connection.getTable(TableName.valueOf("t1"))
try {
val foo1 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("1"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a"))))
assert(foo1 == "foo1")
val foo2 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("2"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b"))))
assert(foo2 == "foo2")
val foo3 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("3"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c"))))
assert(foo3 == "foo3")
val foo4 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("4"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d"))))
assert(foo4 == "foo")
val foo5 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("5"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e"))))
assert(foo5 == "bar")
} finally {
table.close()
connection.close()
}
}
test("bulkDelete to test HBase client") {
val config = TEST_UTIL.getConfiguration
val connection = ConnectionFactory.createConnection(config)
val table = connection.getTable(TableName.valueOf("t1"))
try {
var put = new Put(Bytes.toBytes("delete1"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
table.put(put)
put = new Put(Bytes.toBytes("delete2"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
table.put(put)
put = new Put(Bytes.toBytes("delete3"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
table.put(put)
val rdd = sc.parallelize(Array(
Bytes.toBytes("delete1"),
Bytes.toBytes("delete3")))
val hbaseContext = new HBaseContext(sc, config)
hbaseContext.bulkDelete[Array[Byte]](rdd,
TableName.valueOf(tableName),
putRecord => new Delete(putRecord),
4)
assert(table.get(new Get(Bytes.toBytes("delete1"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null)
assert(table.get(new Get(Bytes.toBytes("delete3"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null)
assert(Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("delete2"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))).equals("foo2"))
} finally {
table.close()
connection.close()
}
}
test("bulkGet to test HBase client") {
val config = TEST_UTIL.getConfiguration
val connection = ConnectionFactory.createConnection(config)
val table = connection.getTable(TableName.valueOf("t1"))
try {
var put = new Put(Bytes.toBytes("get1"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
table.put(put)
put = new Put(Bytes.toBytes("get2"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
table.put(put)
put = new Put(Bytes.toBytes("get3"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
table.put(put)
} finally {
table.close()
connection.close()
}
val rdd = sc.parallelize(Array(
Bytes.toBytes("get1"),
Bytes.toBytes("get2"),
Bytes.toBytes("get3"),
Bytes.toBytes("get4")))
val hbaseContext = new HBaseContext(sc, config)
val getRdd = hbaseContext.bulkGet[Array[Byte], String](
TableName.valueOf(tableName),
2,
rdd,
record => {
new Get(record)
},
(result: Result) => {
if (result.listCells() != null) {
val it = result.listCells().iterator()
val B = new StringBuilder
B.append(Bytes.toString(result.getRow) + ":")
while (it.hasNext) {
val cell = it.next()
val q = Bytes.toString(CellUtil.cloneQualifier(cell))
if (q.equals("counter")) {
B.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")")
} else {
B.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")")
}
}
"" + B.toString
} else {
""
}
})
val getArray = getRdd.collect()
assert(getArray.length == 4)
assert(getArray.contains("get1:(a,foo1)"))
assert(getArray.contains("get2:(a,foo2)"))
assert(getArray.contains("get3:(a,foo3)"))
}
test("BulkGet failure test: bad table") {
val config = TEST_UTIL.getConfiguration
val rdd = sc.parallelize(Array(
Bytes.toBytes("get1"),
Bytes.toBytes("get2"),
Bytes.toBytes("get3"),
Bytes.toBytes("get4")))
val hbaseContext = new HBaseContext(sc, config)
intercept[SparkException] {
try {
val getRdd = hbaseContext.bulkGet[Array[Byte], String](
TableName.valueOf("badTableName"),
2,
rdd,
record => {
new Get(record)
},
(result: Result) => "1")
getRdd.collect()
fail("We should have failed and not reached this line")
} catch {
case ex: SparkException => {
assert(
ex.getMessage.contains(
"org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException"))
throw ex
}
}
}
}
test("BulkGet failure test: bad column") {
val config = TEST_UTIL.getConfiguration
val connection = ConnectionFactory.createConnection(config)
val table = connection.getTable(TableName.valueOf("t1"))
try {
var put = new Put(Bytes.toBytes("get1"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
table.put(put)
put = new Put(Bytes.toBytes("get2"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
table.put(put)
put = new Put(Bytes.toBytes("get3"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
table.put(put)
} finally {
table.close()
connection.close()
}
val rdd = sc.parallelize(Array(
Bytes.toBytes("get1"),
Bytes.toBytes("get2"),
Bytes.toBytes("get3"),
Bytes.toBytes("get4")))
val hbaseContext = new HBaseContext(sc, config)
val getRdd = hbaseContext.bulkGet[Array[Byte], String](
TableName.valueOf(tableName),
2,
rdd,
record => {
new Get(record)
},
(result: Result) => {
if (result.listCells() != null) {
val cellValue = result.getColumnLatestCell(
Bytes.toBytes("c"), Bytes.toBytes("bad_column"))
if (cellValue == null) "null" else "bad"
} else "noValue"
})
var nullCounter = 0
var noValueCounter = 0
getRdd.collect().foreach(r => {
if ("null".equals(r)) nullCounter += 1
else if ("noValue".equals(r)) noValueCounter += 1
})
assert(nullCounter == 3)
assert(noValueCounter == 1)
}
test("distributedScan to test HBase client") {
val config = TEST_UTIL.getConfiguration
val connection = ConnectionFactory.createConnection(config)
val table = connection.getTable(TableName.valueOf("t1"))
try {
var put = new Put(Bytes.toBytes("scan1"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
table.put(put)
put = new Put(Bytes.toBytes("scan2"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
table.put(put)
put = new Put(Bytes.toBytes("scan2"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo-2"))
table.put(put)
put = new Put(Bytes.toBytes("scan3"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
table.put(put)
put = new Put(Bytes.toBytes("scan4"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
table.put(put)
put = new Put(Bytes.toBytes("scan5"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
table.put(put)
} finally {
table.close()
connection.close()
}
val hbaseContext = new HBaseContext(sc, config)
val scan = new Scan()
val filter = new FirstKeyOnlyFilter()
scan.setCaching(100)
scan.setStartRow(Bytes.toBytes("scan2"))
scan.setStopRow(Bytes.toBytes("scan4_"))
scan.setFilter(filter)
val scanRdd = hbaseContext.hbaseRDD(TableName.valueOf(tableName), scan)
try {
val scanList = scanRdd.map(r => r._1.copyBytes()).collect()
assert(scanList.length == 3)
var cnt = 0
scanRdd.map(r => r._2.listCells().size()).collect().foreach(l => {
cnt += l
})
// the number of cells returned would be 4 without the Filter
assert(cnt == 3);
} catch {
case ex: Exception => ex.printStackTrace()
}
}
}

View File

@ -1,142 +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.hadoop.hbase.spark
import org.apache.hadoop.hbase.client._
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseTestingUtility}
import org.apache.spark.rdd.RDD
import org.apache.spark.streaming.{Milliseconds, StreamingContext}
import org.apache.spark.{SparkContext, Logging}
import org.apache.hadoop.hbase.spark.HBaseDStreamFunctions._
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
import scala.collection.mutable
class HBaseDStreamFunctionsSuite extends FunSuite with
BeforeAndAfterEach with BeforeAndAfterAll with Logging {
@transient var sc: SparkContext = null
var TEST_UTIL: HBaseTestingUtility = new HBaseTestingUtility
val tableName = "t1"
val columnFamily = "c"
override def beforeAll() {
TEST_UTIL.startMiniCluster()
logInfo(" - minicluster started")
try
TEST_UTIL.deleteTable(TableName.valueOf(tableName))
catch {
case e: Exception => logInfo(" - no table " + tableName + " found")
}
logInfo(" - creating table " + tableName)
TEST_UTIL.createTable(TableName.valueOf(tableName), Bytes.toBytes(columnFamily))
logInfo(" - created table")
sc = new SparkContext("local", "test")
}
override def afterAll() {
TEST_UTIL.deleteTable(TableName.valueOf(tableName))
TEST_UTIL.shutdownMiniCluster()
sc.stop()
}
test("bulkput to test HBase client") {
val config = TEST_UTIL.getConfiguration
val rdd1 = sc.parallelize(Array(
(Bytes.toBytes("1"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))),
(Bytes.toBytes("2"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))),
(Bytes.toBytes("3"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3"))))))
val rdd2 = sc.parallelize(Array(
(Bytes.toBytes("4"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))),
(Bytes.toBytes("5"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar"))))))
var isFinished = false
val hbaseContext = new HBaseContext(sc, config)
val ssc = new StreamingContext(sc, Milliseconds(200))
val queue = mutable.Queue[RDD[(Array[Byte], Array[(Array[Byte],
Array[Byte], Array[Byte])])]]()
queue += rdd1
queue += rdd2
val dStream = ssc.queueStream(queue)
dStream.hbaseBulkPut(
hbaseContext,
TableName.valueOf(tableName),
(putRecord) => {
val put = new Put(putRecord._1)
putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
put
})
dStream.foreachRDD(rdd => {
if (rdd.count() == 0) {
isFinished = true
}
})
ssc.start()
while (!isFinished) {
Thread.sleep(100)
}
ssc.stop(true, true)
val connection = ConnectionFactory.createConnection(config)
val table = connection.getTable(TableName.valueOf("t1"))
try {
val foo1 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("1"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a"))))
assert(foo1 == "foo1")
val foo2 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("2"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b"))))
assert(foo2 == "foo2")
val foo3 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("3"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c"))))
assert(foo3 == "foo3")
val foo4 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("4"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d"))))
assert(foo4 == "foo")
val foo5 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("5"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e"))))
assert(foo5 == "bar")
} finally {
table.close()
connection.close()
}
}
}

View File

@ -1,398 +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.hadoop.hbase.spark
import org.apache.hadoop.hbase.client._
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseTestingUtility}
import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._
import org.apache.spark.{Logging, SparkContext}
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
import scala.collection.mutable
class HBaseRDDFunctionsSuite extends FunSuite with
BeforeAndAfterEach with BeforeAndAfterAll with Logging {
@transient var sc: SparkContext = null
var TEST_UTIL: HBaseTestingUtility = new HBaseTestingUtility
val tableName = "t1"
val columnFamily = "c"
override def beforeAll() {
TEST_UTIL.startMiniCluster
logInfo(" - minicluster started")
try
TEST_UTIL.deleteTable(TableName.valueOf(tableName))
catch {
case e: Exception => logInfo(" - no table " + tableName + " found")
}
logInfo(" - creating table " + tableName)
TEST_UTIL.createTable(TableName.valueOf(tableName), Bytes.toBytes(columnFamily))
logInfo(" - created table")
sc = new SparkContext("local", "test")
}
override def afterAll() {
TEST_UTIL.deleteTable(TableName.valueOf(tableName))
logInfo("shuting down minicluster")
TEST_UTIL.shutdownMiniCluster()
sc.stop()
}
test("bulkput to test HBase client") {
val config = TEST_UTIL.getConfiguration
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))),
(Bytes.toBytes("2"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))),
(Bytes.toBytes("3"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))),
(Bytes.toBytes("4"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))),
(Bytes.toBytes("5"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar"))))))
val hbaseContext = new HBaseContext(sc, config)
rdd.hbaseBulkPut(
hbaseContext,
TableName.valueOf(tableName),
(putRecord) => {
val put = new Put(putRecord._1)
putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
put
})
val connection = ConnectionFactory.createConnection(config)
val table = connection.getTable(TableName.valueOf("t1"))
try {
val foo1 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("1"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a"))))
assert(foo1 == "foo1")
val foo2 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("2"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b"))))
assert(foo2 == "foo2")
val foo3 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("3"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c"))))
assert(foo3 == "foo3")
val foo4 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("4"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d"))))
assert(foo4 == "foo")
val foo5 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("5"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e"))))
assert(foo5 == "bar")
} finally {
table.close()
connection.close()
}
}
test("bulkDelete to test HBase client") {
val config = TEST_UTIL.getConfiguration
val connection = ConnectionFactory.createConnection(config)
val table = connection.getTable(TableName.valueOf("t1"))
try {
var put = new Put(Bytes.toBytes("delete1"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
table.put(put)
put = new Put(Bytes.toBytes("delete2"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
table.put(put)
put = new Put(Bytes.toBytes("delete3"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
table.put(put)
val rdd = sc.parallelize(Array(
Bytes.toBytes("delete1"),
Bytes.toBytes("delete3")))
val hbaseContext = new HBaseContext(sc, config)
rdd.hbaseBulkDelete(hbaseContext,
TableName.valueOf(tableName),
putRecord => new Delete(putRecord),
4)
assert(table.get(new Get(Bytes.toBytes("delete1"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null)
assert(table.get(new Get(Bytes.toBytes("delete3"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")) == null)
assert(Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("delete2"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a")))).equals("foo2"))
} finally {
table.close()
connection.close()
}
}
test("bulkGet to test HBase client") {
val config = TEST_UTIL.getConfiguration
val connection = ConnectionFactory.createConnection(config)
val table = connection.getTable(TableName.valueOf("t1"))
try {
var put = new Put(Bytes.toBytes("get1"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
table.put(put)
put = new Put(Bytes.toBytes("get2"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
table.put(put)
put = new Put(Bytes.toBytes("get3"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
table.put(put)
} finally {
table.close()
connection.close()
}
val rdd = sc.parallelize(Array(
Bytes.toBytes("get1"),
Bytes.toBytes("get2"),
Bytes.toBytes("get3"),
Bytes.toBytes("get4")))
val hbaseContext = new HBaseContext(sc, config)
//Get with custom convert logic
val getRdd = rdd.hbaseBulkGet[String](hbaseContext, TableName.valueOf(tableName), 2,
record => {
new Get(record)
},
(result: Result) => {
if (result.listCells() != null) {
val it = result.listCells().iterator()
val B = new StringBuilder
B.append(Bytes.toString(result.getRow) + ":")
while (it.hasNext) {
val cell = it.next
val q = Bytes.toString(CellUtil.cloneQualifier(cell))
if (q.equals("counter")) {
B.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")")
} else {
B.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")")
}
}
"" + B.toString
} else {
""
}
})
val getArray = getRdd.collect()
assert(getArray.length == 4)
assert(getArray.contains("get1:(a,foo1)"))
assert(getArray.contains("get2:(a,foo2)"))
assert(getArray.contains("get3:(a,foo3)"))
}
test("bulkGet default converter to test HBase client") {
val config = TEST_UTIL.getConfiguration
val connection = ConnectionFactory.createConnection(config)
val table = connection.getTable(TableName.valueOf("t1"))
try {
var put = new Put(Bytes.toBytes("get1"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
table.put(put)
put = new Put(Bytes.toBytes("get2"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
table.put(put)
put = new Put(Bytes.toBytes("get3"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
table.put(put)
} finally {
table.close()
connection.close()
}
val rdd = sc.parallelize(Array(
Bytes.toBytes("get1"),
Bytes.toBytes("get2"),
Bytes.toBytes("get3"),
Bytes.toBytes("get4")))
val hbaseContext = new HBaseContext(sc, config)
val getRdd = rdd.hbaseBulkGet(hbaseContext, TableName.valueOf("t1"), 2,
record => {
new Get(record)
}).map((row) => {
if (row != null && row._2.listCells() != null) {
val it = row._2.listCells().iterator()
val B = new StringBuilder
B.append(Bytes.toString(row._2.getRow) + ":")
while (it.hasNext) {
val cell = it.next
val q = Bytes.toString(CellUtil.cloneQualifier(cell))
if (q.equals("counter")) {
B.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")")
} else {
B.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")")
}
}
"" + B.toString
} else {
""
}})
val getArray = getRdd.collect()
assert(getArray.length == 4)
assert(getArray.contains("get1:(a,foo1)"))
assert(getArray.contains("get2:(a,foo2)"))
assert(getArray.contains("get3:(a,foo3)"))
}
test("foreachPartition with puts to test HBase client") {
val config = TEST_UTIL.getConfiguration
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1foreach"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1")))),
(Bytes.toBytes("2foreach"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("b"), Bytes.toBytes("foo2")))),
(Bytes.toBytes("3foreach"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("c"), Bytes.toBytes("foo3")))),
(Bytes.toBytes("4foreach"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("d"), Bytes.toBytes("foo")))),
(Bytes.toBytes("5foreach"),
Array((Bytes.toBytes(columnFamily), Bytes.toBytes("e"), Bytes.toBytes("bar"))))))
val hbaseContext = new HBaseContext(sc, config)
rdd.hbaseForeachPartition(hbaseContext, (it, conn) => {
val bufferedMutator = conn.getBufferedMutator(TableName.valueOf("t1"))
it.foreach((putRecord) => {
val put = new Put(putRecord._1)
putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
bufferedMutator.mutate(put)
})
bufferedMutator.flush()
bufferedMutator.close()
})
val connection = ConnectionFactory.createConnection(config)
val table = connection.getTable(TableName.valueOf("t1"))
try {
val foo1 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("1foreach"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("a"))))
assert(foo1 == "foo1")
val foo2 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("2foreach"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("b"))))
assert(foo2 == "foo2")
val foo3 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("3foreach"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("c"))))
assert(foo3 == "foo3")
val foo4 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("4foreach"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("d"))))
assert(foo4 == "foo")
val foo5 = Bytes.toString(CellUtil.cloneValue(table.get(new Get(Bytes.toBytes("5"))).
getColumnLatestCell(Bytes.toBytes(columnFamily), Bytes.toBytes("e"))))
assert(foo5 == "bar")
} finally {
table.close()
connection.close()
}
}
test("mapPartitions with Get from test HBase client") {
val config = TEST_UTIL.getConfiguration
val connection = ConnectionFactory.createConnection(config)
val table = connection.getTable(TableName.valueOf("t1"))
try {
var put = new Put(Bytes.toBytes("get1"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo1"))
table.put(put)
put = new Put(Bytes.toBytes("get2"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo2"))
table.put(put)
put = new Put(Bytes.toBytes("get3"))
put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes("a"), Bytes.toBytes("foo3"))
table.put(put)
} finally {
table.close()
connection.close()
}
val rdd = sc.parallelize(Array(
Bytes.toBytes("get1"),
Bytes.toBytes("get2"),
Bytes.toBytes("get3"),
Bytes.toBytes("get4")))
val hbaseContext = new HBaseContext(sc, config)
//Get with custom convert logic
val getRdd = rdd.hbaseMapPartitions(hbaseContext, (it, conn) => {
val table = conn.getTable(TableName.valueOf("t1"))
var res = mutable.MutableList[String]()
it.foreach(r => {
val get = new Get(r)
val result = table.get(get)
if (result.listCells != null) {
val it = result.listCells().iterator()
val B = new StringBuilder
B.append(Bytes.toString(result.getRow) + ":")
while (it.hasNext) {
val cell = it.next()
val q = Bytes.toString(CellUtil.cloneQualifier(cell))
if (q.equals("counter")) {
B.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")")
} else {
B.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")")
}
}
res += "" + B.toString
} else {
res += ""
}
})
res.iterator
})
val getArray = getRdd.collect()
assert(getArray.length == 4)
assert(getArray.contains("get1:(a,foo1)"))
assert(getArray.contains("get2:(a,foo2)"))
assert(getArray.contains("get3:(a,foo3)"))
}
}

View File

@ -1,62 +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.hadoop.hbase.spark
import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf
import org.apache.spark.SparkEnv
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{Row, SQLContext}
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types._
class HBaseTestSource extends RelationProvider {
override def createRelation(
sqlContext: SQLContext,
parameters: Map[String, String]): BaseRelation = {
DummyScan(
parameters("cacheSize").toInt,
parameters("batchNum").toInt,
parameters("blockCacheingEnable").toBoolean,
parameters("rowNum").toInt)(sqlContext)
}
}
case class DummyScan(
cacheSize: Int,
batchNum: Int,
blockCachingEnable: Boolean,
rowNum: Int)(@transient val sqlContext: SQLContext)
extends BaseRelation with TableScan {
private def sparkConf = SparkEnv.get.conf
override def schema: StructType =
StructType(StructField("i", IntegerType, nullable = false) :: Nil)
override def buildScan(): RDD[Row] = sqlContext.sparkContext.parallelize(0 until rowNum)
.map(Row(_))
.map{ x =>
if (sparkConf.getInt(HBaseSparkConf.QUERY_BATCHSIZE,
-1) != batchNum ||
sparkConf.getInt(HBaseSparkConf.QUERY_CACHEDROWS,
-1) != cacheSize ||
sparkConf.getBoolean(HBaseSparkConf.QUERY_CACHEBLOCKS,
false) != blockCachingEnable) {
throw new Exception("HBase Spark configuration cannot be set properly")
}
x
}
}

View File

@ -1,523 +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.hadoop.hbase.spark
import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf
import org.apache.hadoop.hbase.{TableName, HBaseTestingUtility}
import org.apache.spark.sql.datasources.hbase.HBaseTableCatalog
import org.apache.spark.sql.{DataFrame, SQLContext}
import org.apache.spark.{SparkConf, SparkContext, Logging}
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
case class FilterRangeRecord(
intCol0: Int,
boolCol1: Boolean,
doubleCol2: Double,
floatCol3: Float,
intCol4: Int,
longCol5: Long,
shortCol6: Short,
stringCol7: String,
byteCol8: Byte)
object FilterRangeRecord {
def apply(i: Int): FilterRangeRecord = {
FilterRangeRecord(if (i % 2 == 0) i else -i,
i % 2 == 0,
if (i % 2 == 0) i.toDouble else -i.toDouble,
i.toFloat,
if (i % 2 == 0) i else -i,
i.toLong,
i.toShort,
s"String$i extra",
i.toByte)
}
}
class PartitionFilterSuite extends FunSuite with
BeforeAndAfterEach with BeforeAndAfterAll with Logging {
@transient var sc: SparkContext = null
var TEST_UTIL: HBaseTestingUtility = new HBaseTestingUtility
var sqlContext: SQLContext = null
var df: DataFrame = null
def withCatalog(cat: String): DataFrame = {
sqlContext
.read
.options(Map(HBaseTableCatalog.tableCatalog -> cat))
.format("org.apache.hadoop.hbase.spark")
.load()
}
override def beforeAll() {
TEST_UTIL.startMiniCluster
val sparkConf = new SparkConf
sparkConf.set(HBaseSparkConf.QUERY_CACHEBLOCKS, "true")
sparkConf.set(HBaseSparkConf.QUERY_BATCHSIZE, "100")
sparkConf.set(HBaseSparkConf.QUERY_CACHEDROWS, "100")
sc = new SparkContext("local", "test", sparkConf)
new HBaseContext(sc, TEST_UTIL.getConfiguration)
sqlContext = new SQLContext(sc)
}
override def afterAll() {
logInfo("shutting down minicluster")
TEST_UTIL.shutdownMiniCluster()
sc.stop()
}
override def beforeEach(): Unit = {
DefaultSourceStaticUtils.lastFiveExecutionRules.clear()
}
// The original raw data used for construct result set without going through
// data frame logic. It is used to verify the result set retrieved from data frame logic.
val rawResult = (0 until 32).map { i =>
FilterRangeRecord(i)
}
def collectToSet[T](df: DataFrame): Set[T] = {
df.collect().map(_.getAs[T](0)).toSet
}
val catalog = s"""{
|"table":{"namespace":"default", "name":"rangeTable"},
|"rowkey":"key",
|"columns":{
|"intCol0":{"cf":"rowkey", "col":"key", "type":"int"},
|"boolCol1":{"cf":"cf1", "col":"boolCol1", "type":"boolean"},
|"doubleCol2":{"cf":"cf2", "col":"doubleCol2", "type":"double"},
|"floatCol3":{"cf":"cf3", "col":"floatCol3", "type":"float"},
|"intCol4":{"cf":"cf4", "col":"intCol4", "type":"int"},
|"longCol5":{"cf":"cf5", "col":"longCol5", "type":"bigint"},
|"shortCol6":{"cf":"cf6", "col":"shortCol6", "type":"smallint"},
|"stringCol7":{"cf":"cf7", "col":"stringCol7", "type":"string"},
|"byteCol8":{"cf":"cf8", "col":"byteCol8", "type":"tinyint"}
|}
|}""".stripMargin
test("populate rangeTable") {
val sql = sqlContext
import sql.implicits._
sc.parallelize(rawResult).toDF.write.options(
Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
.format("org.apache.hadoop.hbase.spark")
.save()
}
test("rangeTable full query") {
val df = withCatalog(catalog)
df.show
assert(df.count() === 32)
}
/**
*expected result: only showing top 20 rows
*+-------+
*|intCol0|
*+-------+
*| -31 |
*| -29 |
*| -27 |
*| -25 |
*| -23 |
*| -21 |
*| -19 |
*| -17 |
*| -15 |
*| -13 |
*| -11 |
*| -9 |
*| -7 |
*| -5 |
*| -3 |
*| -1 |
*+---- +
*/
test("rangeTable rowkey less than 0") {
val sql = sqlContext
import sql.implicits._
val df = withCatalog(catalog)
val s = df.filter($"intCol0" < 0).select($"intCol0")
s.show
// filter results without going through dataframe
val expected = rawResult.filter(_.intCol0 < 0).map(_.intCol0).toSet
// filter results going through dataframe
val result = collectToSet[Int](s)
assert(expected === result)
}
/**
*expected result: only showing top 20 rows
*+-------+
*|intCol4|
*+-------+
*| -31 |
*| -29 |
*| -27 |
*| -25 |
*| -23 |
*| -21 |
*| -19 |
*| -17 |
*| -15 |
*| -13 |
*| -11 |
*| -9 |
*| -7 |
*| -5 |
*| -3 |
*| -1 |
*+-------+
*/
test("rangeTable int col less than 0") {
val sql = sqlContext
import sql.implicits._
val df = withCatalog(catalog)
val s = df.filter($"intCol4" < 0).select($"intCol4")
s.show
// filter results without going through dataframe
val expected = rawResult.filter(_.intCol4 < 0).map(_.intCol4).toSet
// filter results going through dataframe
val result = collectToSet[Int](s)
assert(expected === result)
}
/**
*expected result: only showing top 20 rows
*+-----------+
*| doubleCol2|
*+-----------+
*| 0.0 |
*| 2.0 |
*|-31.0 |
*|-29.0 |
*|-27.0 |
*|-25.0 |
*|-23.0 |
*|-21.0 |
*|-19.0 |
*|-17.0 |
*|-15.0 |
*|-13.0 |
*|-11.0 |
*| -9.0 |
*| -7.0 |
*| -5.0 |
*| -3.0 |
*| -1.0 |
*+-----------+
*/
test("rangeTable double col less than 0") {
val sql = sqlContext
import sql.implicits._
val df = withCatalog(catalog)
val s = df.filter($"doubleCol2" < 3.0).select($"doubleCol2")
s.show
// filter results without going through dataframe
val expected = rawResult.filter(_.doubleCol2 < 3.0).map(_.doubleCol2).toSet
// filter results going through dataframe
val result = collectToSet[Double](s)
assert(expected === result)
}
/**
* expected result: only showing top 20 rows
*+-------+
*|intCol0|
*+-------+
*| -31 |
*| -29 |
*| -27 |
*| -25 |
*| -23 |
*| -21 |
*| -19 |
*| -17 |
*| -15 |
*| -13 |
*| -11 |
*+-------+
*
*/
test("rangeTable lessequal than -10") {
val sql = sqlContext
import sql.implicits._
val df = withCatalog(catalog)
val s = df.filter($"intCol0" <= -10).select($"intCol0")
s.show
// filter results without going through dataframe
val expected = rawResult.filter(_.intCol0 <= -10).map(_.intCol0).toSet
// filter results going through dataframe
val result = collectToSet[Int](s)
assert(expected === result)
}
/**
*expected result: only showing top 20 rows
*+-------+
*|intCol0|
*+----+
*| -31 |
*| -29 |
*| -27 |
*| -25 |
*| -23 |
*| -21 |
*| -19 |
*| -17 |
*| -15 |
*| -13 |
*| -11 |
*| -9 |
*+-------+
*/
test("rangeTable lessequal than -9") {
val sql = sqlContext
import sql.implicits._
val df = withCatalog(catalog)
val s = df.filter($"intCol0" <= -9).select($"intCol0")
s.show
// filter results without going through dataframe
val expected = rawResult.filter(_.intCol0 <= -9).map(_.intCol0).toSet
// filter results going through dataframe
val result = collectToSet[Int](s)
assert(expected === result)
}
/**
*expected result: only showing top 20 rows
*+-------+
*|intCol0|
*+-------+
*| 0 |
*| 2 |
*| 4 |
*| 6 |
*| 8 |
*| 10 |
*| 12 |
*| 14 |
*| 16 |
*| 18 |
*| 20 |
*| 22 |
*| 24 |
*| 26 |
*| 28 |
*| 30 |
*| -9 |
*| -7 |
*| -5 |
*| -3 |
*+-------+
*/
test("rangeTable greaterequal than -9") {
val sql = sqlContext
import sql.implicits._
val df = withCatalog(catalog)
val s = df.filter($"intCol0" >= -9).select($"intCol0")
s.show
// filter results without going through dataframe
val expected = rawResult.filter(_.intCol0 >= -9).map(_.intCol0).toSet
// filter results going through dataframe
val result = collectToSet[Int](s)
assert(expected === result)
}
/**
*expected result: only showing top 20 rows
*+-------+
*|intCol0|
*+-------+
*| 0 |
*| 2 |
*| 4 |
*| 6 |
*| 8 |
*| 10 |
*| 12 |
*| 14 |
*| 16 |
*| 18 |
*| 20 |
*| 22 |
*| 24 |
*| 26 |
*| 28 |
*| 30 |
*+-------+
*/
test("rangeTable greaterequal than 0") {
val sql = sqlContext
import sql.implicits._
val df = withCatalog(catalog)
val s = df.filter($"intCol0" >= 0).select($"intCol0")
s.show
// filter results without going through dataframe
val expected = rawResult.filter(_.intCol0 >= 0).map(_.intCol0).toSet
// filter results going through dataframe
val result = collectToSet[Int](s)
assert(expected === result)
}
/**
*expected result: only showing top 20 rows
*+-------+
*|intCol0|
*+-------+
*| 12 |
*| 14 |
*| 16 |
*| 18 |
*| 20 |
*| 22 |
*| 24 |
*| 26 |
*| 28 |
*| 30 |
*+-------+
*/
test("rangeTable greater than 10") {
val sql = sqlContext
import sql.implicits._
val df = withCatalog(catalog)
val s = df.filter($"intCol0" > 10).select($"intCol0")
s.show
// filter results without going through dataframe
val expected = rawResult.filter(_.intCol0 > 10).map(_.intCol0).toSet
// filter results going through dataframe
val result = collectToSet[Int](s)
assert(expected === result)
}
/**
*expected result: only showing top 20 rows
*+-------+
*|intCol0|
*+-------+
*| 0 |
*| 2 |
*| 4 |
*| 6 |
*| 8 |
*| 10 |
*| -9 |
*| -7 |
*| -5 |
*| -3 |
*| -1 |
*+-------+
*/
test("rangeTable and") {
val sql = sqlContext
import sql.implicits._
val df = withCatalog(catalog)
val s = df.filter($"intCol0" > -10 && $"intCol0" <= 10).select($"intCol0")
s.show
// filter results without going through dataframe
val expected = rawResult.filter(x => x.intCol0 > -10 && x.intCol0 <= 10 ).map(_.intCol0).toSet
// filter results going through dataframe
val result = collectToSet[Int](s)
assert(expected === result)
}
/**
*expected result: only showing top 20 rows
*+-------+
*|intCol0|
*+-------+
*| 12 |
*| 14 |
*| 16 |
*| 18 |
*| 20 |
*| 22 |
*| 24 |
*| 26 |
*| 28 |
*| 30 |
*| -31 |
*| -29 |
*| -27 |
*| -25 |
*| -23 |
*| -21 |
*| -19 |
*| -17 |
*| -15 |
*| -13 |
*+-------+
*/
test("or") {
val sql = sqlContext
import sql.implicits._
val df = withCatalog(catalog)
val s = df.filter($"intCol0" <= -10 || $"intCol0" > 10).select($"intCol0")
s.show
// filter results without going through dataframe
val expected = rawResult.filter(x => x.intCol0 <= -10 || x.intCol0 > 10).map(_.intCol0).toSet
// filter results going through dataframe
val result = collectToSet[Int](s)
assert(expected === result)
}
/**
*expected result: only showing top 20 rows
*+-------+
*|intCol0|
*+-------+
*| 0 |
*| 2 |
*| 4 |
*| 6 |
*| 8 |
*| 10 |
*| 12 |
*| 14 |
*| 16 |
*| 18 |
*| 20 |
*| 22 |
*| 24 |
*| 26 |
*| 28 |
*| 30 |
*| -31 |
*| -29 |
*| -27 |
*| -25 |
*+-------+
*/
test("rangeTable all") {
val sql = sqlContext
import sql.implicits._
val df = withCatalog(catalog)
val s = df.filter($"intCol0" >= -100).select($"intCol0")
s.show
// filter results without going through dataframe
val expected = rawResult.filter(_.intCol0 >= -100).map(_.intCol0).toSet
// filter results going through dataframe
val result = collectToSet[Int](s)
assert(expected === result)
}
}

38
pom.xml
View File

@ -85,11 +85,9 @@
<module>hbase-checkstyle</module>
<module>hbase-external-blockcache</module>
<module>hbase-shaded</module>
<module>hbase-spark</module>
<module>hbase-archetypes</module>
<module>hbase-metrics-api</module>
<module>hbase-metrics</module>
<module>hbase-spark-it</module>
<module>hbase-backup</module>
</modules>
<!--Add apache snapshots in case we want to use unreleased versions of plugins:
@ -997,38 +995,6 @@
</rules>
</configuration>
</execution>
<execution>
<id>banned-scala</id>
<goals>
<goal>enforce</goal>
</goals>
<configuration>
<rules>
<bannedDependencies>
<excludes>
<exclude>org.scala-lang:scala-library</exclude>
</excludes>
<message>We don't allow Scala outside of the hbase-spark module, see HBASE-13992.</message>
</bannedDependencies>
</rules>
</configuration>
</execution>
<execution>
<id>banned-hbase-spark</id>
<goals>
<goal>enforce</goal>
</goals>
<configuration>
<rules>
<bannedDependencies>
<excludes>
<exclude>org.apache.hbase:hbase-spark</exclude>
</excludes>
<message>We don't allow other modules to depend on hbase-spark, see HBASE-13992.</message>
</bannedDependencies>
</rules>
</configuration>
</execution>
<execution>
<id>check-aggregate-license</id>
<!-- must check after LICENSE is built at 'generate-resources' -->
@ -3305,7 +3271,7 @@
<exclude>**/protobuf/*</exclude>
<exclude>**/*.scala</exclude>
</sourceFileExcludes>
<excludePackageNames>org.apache.hadoop.hbase.tmpl.common:com.google.protobuf:org.apache.hadoop.hbase.spark:org.apache.hadoop.hbase.generated*</excludePackageNames>
<excludePackageNames>org.apache.hadoop.hbase.tmpl.common:com.google.protobuf:org.apache.hadoop.hbase.generated*</excludePackageNames>
<show>private</show> <!-- (shows all classes and members) -->
<quiet>true</quiet>
<linksource>true</linksource>
@ -3348,7 +3314,7 @@
<exclude>**/protobuf/*</exclude>
<exclude>**/*.scala</exclude>
</sourceFileExcludes>
<excludePackageNames>org.apache.hadoop.hbase.tmpl.common:com.google.protobuf:org.apache.hadoop.hbase.spark:org.apache.hadoop.hbase.generated*</excludePackageNames>
<excludePackageNames>org.apache.hadoop.hbase.tmpl.common:com.google.protobuf:org.apache.hadoop.hbase.generated*</excludePackageNames>
<show>private</show> <!-- (shows all classes and members) -->
<quiet>true</quiet>
<linksource>true</linksource>

View File

@ -1,690 +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.
*/
////
[[spark]]
= HBase and Spark
:doctype: book
:numbered:
:toc: left
:icons: font
:experimental:
link:http://spark.apache.org/[Apache Spark] is a software framework that is used
to process data in memory in a distributed manner, and is replacing MapReduce in
many use cases.
Spark itself is out of scope of this document, please refer to the Spark site for
more information on the Spark project and subprojects. This document will focus
on 4 main interaction points between Spark and HBase. Those interaction points are:
Basic Spark::
The ability to have an HBase Connection at any point in your Spark DAG.
Spark Streaming::
The ability to have an HBase Connection at any point in your Spark Streaming
application.
Spark Bulk Load::
The ability to write directly to HBase HFiles for bulk insertion into HBase
SparkSQL/DataFrames::
The ability to write SparkSQL that draws on tables that are represented in HBase.
The following sections will walk through examples of all these interaction points.
== Basic Spark
This section discusses Spark HBase integration at the lowest and simplest levels.
All the other interaction points are built upon the concepts that will be described
here.
At the root of all Spark and HBase integration is the HBaseContext. The HBaseContext
takes in HBase configurations and pushes them to the Spark executors. This allows
us to have an HBase Connection per Spark Executor in a static location.
For reference, Spark Executors can be on the same nodes as the Region Servers or
on different nodes there is no dependence of co-location. Think of every Spark
Executor as a multi-threaded client application. This allows any Spark Tasks
running on the executors to access the shared Connection object.
.HBaseContext Usage Example
====
This example shows how HBaseContext can be used to do a `foreachPartition` on a RDD
in Scala:
[source, scala]
----
val sc = new SparkContext("local", "test")
val config = new HBaseConfiguration()
...
val hbaseContext = new HBaseContext(sc, config)
rdd.hbaseForeachPartition(hbaseContext, (it, conn) => {
val bufferedMutator = conn.getBufferedMutator(TableName.valueOf("t1"))
it.foreach((putRecord) => {
. val put = new Put(putRecord._1)
. putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
. bufferedMutator.mutate(put)
})
bufferedMutator.flush()
bufferedMutator.close()
})
----
Here is the same example implemented in Java:
[source, java]
----
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
try {
List<byte[]> list = new ArrayList<>();
list.add(Bytes.toBytes("1"));
...
list.add(Bytes.toBytes("5"));
JavaRDD<byte[]> rdd = jsc.parallelize(list);
Configuration conf = HBaseConfiguration.create();
JavaHBaseContext hbaseContext = new JavaHBaseContext(jsc, conf);
hbaseContext.foreachPartition(rdd,
new VoidFunction<Tuple2<Iterator<byte[]>, Connection>>() {
public void call(Tuple2<Iterator<byte[]>, Connection> t)
throws Exception {
Table table = t._2().getTable(TableName.valueOf(tableName));
BufferedMutator mutator = t._2().getBufferedMutator(TableName.valueOf(tableName));
while (t._1().hasNext()) {
byte[] b = t._1().next();
Result r = table.get(new Get(b));
if (r.getExists()) {
mutator.mutate(new Put(b));
}
}
mutator.flush();
mutator.close();
table.close();
}
});
} finally {
jsc.stop();
}
----
====
All functionality between Spark and HBase will be supported both in Scala and in
Java, with the exception of SparkSQL which will support any language that is
supported by Spark. For the remaining of this documentation we will focus on
Scala examples for now.
The examples above illustrate how to do a foreachPartition with a connection. A
number of other Spark base functions are supported out of the box:
// tag::spark_base_functions[]
`bulkPut`:: For massively parallel sending of puts to HBase
`bulkDelete`:: For massively parallel sending of deletes to HBase
`bulkGet`:: For massively parallel sending of gets to HBase to create a new RDD
`mapPartition`:: To do a Spark Map function with a Connection object to allow full
access to HBase
`hBaseRDD`:: To simplify a distributed scan to create a RDD
// end::spark_base_functions[]
For examples of all these functionalities, see the HBase-Spark Module.
== Spark Streaming
http://spark.apache.org/streaming/[Spark Streaming] is a micro batching stream
processing framework built on top of Spark. HBase and Spark Streaming make great
companions in that HBase can help serve the following benefits alongside Spark
Streaming.
* A place to grab reference data or profile data on the fly
* A place to store counts or aggregates in a way that supports Spark Streaming
promise of _only once processing_.
The HBase-Spark modules integration points with Spark Streaming are similar to
its normal Spark integration points, in that the following commands are possible
straight off a Spark Streaming DStream.
include::spark.adoc[tags=spark_base_functions]
.`bulkPut` Example with DStreams
====
Below is an example of bulkPut with DStreams. It is very close in feel to the RDD
bulk put.
[source, scala]
----
val sc = new SparkContext("local", "test")
val config = new HBaseConfiguration()
val hbaseContext = new HBaseContext(sc, config)
val ssc = new StreamingContext(sc, Milliseconds(200))
val rdd1 = ...
val rdd2 = ...
val queue = mutable.Queue[RDD[(Array[Byte], Array[(Array[Byte],
Array[Byte], Array[Byte])])]]()
queue += rdd1
queue += rdd2
val dStream = ssc.queueStream(queue)
dStream.hbaseBulkPut(
hbaseContext,
TableName.valueOf(tableName),
(putRecord) => {
val put = new Put(putRecord._1)
putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, putValue._3))
put
})
----
There are three inputs to the `hbaseBulkPut` function.
. The hbaseContext that carries the configuration boardcast information link us
to the HBase Connections in the executors
. The table name of the table we are putting data into
. A function that will convert a record in the DStream into an HBase Put object.
====
== Bulk Load
There are two options for bulk loading data into HBase with Spark. There is the
basic bulk load functionality that will work for cases where your rows have
millions of columns and cases where your columns are not consolidated and
partitions before the on the map side of the Spark bulk load process.
There is also a thin record bulk load option with Spark, this second option is
designed for tables that have less then 10k columns per row. The advantage
of this second option is higher throughput and less over all load on the Spark
shuffle operation.
Both implementations work more or less like the MapReduce bulk load process in
that a partitioner partitions the rowkeys based on region splits and
the row keys are sent to the reducers in order, so that HFiles can be written
out directly from the reduce phase.
In Spark terms, the bulk load will be implemented around a the Spark
`repartitionAndSortWithinPartitions` followed by a Spark `foreachPartition`.
First lets look at an example of using the basic bulk load functionality
.Bulk Loading Example
====
The following example shows bulk loading with Spark.
[source, scala]
----
val sc = new SparkContext("local", "test")
val config = new HBaseConfiguration()
val hbaseContext = new HBaseContext(sc, config)
val stagingFolder = ...
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
(Bytes.toBytes("3"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))), ...
rdd.hbaseBulkLoad(TableName.valueOf(tableName),
t => {
val rowKey = t._1
val family:Array[Byte] = t._2(0)._1
val qualifier = t._2(0)._2
val value = t._2(0)._3
val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier)
Seq((keyFamilyQualifier, value)).iterator
},
stagingFolder.getPath)
val load = new LoadIncrementalHFiles(config)
load.doBulkLoad(new Path(stagingFolder.getPath),
conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))
----
====
The `hbaseBulkLoad` function takes three required parameters:
. The table name of the table we intend to bulk load too
. A function that will convert a record in the RDD to a tuple key value par. With
the tuple key being a KeyFamilyQualifer object and the value being the cell value.
The KeyFamilyQualifer object will hold the RowKey, Column Family, and Column Qualifier.
The shuffle will partition on the RowKey but will sort by all three values.
. The temporary path for the HFile to be written out too
Following the Spark bulk load command, use the HBase's LoadIncrementalHFiles object
to load the newly created HFiles into HBase.
.Additional Parameters for Bulk Loading with Spark
You can set the following attributes with additional parameter options on hbaseBulkLoad.
* Max file size of the HFiles
* A flag to exclude HFiles from compactions
* Column Family settings for compression, bloomType, blockSize, and dataBlockEncoding
.Using Additional Parameters
====
[source, scala]
----
val sc = new SparkContext("local", "test")
val config = new HBaseConfiguration()
val hbaseContext = new HBaseContext(sc, config)
val stagingFolder = ...
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
(Bytes.toBytes("3"),
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))), ...
val familyHBaseWriterOptions = new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions]
val f1Options = new FamilyHFileWriteOptions("GZ", "ROW", 128, "PREFIX")
familyHBaseWriterOptions.put(Bytes.toBytes("columnFamily1"), f1Options)
rdd.hbaseBulkLoad(TableName.valueOf(tableName),
t => {
val rowKey = t._1
val family:Array[Byte] = t._2(0)._1
val qualifier = t._2(0)._2
val value = t._2(0)._3
val keyFamilyQualifier= new KeyFamilyQualifier(rowKey, family, qualifier)
Seq((keyFamilyQualifier, value)).iterator
},
stagingFolder.getPath,
familyHBaseWriterOptions,
compactionExclude = false,
HConstants.DEFAULT_MAX_FILE_SIZE)
val load = new LoadIncrementalHFiles(config)
load.doBulkLoad(new Path(stagingFolder.getPath),
conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))
----
====
Now lets look at how you would call the thin record bulk load implementation
.Using thin record bulk load
====
[source, scala]
----
val sc = new SparkContext("local", "test")
val config = new HBaseConfiguration()
val hbaseContext = new HBaseContext(sc, config)
val stagingFolder = ...
val rdd = sc.parallelize(Array(
("1",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("a"), Bytes.toBytes("foo1"))),
("3",
(Bytes.toBytes(columnFamily1), Bytes.toBytes("b"), Bytes.toBytes("foo2.b"))), ...
rdd.hbaseBulkLoadThinRows(hbaseContext,
TableName.valueOf(tableName),
t => {
val rowKey = t._1
val familyQualifiersValues = new FamiliesQualifiersValues
t._2.foreach(f => {
val family:Array[Byte] = f._1
val qualifier = f._2
val value:Array[Byte] = f._3
familyQualifiersValues +=(family, qualifier, value)
})
(new ByteArrayWrapper(Bytes.toBytes(rowKey)), familyQualifiersValues)
},
stagingFolder.getPath,
new java.util.HashMap[Array[Byte], FamilyHFileWriteOptions],
compactionExclude = false,
20)
val load = new LoadIncrementalHFiles(config)
load.doBulkLoad(new Path(stagingFolder.getPath),
conn.getAdmin, table, conn.getRegionLocator(TableName.valueOf(tableName)))
----
====
Note that the big difference in using bulk load for thin rows is the function
returns a tuple with the first value being the row key and the second value
being an object of FamiliesQualifiersValues, which will contain all the
values for this row for all column families.
== SparkSQL/DataFrames
HBase-Spark Connector (in HBase-Spark Module) leverages
link:https://databricks.com/blog/2015/01/09/spark-sql-data-sources-api-unified-data-access-for-the-spark-platform.html[DataSource API]
(link:https://issues.apache.org/jira/browse/SPARK-3247[SPARK-3247])
introduced in Spark-1.2.0, bridges the gap between simple HBase KV store and complex
relational SQL queries and enables users to perform complex data analytical work
on top of HBase using Spark. HBase Dataframe is a standard Spark Dataframe, and is able to
interact with any other data sources such as Hive, Orc, Parquet, JSON, etc.
HBase-Spark Connector applies critical techniques such as partition pruning, column pruning,
predicate pushdown and data locality.
To use HBase-Spark connector, users need to define the Catalog for the schema mapping
between HBase and Spark tables, prepare the data and populate the HBase table,
then load HBase DataFrame. After that, users can do integrated query and access records
in HBase table with SQL query. Following illustrates the basic procedure.
=== Define catalog
[source, scala]
----
def catalog = s"""{
       |"table":{"namespace":"default", "name":"table1"},
       |"rowkey":"key",
       |"columns":{
         |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
         |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
         |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
         |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
         |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
         |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
         |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
         |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
         |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
       |}
     |}""".stripMargin
----
Catalog defines a mapping between HBase and Spark tables. There are two critical parts of this catalog.
One is the rowkey definition and the other is the mapping between table column in Spark and
the column family and column qualifier in HBase. The above defines a schema for a HBase table
with name as table1, row key as key and a number of columns (col1 `-` col8). Note that the rowkey
also has to be defined in details as a column (col0), which has a specific cf (rowkey).
=== Save the DataFrame
[source, scala]
----
case class HBaseRecord(
col0: String,
col1: Boolean,
col2: Double,
col3: Float,
col4: Int,       
col5: Long,
col6: Short,
col7: String,
col8: Byte)
object HBaseRecord
{                                                                                                             
def apply(i: Int, t: String): HBaseRecord = {
val s = s"""row${"%03d".format(i)}"""       
HBaseRecord(s,
i % 2 == 0,
i.toDouble,
i.toFloat,  
i,
i.toLong,
i.toShort,  
s"String$i: $t",      
i.toByte)
}
}
val data = (0 to 255).map { i =>  HBaseRecord(i, "extra")}
sc.parallelize(data).toDF.write.options(
 Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
 .format("org.apache.hadoop.hbase.spark ")
 .save()
----
`data` prepared by the user is a local Scala collection which has 256 HBaseRecord objects.
`sc.parallelize(data)` function distributes `data` to form an RDD. `toDF` returns a DataFrame.
`write` function returns a DataFrameWriter used to write the DataFrame to external storage
systems (e.g. HBase here). Given a DataFrame with specified schema `catalog`, `save` function
will create an HBase table with 5 regions and save the DataFrame inside.
=== Load the DataFrame
[source, scala]
----
def withCatalog(cat: String): DataFrame = {
sqlContext
.read
.options(Map(HBaseTableCatalog.tableCatalog->cat))
.format("org.apache.hadoop.hbase.spark")
.load()
}
val df = withCatalog(catalog)
----
In withCatalog function, sqlContext is a variable of SQLContext, which is the entry point
for working with structured data (rows and columns) in Spark.
`read` returns a DataFrameReader that can be used to read data in as a DataFrame.
`option` function adds input options for the underlying data source to the DataFrameReader,
and `format` function specifies the input data source format for the DataFrameReader.
The `load()` function loads input in as a DataFrame. The date frame `df` returned
by `withCatalog` function could be used to access HBase table, such as 4.4 and 4.5.
=== Language Integrated Query
[source, scala]
----
val s = df.filter(($"col0" <= "row050" && $"col0" > "row040") ||
$"col0" === "row005" ||
$"col0" <= "row005")
.select("col0", "col1", "col4")
s.show
----
DataFrame can do various operations, such as join, sort, select, filter, orderBy and so on.
`df.filter` above filters rows using the given SQL expression. `select` selects a set of columns:
`col0`, `col1` and `col4`.
=== SQL Query
[source, scala]
----
df.registerTempTable("table1")
sqlContext.sql("select count(col1) from table1").show
----
`registerTempTable` registers `df` DataFrame as a temporary table using the table name `table1`.
The lifetime of this temporary table is tied to the SQLContext that was used to create `df`.
`sqlContext.sql` function allows the user to execute SQL queries.
=== Others
.Query with different timestamps
====
In HBaseSparkConf, four parameters related to timestamp can be set. They are TIMESTAMP,
MIN_TIMESTAMP, MAX_TIMESTAMP and MAX_VERSIONS respectively. Users can query records with
different timestamps or time ranges with MIN_TIMESTAMP and MAX_TIMESTAMP. In the meantime,
use concrete value instead of tsSpecified and oldMs in the examples below.
The example below shows how to load df DataFrame with different timestamps.
tsSpecified is specified by the user.
HBaseTableCatalog defines the HBase and Relation relation schema.
writeCatalog defines catalog for the schema mapping.
[source, scala]
----
val df = sqlContext.read
.options(Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseSparkConf.TIMESTAMP -> tsSpecified.toString))
.format("org.apache.hadoop.hbase.spark")
.load()
----
The example below shows how to load df DataFrame with different time ranges.
oldMs is specified by the user.
[source, scala]
----
val df = sqlContext.read
.options(Map(HBaseTableCatalog.tableCatalog -> writeCatalog, HBaseSparkConf.MIN_TIMESTAMP -> "0",
HBaseSparkConf.MAX_TIMESTAMP -> oldMs.toString))
.format("org.apache.hadoop.hbase.spark")
.load()
----
After loading df DataFrame, users can query data.
[source, scala]
----
df.registerTempTable("table")
sqlContext.sql("select count(col1) from table").show
----
====
.Native Avro support
====
HBase-Spark Connector support different data formats like Avro, Jason, etc. The use case below
shows how spark supports Avro. User can persist the Avro record into HBase directly. Internally,
the Avro schema is converted to a native Spark Catalyst data type automatically.
Note that both key-value parts in an HBase table can be defined in Avro format.
1) Define catalog for the schema mapping:
[source, scala]
----
def catalog = s"""{
|"table":{"namespace":"default", "name":"Avrotable"},
|"rowkey":"key",
|"columns":{
|"col0":{"cf":"rowkey", "col":"key", "type":"string"},
|"col1":{"cf":"cf1", "col":"col1", "type":"binary"}
|}
|}""".stripMargin
----
`catalog` is a schema for a HBase table named `Avrotable`. row key as key and
one column col1. The rowkey also has to be defined in details as a column (col0),
which has a specific cf (rowkey).
2) Prepare the Data:
[source, scala]
----
object AvroHBaseRecord {
val schemaString =
s"""{"namespace": "example.avro",
| "type": "record", "name": "User",
| "fields": [
| {"name": "name", "type": "string"},
| {"name": "favorite_number", "type": ["int", "null"]},
| {"name": "favorite_color", "type": ["string", "null"]},
| {"name": "favorite_array", "type": {"type": "array", "items": "string"}},
| {"name": "favorite_map", "type": {"type": "map", "values": "int"}}
| ] }""".stripMargin
val avroSchema: Schema = {
val p = new Schema.Parser
p.parse(schemaString)
}
def apply(i: Int): AvroHBaseRecord = {
val user = new GenericData.Record(avroSchema);
user.put("name", s"name${"%03d".format(i)}")
user.put("favorite_number", i)
user.put("favorite_color", s"color${"%03d".format(i)}")
val favoriteArray = new GenericData.Array[String](2, avroSchema.getField("favorite_array").schema())
favoriteArray.add(s"number${i}")
favoriteArray.add(s"number${i+1}")
user.put("favorite_array", favoriteArray)
import collection.JavaConverters._
val favoriteMap = Map[String, Int](("key1" -> i), ("key2" -> (i+1))).asJava
user.put("favorite_map", favoriteMap)
val avroByte = AvroSedes.serialize(user, avroSchema)
AvroHBaseRecord(s"name${"%03d".format(i)}", avroByte)
}
}
val data = (0 to 255).map { i =>
AvroHBaseRecord(i)
}
----
`schemaString` is defined first, then it is parsed to get `avroSchema`. `avroSchema` is used to
generate `AvroHBaseRecord`. `data` prepared by users is a local Scala collection
which has 256 `AvroHBaseRecord` objects.
3) Save DataFrame:
[source, scala]
----
sc.parallelize(data).toDF.write.options(
Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
.format("org.apache.spark.sql.execution.datasources.hbase")
.save()
----
Given a data frame with specified schema `catalog`, above will create an HBase table with 5
regions and save the data frame inside.
4) Load the DataFrame
[source, scala]
----
def avroCatalog = s"""{
|"table":{"namespace":"default", "name":"avrotable"},
|"rowkey":"key",
|"columns":{
|"col0":{"cf":"rowkey", "col":"key", "type":"string"},
|"col1":{"cf":"cf1", "col":"col1", "avro":"avroSchema"}
|}
|}""".stripMargin
def withCatalog(cat: String): DataFrame = {
sqlContext
.read
.options(Map("avroSchema" -> AvroHBaseRecord.schemaString, HBaseTableCatalog.tableCatalog -> avroCatalog))
.format("org.apache.spark.sql.execution.datasources.hbase")
.load()
}
val df = withCatalog(catalog)
----
In `withCatalog` function, `read` returns a DataFrameReader that can be used to read data in as a DataFrame.
The `option` function adds input options for the underlying data source to the DataFrameReader.
There are two options: one is to set `avroSchema` as `AvroHBaseRecord.schemaString`, and one is to
set `HBaseTableCatalog.tableCatalog` as `avroCatalog`. The `load()` function loads input in as a DataFrame.
The date frame `df` returned by `withCatalog` function could be used to access the HBase table.
5) SQL Query
[source, scala]
----
df.registerTempTable("avrotable")
val c = sqlContext.sql("select count(1) from avrotable").
----
After loading df DataFrame, users can query data. registerTempTable registers df DataFrame
as a temporary table using the table name avrotable. `sqlContext.sql` function allows the
user to execute SQL queries.
====

View File

@ -65,7 +65,6 @@ include::_chapters/hbase_mob.adoc[]
include::_chapters/hbase_apis.adoc[]
include::_chapters/external_apis.adoc[]
include::_chapters/thrift_filter_language.adoc[]
include::_chapters/spark.adoc[]
include::_chapters/cp.adoc[]
include::_chapters/performance.adoc[]
include::_chapters/troubleshooting.adoc[]