incubator-blur-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From amccu...@apache.org
Subject [1/4] git commit: Spark Blur Connector
Date Mon, 12 Jan 2015 04:21:50 GMT
Repository: incubator-blur
Updated Branches:
  refs/heads/master 4728ce990 -> 04236cfb5


Spark Blur Connector

Signed-off-by: Aaron McCurry <amccurry@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-blur/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-blur/commit/dbd529a2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-blur/tree/dbd529a2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-blur/diff/dbd529a2

Branch: refs/heads/master
Commit: dbd529a266228f4e5642aa696e364cd1a9f53db4
Parents: 4728ce9
Author: Dibyendu Bhattacharya <dibyendu.bhattacharya1@pearson.com>
Authored: Sun Oct 26 11:41:52 2014 +0530
Committer: Aaron McCurry <amccurry@gmail.com>
Committed: Sun Jan 11 16:28:02 2015 -0500

----------------------------------------------------------------------
 contrib/blur-spark/README                       |  71 +++++
 contrib/blur-spark/pom.xml                      | 169 +++++++++++
 .../apache/blur/spark/BlurSparkPartitioner.java |  45 +++
 .../java/org/apache/blur/spark/Consumer.java    | 280 +++++++++++++++++++
 .../org/apache/blur/spark/ConsumerEnqueue.java  | 240 ++++++++++++++++
 .../services/org.apache.lucene.codecs.Codec     |  19 ++
 6 files changed, 824 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/dbd529a2/contrib/blur-spark/README
----------------------------------------------------------------------
diff --git a/contrib/blur-spark/README b/contrib/blur-spark/README
new file mode 100644
index 0000000..73eb83b
--- /dev/null
+++ b/contrib/blur-spark/README
@@ -0,0 +1,71 @@
+
+This Spark Blur connector index Kafka Messages to Apache Blur using following Kafka Consumer.
+
+https://github.com/dibbhatt/kafka-spark-consumer
+
+This Fault Tolerant Kafka Consumer uses Low Level Kafka API to pull messages from Kafka Topic
Partition using Spark Custom Receiver.
+
+For more details please refer to : https://github.com/dibbhatt/kafka-spark-consumer
+
+Spark Blur Connector use this Kafka Consumer to index Kafka Messages using Spark Hadoop APIs.

+
+This Project used Custom Blur Outputformat, RecordWriter and OutputComitter which has some
minor modification from existing Blur Hadoop1 codebase.
+
+The Kafka DStream is repartitioned into number of partitions which is same as number of Shards
of Target Blur Table.
+
+This connector uses a Custom Spark Partitioner to map keys to correct RDD partition which
intern maps to same Blur Shard.
+
+
+Following are the instructions to build 
+========================================
+
+You need to install https://github.com/dibbhatt/kafka-spark-consumer first using maven
+
+Then package Blur Connector
+
+>git clone https://github.com/dibbhatt/spark-blur-connector
+
+>cd spark-blur-connector
+
+>mvn package
+
+Integrating Spark-Kafka-Consumer
+=================================
+
+Below are the properties file for Kafka Spark Consumer.
+
+spark-kafka.properties
+=====================
+#Kafka ZK details from where messages will be pulled
+
+#Kafka ZK host details
+zookeeper.hosts=host1,host2
+#Kafka ZK Port
+zookeeper.port=2181
+# Kafka Broker path in ZK
+zookeeper.broker.path=/brokers
+#Kafka Topic to consume
+kafka.topic=topic-name
+
+#Consumer ZK Path. This will be used to store the consumed offset
+zookeeper.consumer.connection=localhost:2182
+#ZK Path for storing Kafka Consumer offset
+zookeeper.consumer.path=/spark-kafka
+# Kafka Consumer ID. This ID will be used for accessing offset details in $zookeeper.consumer.path
+kafka.consumer.id=12345
+
+
+Running Spark Kafka Consumer
+===========================
+
+Launch this using spark-submit
+
+./bin/spark-submit --class org.apache.blur.spark.Consumer --master spark://x.x.x.x:7077 --executor-memory
5G /<Path_To>/spark-blur-connector-0.0.1-SNAPSHOT-jar-with-dependencies.jar -P /<Path_To>/spark-kafka.properties
+
+
+-P external properties filename
+-p properties filename from the classpath
+
+This will start the Spark Receiver and Fetch Kafka Messages for every partition of the given
topic and generates the DStream and start writing indexing the Kafka Stream into Blur Table.
+
+ 

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/dbd529a2/contrib/blur-spark/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/blur-spark/pom.xml b/contrib/blur-spark/pom.xml
new file mode 100644
index 0000000..63a6b37
--- /dev/null
+++ b/contrib/blur-spark/pom.xml
@@ -0,0 +1,169 @@
+<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>
+
+	<groupId>spark-blur-connector</groupId>
+	<artifactId>spark-blur-connector</artifactId>
+	<version>0.0.1-SNAPSHOT</version>
+	<packaging>jar</packaging>
+
+	<name>spark-blur-connector</name>
+
+	<repositories>
+		<repository>
+			<id>apache-releases</id>
+			<url>https://repository.apache.org/content/groups/public</url>
+		</repository>
+		<repository>
+			<id>restlet-releases</id>
+			<url>http://maven.restlet.org/</url>
+		</repository>
+		<repository>
+			<id>libdir</id>
+			<url>file://${basedir}/../lib</url>
+		</repository>
+	</repositories>
+
+	<build>
+
+		<resources>
+			<resource>
+				<directory>src/main/resources/services</directory>
+				<targetPath>META-INF/services</targetPath>
+			</resource>
+		</resources>
+
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+			</plugin>
+			<plugin>
+				<artifactId>maven-assembly-plugin</artifactId>
+				<configuration>
+					<descriptorRefs>
+						<descriptorRef>jar-with-dependencies</descriptorRef>
+					</descriptorRefs>
+				</configuration>
+				<executions>
+					<execution>
+						<id>make-assembly</id>
+						<phase>package</phase>
+						<goals>
+							<goal>single</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.blur</groupId>
+			<artifactId>blur-thrift</artifactId>
+			<version>0.2.4-incubating-SNAPSHOT-hadoop1</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.blur</groupId>
+			<artifactId>blur-util</artifactId>
+			<version>0.2.4-incubating-SNAPSHOT-hadoop1</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.blur</groupId>
+			<artifactId>blur-core</artifactId>
+			<version>0.2.4-incubating-SNAPSHOT-hadoop1</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.blur</groupId>
+			<artifactId>blur-store</artifactId>
+			<version>0.2.4-incubating-SNAPSHOT-hadoop1</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.blur</groupId>
+			<artifactId>blur-query</artifactId>
+			<version>0.2.4-incubating-SNAPSHOT-hadoop1</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.blur</groupId>
+			<artifactId>blur-mapred-hadoop1</artifactId>
+			<version>0.2.4-incubating-SNAPSHOT-hadoop1</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.lucene</groupId>
+			<artifactId>lucene-core</artifactId>
+			<version>4.3.0</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.lucene</groupId>
+			<artifactId>lucene-spatial</artifactId>
+			<version>4.3.0</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.lucene</groupId>
+			<artifactId>lucene-codecs</artifactId>
+			<version>4.3.0</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.lucene</groupId>
+			<artifactId>lucene-analyzers-common</artifactId>
+			<version>4.3.0</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.lucene</groupId>
+			<artifactId>lucene-queries</artifactId>
+			<version>4.3.0</version>
+		</dependency>
+		<dependency>
+			<groupId>junit</groupId>
+			<artifactId>junit</artifactId>
+			<version>4.11</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>commons-codec</groupId>
+			<artifactId>commons-codec</artifactId>
+			<version>1.3</version>
+		</dependency>
+		<dependency>
+			<groupId>commons-logging</groupId>
+			<artifactId>commons-logging</artifactId>
+			<version>1.1.3</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-core</artifactId>
+			<version>1.2.1</version>
+		</dependency>
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-log4j12</artifactId>
+			<version>1.7.4</version>
+		</dependency>
+		<dependency>
+			<groupId>kafka.spark.consumer</groupId>
+			<artifactId>kafka-spark-consumer</artifactId>
+			<version>0.0.1-SNAPSHOT</version>
+		</dependency>
+	</dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/dbd529a2/contrib/blur-spark/src/main/java/org/apache/blur/spark/BlurSparkPartitioner.java
----------------------------------------------------------------------
diff --git a/contrib/blur-spark/src/main/java/org/apache/blur/spark/BlurSparkPartitioner.java
b/contrib/blur-spark/src/main/java/org/apache/blur/spark/BlurSparkPartitioner.java
new file mode 100644
index 0000000..728ad36
--- /dev/null
+++ b/contrib/blur-spark/src/main/java/org/apache/blur/spark/BlurSparkPartitioner.java
@@ -0,0 +1,45 @@
+package org.apache.blur.spark;
+
+/**
+ * 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.hadoop.io.Text;
+import org.apache.spark.HashPartitioner;
+
+public class BlurSparkPartitioner extends HashPartitioner {
+
+	private static final long serialVersionUID = 9853263327838L;
+
+	int totalShard;
+
+	public BlurSparkPartitioner(int partitions) {
+
+		super(partitions);
+		totalShard = partitions;
+	}
+
+	@Override
+	public int getPartition(Object key) {
+
+		if (key instanceof Text) {
+
+			return (key.hashCode() & Integer.MAX_VALUE) % totalShard;
+		} else {
+			return super.getPartition(key);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/dbd529a2/contrib/blur-spark/src/main/java/org/apache/blur/spark/Consumer.java
----------------------------------------------------------------------
diff --git a/contrib/blur-spark/src/main/java/org/apache/blur/spark/Consumer.java b/contrib/blur-spark/src/main/java/org/apache/blur/spark/Consumer.java
new file mode 100644
index 0000000..962faa4
--- /dev/null
+++ b/contrib/blur-spark/src/main/java/org/apache/blur/spark/Consumer.java
@@ -0,0 +1,280 @@
+package org.apache.blur.spark;
+
+/**
+ * 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 java.io.File;
+import java.io.FileInputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.blur.manager.BlurPartitioner;
+import org.apache.blur.mapreduce.lib.BlurMapReduceUtil;
+import org.apache.blur.mapreduce.lib.BlurMutate;
+import org.apache.blur.mapreduce.lib.BlurOutputCommitter;
+import org.apache.blur.mapreduce.lib.BlurOutputFormat;
+import org.apache.blur.mapreduce.lib.DefaultBlurReducer;
+import org.apache.blur.thrift.generated.TableDescriptor;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.PosixParser;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.Time;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+
+import scala.Tuple2;
+import consumer.kafka.KafkaConfig;
+import consumer.kafka.MessageAndMetadata;
+import consumer.kafka.client.KafkaReceiver;
+
+/*
+ * This Consumer uses Spark RDD saveAsNewAPIHadoopFile API to index BlurMutate
+ */
+
+public class Consumer implements Serializable {
+
+	private static final long serialVersionUID = 4332618245650072140L;
+	private Properties _props;
+	private KafkaConfig _kafkaConfig;
+
+	public void start() throws InstantiationException, IllegalAccessException,
+			ClassNotFoundException {
+
+		_kafkaConfig = new KafkaConfig(_props);
+		run();
+	}
+
+	private void init(String[] args) throws Exception {
+
+		Options options = new Options();
+		this._props = new Properties();
+
+		options.addOption("p", true, "properties filename from the classpath");
+		options.addOption("P", true, "external properties filename");
+
+		OptionBuilder.withArgName("property=value");
+		OptionBuilder.hasArgs(2);
+		OptionBuilder.withValueSeparator();
+		OptionBuilder.withDescription("use value for given property");
+		options.addOption(OptionBuilder.create("D"));
+
+		CommandLineParser parser = new PosixParser();
+		CommandLine cmd = parser.parse(options, args);
+		if (cmd.hasOption('p')) {
+			this._props.load(ClassLoader.getSystemClassLoader()
+					.getResourceAsStream(cmd.getOptionValue('p')));
+		}
+		if (cmd.hasOption('P')) {
+			File file = new File(cmd.getOptionValue('P'));
+			FileInputStream fStream = new FileInputStream(file);
+			this._props.load(fStream);
+		}
+		this._props.putAll(cmd.getOptionProperties("D"));
+
+	}
+
+	private void run() {
+
+		String checkpointDirectory = "hdfs://10.252.5.113:9000/user/hadoop/spark";
+
+		// number of partition for Kafka Topic
+
+		int _partitionCount = 5;
+
+		List<JavaDStream<MessageAndMetadata>> streamsList = new ArrayList<JavaDStream<MessageAndMetadata>>(
+				_partitionCount);
+		JavaDStream<MessageAndMetadata> unionStreams;
+
+		SparkConf conf = new SparkConf().setAppName("KafkaReceiver").set(
+				"spark.streaming.blockInterval", "200");
+
+		// Path to Blur Libraries . Can be copied to each Node of Spark Cluster.
+
+		conf.set("spark.executor.extraClassPath",
+				"/home/apache-blur-0.2.4/lib/*");
+
+		// Used KryoSerializer for BlurMutate and Text.
+		conf.set("spark.serializer",
+				"org.apache.spark.serializer.KryoSerializer");
+
+		JavaStreamingContext ssc = new JavaStreamingContext(conf, new Duration(
+				3000));
+
+		/*
+		 * Receive Kafka Stream. Create individual Receivers for each Topic
+		 * Partition
+		 */
+
+		for (int i = 0; i < _partitionCount; i++) {
+
+			streamsList.add(ssc.receiverStream(new KafkaReceiver(_props, i)));
+
+		}
+
+		/*
+		 * Union all the streams if there is more than 1 stream
+		 */
+
+		if (streamsList.size() > 1) {
+			unionStreams = ssc.union(streamsList.get(0),
+					streamsList.subList(1, streamsList.size()));
+		} else {
+			// Otherwise, just use the 1 stream
+			unionStreams = streamsList.get(0);
+		}
+
+		/*
+		 * Generate JavaPairDStream
+		 */
+
+		JavaPairDStream<Text, BlurMutate> pairDStream = unionStreams
+				.mapToPair(new PairFunction<MessageAndMetadata, Text, BlurMutate>() {
+
+					private static final long serialVersionUID = 443235214978L;
+
+					public Tuple2<Text, BlurMutate> call(
+							MessageAndMetadata mmeta) {
+
+						/*
+						 * create the BlurMutate from MessageAndMetadata
+						 */
+
+						String message = new String(mmeta.getPayload());
+						String keyStr = DigestUtils.shaHex(message);
+						Text key = new Text((keyStr).getBytes());
+						BlurMutate mutate = new BlurMutate(
+								BlurMutate.MUTATE_TYPE.REPLACE, keyStr, keyStr,
+								"family");
+						mutate.addColumn("message", message);
+
+						return new Tuple2<Text, BlurMutate>(key, mutate);
+					}
+				});
+
+		pairDStream
+				.foreachRDD(new Function2<JavaPairRDD<Text, BlurMutate>, Time, Void>() {
+
+					private static final long serialVersionUID = 88875777435L;
+
+					@Override
+					public Void call(JavaPairRDD<Text, BlurMutate> rdd,
+							Time time) throws Exception {
+
+						/*
+						 * Blur Table Details
+						 */
+						TableDescriptor tableDescriptor = new TableDescriptor();
+						String tableUri = new Path(
+								"hdfs://10.252.5.113:9000/blur/tables/nrt")
+								.toString();
+						tableDescriptor.tableUri = tableUri;
+						tableDescriptor.cluster = "pearson";
+						tableDescriptor.name = "nrt";
+						tableDescriptor.shardCount = 9;
+						Configuration conf = new Configuration();
+
+						/*
+						 * Partition RDD to match Blur Table Shard Count. Used
+						 * Custom Partitioner to channel correct BlurMutate to
+						 * correct Shard.
+						 */
+
+						final JavaPairRDD<Text, BlurMutate> pRdd = rdd
+								.partitionBy(
+										new BlurSparkPartitioner(
+												tableDescriptor.shardCount))
+								.persist(StorageLevel.MEMORY_ONLY_2());
+
+						/*
+						 * Blur specific Configuration
+						 */
+
+						BlurOutputFormat.setIndexLocally(conf, false);
+						BlurOutputFormat.setOptimizeInFlight(conf, false);
+						conf.setClass("mapreduce.reduce.class",
+								DefaultBlurReducer.class, Reducer.class);
+						conf.setClass("mapreduce.outputformat.class",
+								BlurOutputFormat.class, OutputFormat.class);
+						conf.setClass("mapreduce.partitioner.class",
+								BlurPartitioner.class, Partitioner.class);
+						conf.set("mapred.output.committer.class",
+								BlurOutputCommitter.class.getName());
+						conf.setInt("blur.output.max.document.buffer.size",
+								10000);
+
+						BlurOutputFormat.setTableDescriptor(conf,
+								tableDescriptor);
+
+						JobConf jobConf = new JobConf(conf);
+
+						jobConf.setNumReduceTasks(tableDescriptor.shardCount);
+						jobConf.setOutputKeyClass(Text.class);
+						jobConf.setOutputValueClass(BlurMutate.class);
+
+						BlurMapReduceUtil.addAllJarsInBlurLib(conf);
+						BlurMapReduceUtil
+								.addDependencyJars(
+										conf,
+										org.apache.zookeeper.ZooKeeper.class,
+										org.apache.lucene.codecs.lucene42.Lucene42Codec.class,
+										jobConf.getOutputKeyClass(),
+										jobConf.getOutputValueClass());
+
+						/*
+						 * Write the RDD to Blur Table
+						 */
+
+						if (pRdd.count() > 0)
+							pRdd.saveAsNewAPIHadoopFile(tableUri, Text.class,
+									BlurMutate.class, BlurOutputFormat.class,
+									jobConf);
+
+						return null;
+					}
+				});
+
+		// ssc.checkpoint(checkpointDirectory);
+		ssc.start();
+		ssc.awaitTermination();
+	}
+
+	public static void main(String[] args) throws Exception {
+
+		Consumer consumer = new Consumer();
+		consumer.init(args);
+		consumer.start();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/dbd529a2/contrib/blur-spark/src/main/java/org/apache/blur/spark/ConsumerEnqueue.java
----------------------------------------------------------------------
diff --git a/contrib/blur-spark/src/main/java/org/apache/blur/spark/ConsumerEnqueue.java b/contrib/blur-spark/src/main/java/org/apache/blur/spark/ConsumerEnqueue.java
new file mode 100644
index 0000000..25f2d58
--- /dev/null
+++ b/contrib/blur-spark/src/main/java/org/apache/blur/spark/ConsumerEnqueue.java
@@ -0,0 +1,240 @@
+package org.apache.blur.spark;
+
+/**
+ * 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 java.io.File;
+import java.io.FileInputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.blur.thrift.BlurClient;
+import org.apache.blur.thrift.generated.Blur.Iface;
+import org.apache.blur.thrift.generated.Column;
+import org.apache.blur.thrift.generated.Record;
+import org.apache.blur.thrift.generated.RecordMutation;
+import org.apache.blur.thrift.generated.RecordMutationType;
+import org.apache.blur.thrift.generated.RowMutation;
+import org.apache.blur.thrift.generated.RowMutationType;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.PosixParser;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.Time;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+
+import scala.Tuple2;
+import consumer.kafka.KafkaConfig;
+import consumer.kafka.MessageAndMetadata;
+import consumer.kafka.client.KafkaReceiver;
+
+
+/*
+ * This Consumer uses Blur Thrift Client enqueue mutate call to index Rowmutation
+ */
+public class ConsumerEnqueue implements Serializable {
+
+	private static final long serialVersionUID = 4332618245650072140L;
+	private Properties _props;
+	private KafkaConfig _kafkaConfig;
+
+	public void start() throws InstantiationException, IllegalAccessException,
+			ClassNotFoundException {
+
+		_kafkaConfig = new KafkaConfig(_props);
+		run();
+	}
+
+	private void init(String[] args) throws Exception {
+
+		Options options = new Options();
+		this._props = new Properties();
+
+		options.addOption("p", true, "properties filename from the classpath");
+		options.addOption("P", true, "external properties filename");
+
+		OptionBuilder.withArgName("property=value");
+		OptionBuilder.hasArgs(2);
+		OptionBuilder.withValueSeparator();
+		OptionBuilder.withDescription("use value for given property");
+		options.addOption(OptionBuilder.create("D"));
+
+		CommandLineParser parser = new PosixParser();
+		CommandLine cmd = parser.parse(options, args);
+		if (cmd.hasOption('p')) {
+			this._props.load(ClassLoader.getSystemClassLoader()
+					.getResourceAsStream(cmd.getOptionValue('p')));
+		}
+		if (cmd.hasOption('P')) {
+			File file = new File(cmd.getOptionValue('P'));
+			FileInputStream fStream = new FileInputStream(file);
+			this._props.load(fStream);
+		}
+		this._props.putAll(cmd.getOptionProperties("D"));
+
+	}
+
+	private void run() {
+
+		String checkpointDirectory = "hdfs://10.252.5.113:9000/user/hadoop/spark";
+
+		// number of partition for Kafka Topic
+
+		int _partitionCount = 5;
+
+		List<JavaDStream<MessageAndMetadata>> streamsList = new ArrayList<JavaDStream<MessageAndMetadata>>(
+				_partitionCount);
+		JavaDStream<MessageAndMetadata> unionStreams;
+
+		SparkConf conf = new SparkConf().setAppName("KafkaReceiver").set(
+				"spark.streaming.blockInterval", "200");
+
+		// Path to Blur Libraries . Can be copied to each Node of Spark Cluster.
+
+		conf.set("spark.executor.extraClassPath",
+				"/home/apache-blur-0.2.4/lib/*");
+
+		// Used KryoSerializer for BlurMutate and Text.
+		conf.set("spark.serializer",
+				"org.apache.spark.serializer.KryoSerializer");
+
+		JavaStreamingContext ssc = new JavaStreamingContext(conf, new Duration(
+				3000));
+
+		/*
+		 * Receive Kafka Stream. Create individual Receivers for each Topic
+		 * Partition
+		 */
+
+		for (int i = 0; i < _partitionCount; i++) {
+
+			streamsList.add(ssc.receiverStream(new KafkaReceiver(_props, i)));
+
+		}
+
+		/*
+		 * Union all the streams if there is more than 1 stream
+		 */
+
+		if (streamsList.size() > 1) {
+			unionStreams = ssc.union(streamsList.get(0),
+					streamsList.subList(1, streamsList.size()));
+		} else {
+			// Otherwise, just use the 1 stream
+			unionStreams = streamsList.get(0);
+		}
+
+		/*
+		 * Generate JavaPairDStream
+		 */
+
+		JavaPairDStream<String, RowMutation> pairDStream = unionStreams
+				.mapToPair(new PairFunction<MessageAndMetadata, String, RowMutation>() {
+
+					private static final long serialVersionUID = 443235214978L;
+
+					public Tuple2<String, RowMutation> call(
+							MessageAndMetadata mmeta) {
+
+						/*
+						 * create the RowMutation from MessageAndMetadata
+						 */
+
+						String message = new String(mmeta.getPayload());
+						String keyStr = DigestUtils.shaHex(message);
+
+						Record record = new Record();
+						record.setRecordId(keyStr);
+						record.addToColumns(new Column("message", message));
+						record.setFamily("family");
+
+						List recordMutations = new ArrayList();
+						recordMutations.add(new RecordMutation(
+								RecordMutationType.REPLACE_ENTIRE_RECORD,
+								record));
+						RowMutation mutation = new RowMutation("nrt", keyStr,
+								RowMutationType.REPLACE_ROW, recordMutations);
+						mutation.setRecordMutations(recordMutations);
+
+						return new Tuple2<String, RowMutation>(keyStr, mutation);
+					}
+				});
+
+		pairDStream
+				.foreachRDD(new Function2<JavaPairRDD<String, RowMutation>, Time, Void>()
{
+
+					private static final long serialVersionUID = 88875777435L;
+					
+					/*
+					 * Blur Thrift Client
+					 */
+
+					Iface client = BlurClient.getClient("10.252.5.113:40010");
+
+					@Override
+					public Void call(JavaPairRDD<String, RowMutation> rdd,
+							Time time) throws Exception {
+
+						for (Tuple2<String, RowMutation> tuple : rdd.collect()) {
+
+							if (tuple != null) {
+
+								try {
+
+									RowMutation rm = tuple._2;
+									
+									/*
+									 * Index using enqueue mutate call
+									 */
+									client.enqueueMutate(rm);
+
+								} catch (Exception ex) {
+
+									ex.printStackTrace();
+								}
+
+							}
+
+						}
+
+						return null;
+					}
+				});
+
+		// ssc.checkpoint(checkpointDirectory);
+		ssc.start();
+		ssc.awaitTermination();
+	}
+
+	public static void main(String[] args) throws Exception {
+
+		ConsumerEnqueue consumer = new ConsumerEnqueue();
+		consumer.init(args);
+		consumer.start();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-blur/blob/dbd529a2/contrib/blur-spark/src/main/resources/services/org.apache.lucene.codecs.Codec
----------------------------------------------------------------------
diff --git a/contrib/blur-spark/src/main/resources/services/org.apache.lucene.codecs.Codec
b/contrib/blur-spark/src/main/resources/services/org.apache.lucene.codecs.Codec
new file mode 100644
index 0000000..fe8abbd
--- /dev/null
+++ b/contrib/blur-spark/src/main/resources/services/org.apache.lucene.codecs.Codec
@@ -0,0 +1,19 @@
+#  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.
+
+
+org.apache.blur.lucene.codec.Blur021Codec
+org.apache.blur.lucene.codec.Blur022Codec
+org.apache.lucene.codecs.lucene42.Lucene42Codec


Mime
View raw message