├── .gitignore ├── src ├── main │ ├── resources │ │ └── log4j.xml │ ├── config │ │ ├── luwak.properties │ │ └── combiner.properties │ ├── java │ │ └── uk │ │ │ └── co │ │ │ └── flax │ │ │ └── samzaluwak │ │ │ ├── Interact.java │ │ │ ├── MatchRecombinerTask.java │ │ │ └── MonitorTask.java │ └── assembly │ │ └── src.xml └── test │ └── java │ └── uk │ └── co │ └── flax │ └── samzaluwak │ └── TestMonitorTask.java ├── README.md └── pom.xml /.gitignore: -------------------------------------------------------------------------------- 1 | target 2 | deploy 3 | *.iml 4 | 5 | .idea/ 6 | -------------------------------------------------------------------------------- /src/main/resources/log4j.xml: -------------------------------------------------------------------------------- 1 | 2 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | -------------------------------------------------------------------------------- /src/main/config/luwak.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | # Job 19 | job.factory.class=org.apache.samza.job.yarn.YarnJobFactory 20 | job.name=luwak 21 | 22 | # YARN 23 | yarn.package.path=file://${basedir}/target/${project.artifactId}-${pom.version}-dist.tar.gz 24 | 25 | # Task 26 | task.class=uk.co.flax.samzaluwak.MonitorTask 27 | task.inputs=kafka.queries,kafka.documents 28 | task.checkpoint.factory=org.apache.samza.checkpoint.kafka.KafkaCheckpointManagerFactory 29 | task.checkpoint.system=kafka 30 | task.checkpoint.replication.factor=1 31 | task.consumer.batch.size=1 32 | 33 | # Serializers 34 | serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory 35 | serializers.registry.string.class=org.apache.samza.serializers.StringSerdeFactory 36 | 37 | # Kafka System 38 | systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory 39 | systems.kafka.samza.msg.serde=json 40 | systems.kafka.samza.key.serde=string 41 | systems.kafka.consumer.zookeeper.connect=localhost:2181/ 42 | systems.kafka.producer.metadata.broker.list=localhost:9092 43 | systems.kafka.producer.producer.type=sync 44 | # Normally, we'd set this much higher, but we want things to look snappy in the demo. 45 | systems.kafka.producer.batch.num.messages=1 46 | 47 | systems.kafka.streams.queries.samza.bootstrap=true 48 | systems.kafka.streams.queries.samza.reset.offset=true 49 | systems.kafka.streams.queries.samza.offset.default=oldest 50 | -------------------------------------------------------------------------------- /src/main/config/combiner.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | # Job 19 | job.factory.class=org.apache.samza.job.yarn.YarnJobFactory 20 | job.name=combiner 21 | 22 | # YARN 23 | yarn.package.path=file://${basedir}/target/${project.artifactId}-${pom.version}-dist.tar.gz 24 | 25 | # Task 26 | task.class=uk.co.flax.samzaluwak.MatchRecombinerTask 27 | task.inputs=kafka.matches1 28 | task.checkpoint.factory=org.apache.samza.checkpoint.kafka.KafkaCheckpointManagerFactory 29 | task.checkpoint.system=kafka 30 | task.checkpoint.replication.factor=1 31 | task.consumer.batch.size=1 32 | 33 | # Serializers 34 | serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory 35 | serializers.registry.string.class=org.apache.samza.serializers.StringSerdeFactory 36 | 37 | # Kafka System 38 | systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory 39 | systems.kafka.samza.msg.serde=json 40 | systems.kafka.samza.key.serde=string 41 | systems.kafka.consumer.zookeeper.connect=localhost:2181/ 42 | systems.kafka.producer.metadata.broker.list=localhost:9092 43 | systems.kafka.producer.producer.type=sync 44 | # Normally, we'd set this much higher, but we want things to look snappy in the demo. 45 | systems.kafka.producer.batch.num.messages=1 46 | 47 | stores.matches.factory=org.apache.samza.storage.kv.LevelDbKeyValueStorageEngineFactory 48 | stores.matches.changelog=kafka.matches-combiner-changelog 49 | stores.matches.key.serde=string 50 | stores.matches.msg.serde=json -------------------------------------------------------------------------------- /src/test/java/uk/co/flax/samzaluwak/TestMonitorTask.java: -------------------------------------------------------------------------------- 1 | package uk.co.flax.samzaluwak; 2 | 3 | import com.google.common.collect.ImmutableMap; 4 | import org.apache.samza.Partition; 5 | import org.apache.samza.system.IncomingMessageEnvelope; 6 | import org.apache.samza.system.OutgoingMessageEnvelope; 7 | import org.apache.samza.system.SystemStreamPartition; 8 | import org.apache.samza.task.MessageCollector; 9 | import org.junit.Test; 10 | 11 | import static org.mockito.Matchers.any; 12 | import static org.mockito.Mockito.mock; 13 | import static org.mockito.Mockito.times; 14 | import static org.mockito.Mockito.verify; 15 | 16 | /** 17 | * Copyright (c) 2014 Lemur Consulting Ltd. 18 | *

19 | * Licensed under the Apache License, Version 2.0 (the "License"); 20 | * you may not use this file except in compliance with the License. 21 | * You may obtain a copy of the License at 22 | *

23 | * http://www.apache.org/licenses/LICENSE-2.0 24 | *

25 | * Unless required by applicable law or agreed to in writing, software 26 | * distributed under the License is distributed on an "AS IS" BASIS, 27 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 28 | * See the License for the specific language governing permissions and 29 | * limitations under the License. 30 | */ 31 | 32 | public class TestMonitorTask { 33 | 34 | public static final SystemStreamPartition QUERY_PART = new SystemStreamPartition("kafka", MonitorTask.QUERIES_STREAM, new Partition(0)); 35 | public static final SystemStreamPartition DOCS_PART = new SystemStreamPartition("kafka", MonitorTask.DOCS_STREAM, new Partition(0)); 36 | 37 | @Test 38 | public void testTask() throws Exception { 39 | 40 | MonitorTask task = new MonitorTask(); 41 | task.init(null, null); 42 | 43 | MessageCollector collector = mock(MessageCollector.class); 44 | 45 | IncomingMessageEnvelope query = new IncomingMessageEnvelope(QUERY_PART, "", "1", ImmutableMap.of("query", "hello world")); 46 | task.process(query, collector, null); 47 | 48 | IncomingMessageEnvelope doc = new IncomingMessageEnvelope(DOCS_PART, "", "doc1", ImmutableMap.of("f", "hello world")); 49 | task.process(doc, collector, null); 50 | 51 | verify(collector, times(1)).send(any(OutgoingMessageEnvelope.class)); 52 | 53 | } 54 | 55 | 56 | } 57 | -------------------------------------------------------------------------------- /src/main/java/uk/co/flax/samzaluwak/Interact.java: -------------------------------------------------------------------------------- 1 | package uk.co.flax.samzaluwak; 2 | 3 | import java.io.BufferedReader; 4 | import java.io.IOException; 5 | import java.io.InputStreamReader; 6 | import java.util.Properties; 7 | 8 | import kafka.javaapi.producer.Producer; 9 | import kafka.producer.KeyedMessage; 10 | import kafka.producer.ProducerConfig; 11 | 12 | /** 13 | * Copyright (c) 2014 Lemur Consulting Ltd. 14 | *

15 | * Licensed under the Apache License, Version 2.0 (the "License"); 16 | * you may not use this file except in compliance with the License. 17 | * You may obtain a copy of the License at 18 | *

19 | * http://www.apache.org/licenses/LICENSE-2.0 20 | *

21 | * Unless required by applicable law or agreed to in writing, software 22 | * distributed under the License is distributed on an "AS IS" BASIS, 23 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 24 | * See the License for the specific language governing permissions and 25 | * limitations under the License. 26 | */ 27 | 28 | public class Interact { 29 | 30 | public static void main(String... args) throws IOException { 31 | 32 | Properties props = new Properties(); 33 | props.setProperty("metadata.broker.list", "localhost:9092"); 34 | props.setProperty("serializer.class", "kafka.serializer.StringEncoder"); 35 | ProducerConfig config = new ProducerConfig(props); 36 | 37 | Producer producer = new Producer<>(config); 38 | 39 | BufferedReader reader = new BufferedReader(new InputStreamReader(System.in)); 40 | 41 | while (true) { 42 | String cmd = reader.readLine(); 43 | if ("quit".equals(cmd)) 44 | return; 45 | if (cmd.startsWith("q ")) 46 | sendQueryUpdate(cmd, producer); 47 | if (cmd.startsWith("d ")) 48 | sendDocument(cmd, producer); 49 | } 50 | 51 | } 52 | 53 | private static void sendQueryUpdate(String cmd, Producer producer) { 54 | 55 | String[] parts = cmd.split("\\s", 3); 56 | String id = parts[1]; 57 | String query = parts[2]; 58 | 59 | KeyedMessage message 60 | = new KeyedMessage<>(MonitorTask.QUERIES_STREAM, id, "{ \"query\" : \"" + query + "\"}"); 61 | producer.send(message); 62 | 63 | } 64 | 65 | private static void sendDocument(String cmd, Producer producer) { 66 | 67 | String[] parts = cmd.split("\\s", 3); 68 | String id = parts[1]; 69 | String doc = parts[2]; 70 | 71 | for (int i = 0; i < MatchRecombinerTask.QUERY_PARTITIONS; i++) { 72 | KeyedMessage message = 73 | new KeyedMessage<>(MonitorTask.DOCS_STREAM, id, i, "{ \"f\" : \"" + doc + "\"}"); 74 | producer.send(message); 75 | } 76 | 77 | } 78 | 79 | } 80 | -------------------------------------------------------------------------------- /src/main/assembly/src.xml: -------------------------------------------------------------------------------- 1 | 2 | 12 | 13 | 17 | dist 18 | 19 | tar.gz 20 | 21 | false 22 | 23 | 24 | ${basedir}/src/main/resources/log4j.xml 25 | lib 26 | 27 | 29 | 30 | ${basedir}/src/main/config/luwak.properties 31 | config 32 | true 33 | 34 | 35 | ${basedir}/src/main/config/combiner.properties 36 | config 37 | true 38 | 39 | 40 | 41 | 42 | bin 43 | 44 | org.apache.samza:samza-shell:tgz:dist:* 45 | 46 | 0744 47 | true 48 | 49 | 50 | lib 51 | 52 | org.apache.samza:samza-core_2.10 53 | org.apache.samza:samza-kafka_2.10 54 | org.apache.samza:samza-serializers_2.10 55 | org.apache.samza:samza-yarn_2.10 56 | org.apache.samza:samza-kv-leveldb_2.10 57 | org.apache.samza:samza-log4j 58 | org.slf4j:slf4j-log4j12 59 | org.apache.kafka:kafka_2.10 60 | org.apache.hadoop:hadoop-hdfs 61 | uk.co.flax:samza-luwak 62 | uk.co.flax:luwak 63 | 64 | true 65 | 66 | 67 | 68 | -------------------------------------------------------------------------------- /src/main/java/uk/co/flax/samzaluwak/MatchRecombinerTask.java: -------------------------------------------------------------------------------- 1 | package uk.co.flax.samzaluwak; 2 | 3 | import java.util.HashMap; 4 | import java.util.List; 5 | import java.util.Map; 6 | 7 | import com.google.common.collect.Lists; 8 | import org.apache.samza.config.Config; 9 | import org.apache.samza.storage.kv.Entry; 10 | import org.apache.samza.storage.kv.KeyValueIterator; 11 | import org.apache.samza.storage.kv.KeyValueStore; 12 | import org.apache.samza.system.IncomingMessageEnvelope; 13 | import org.apache.samza.system.OutgoingMessageEnvelope; 14 | import org.apache.samza.system.SystemStream; 15 | import org.apache.samza.task.*; 16 | import org.slf4j.Logger; 17 | import org.slf4j.LoggerFactory; 18 | 19 | /** 20 | * Copyright (c) 2014 Lemur Consulting Ltd. 21 | *

22 | * Licensed under the Apache License, Version 2.0 (the "License"); 23 | * you may not use this file except in compliance with the License. 24 | * You may obtain a copy of the License at 25 | *

26 | * http://www.apache.org/licenses/LICENSE-2.0 27 | *

28 | * Unless required by applicable law or agreed to in writing, software 29 | * distributed under the License is distributed on an "AS IS" BASIS, 30 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 31 | * See the License for the specific language governing permissions and 32 | * limitations under the License. 33 | */ 34 | 35 | public class MatchRecombinerTask implements StreamTask, InitableTask { 36 | 37 | private static final Logger logger = LoggerFactory.getLogger(MatchRecombinerTask.class); 38 | 39 | public static final int QUERY_PARTITIONS = 2; 40 | 41 | private KeyValueStore> store; 42 | 43 | public static final SystemStream MATCHES_STREAM = new SystemStream("kafka", "combinedmatches"); 44 | 45 | @Override 46 | public void process(IncomingMessageEnvelope message, MessageCollector collector, TaskCoordinator taskCoordinator) throws Exception { 47 | String key = (String) message.getKey(); 48 | Map matches = (Map) message.getMessage(); 49 | store.put(key, matches); 50 | logger.info("Got partial match for {}", key); 51 | 52 | String originalKey = originalKey(key); 53 | 54 | Map> parts = collectMatches(originalKey); 55 | if (parts.size() != QUERY_PARTITIONS) 56 | return; 57 | 58 | logger.info("All partial matches for {} received", originalKey); 59 | List> combinedMatches = combineMatches(parts); 60 | 61 | collector.send(new OutgoingMessageEnvelope(MATCHES_STREAM, originalKey, combinedMatches)); 62 | for (String storekey : parts.keySet()) { 63 | store.delete(storekey); 64 | } 65 | 66 | } 67 | 68 | private List> combineMatches(Map> parts) { 69 | return Lists.newArrayList(parts.values()); 70 | } 71 | 72 | @Override 73 | public void init(Config config, TaskContext taskContext) throws Exception { 74 | this.store = (KeyValueStore>) taskContext.getStore("matches"); 75 | } 76 | 77 | private Map> collectMatches(String key) { 78 | Map> parts = new HashMap<>(); 79 | KeyValueIterator> it = store.range(key + "_", key + "_a"); 80 | try { 81 | while (it.hasNext()) { 82 | Entry> entry = it.next(); 83 | parts.put(entry.getKey(), entry.getValue()); 84 | } 85 | logger.info("Found {} partial matches in store for {}", parts.size(), key); 86 | return parts; 87 | } 88 | finally { 89 | it.close(); 90 | } 91 | 92 | } 93 | 94 | private String originalKey(String key) { 95 | return key.replaceAll("_.*?$", ""); 96 | } 97 | } 98 | -------------------------------------------------------------------------------- /src/main/java/uk/co/flax/samzaluwak/MonitorTask.java: -------------------------------------------------------------------------------- 1 | package uk.co.flax.samzaluwak; 2 | 3 | import java.io.IOException; 4 | import java.util.Map; 5 | 6 | import org.apache.lucene.analysis.Analyzer; 7 | import org.apache.lucene.analysis.standard.StandardAnalyzer; 8 | import org.apache.samza.config.Config; 9 | import org.apache.samza.system.IncomingMessageEnvelope; 10 | import org.apache.samza.system.OutgoingMessageEnvelope; 11 | import org.apache.samza.system.SystemStream; 12 | import org.apache.samza.task.*; 13 | import org.slf4j.Logger; 14 | import org.slf4j.LoggerFactory; 15 | import uk.co.flax.luwak.*; 16 | import uk.co.flax.luwak.matchers.SimpleMatcher; 17 | import uk.co.flax.luwak.presearcher.MatchAllPresearcher; 18 | import uk.co.flax.luwak.queryparsers.LuceneQueryParser; 19 | 20 | /** 21 | * Copyright (c) 2014 Lemur Consulting Ltd. 22 | *

23 | * Licensed under the Apache License, Version 2.0 (the "License"); 24 | * you may not use this file except in compliance with the License. 25 | * You may obtain a copy of the License at 26 | *

27 | * http://www.apache.org/licenses/LICENSE-2.0 28 | *

29 | * Unless required by applicable law or agreed to in writing, software 30 | * distributed under the License is distributed on an "AS IS" BASIS, 31 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 32 | * See the License for the specific language governing permissions and 33 | * limitations under the License. 34 | */ 35 | 36 | public class MonitorTask implements StreamTask, InitableTask { 37 | 38 | private static final Logger logger = LoggerFactory.getLogger(MonitorTask.class); 39 | 40 | public static final String QUERIES_STREAM = "queries"; 41 | public static final String DOCS_STREAM = "documents"; 42 | 43 | private Monitor monitor; 44 | private Analyzer analyzer = new StandardAnalyzer(); 45 | private MatcherFactory matcherFactory = SimpleMatcher.FACTORY; 46 | 47 | public static final SystemStream MATCHES_STREAM = new SystemStream("kafka", "matches1"); 48 | 49 | @Override 50 | public void process(IncomingMessageEnvelope message, MessageCollector collector, TaskCoordinator taskCoordinator) throws Exception { 51 | String stream = message.getSystemStreamPartition().getStream(); 52 | switch (stream) { 53 | case QUERIES_STREAM: 54 | logger.info("Adding new query for partition {}: {}", message.getSystemStreamPartition().getPartition().getPartitionId(), message.getMessage()); 55 | update((String) message.getKey(), (Map) message.getMessage()); 56 | break; 57 | case DOCS_STREAM: 58 | Matches matches = match((String) message.getKey(), (Map) message.getMessage()); 59 | String key = message.getKey() + "_" + message.getSystemStreamPartition().getPartition().getPartitionId(); 60 | collector.send(new OutgoingMessageEnvelope(MATCHES_STREAM, message.getKey(), key, matches)); 61 | break; 62 | default: 63 | throw new RuntimeException("Unknown stream: " + stream); 64 | } 65 | } 66 | 67 | @Override 68 | public void init(Config config, TaskContext taskContext) throws Exception { 69 | monitor = new Monitor(new LuceneQueryParser("f"), new MatchAllPresearcher()); 70 | } 71 | 72 | private void update(String id, Map message) throws IOException { 73 | MonitorQuery mq = new MonitorQuery(id, message.get("query"), message); 74 | logger.info("Adding new query: {}", mq); 75 | for (QueryError error : monitor.update(mq)) { 76 | logger.warn(error.toString()); 77 | } 78 | } 79 | 80 | private Matches match(String id, Map fields) throws IOException { 81 | 82 | InputDocument.Builder builder = InputDocument.builder(id); 83 | for (Map.Entry field : fields.entrySet()) { 84 | builder.addField(field.getKey(), field.getValue(), analyzer); 85 | } 86 | 87 | logger.info("Matching document {}", id); 88 | return monitor.match(builder.build(), matcherFactory); 89 | 90 | } 91 | 92 | } 93 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | Samza-Luwak Proof of Concept 2 | ============================ 3 | 4 | This project is an experimental setup for using [Luwak](https://github.com/flaxsearch/luwak) 5 | within [Samza](http://samza.incubator.apache.org/). 6 | 7 | Use case: imagine you want to implement something like 8 | [Google Alerts](https://www.google.com/alerts). Every time a new document is published, you want to 9 | run it through a list of search queries, and notify someone if the document matches one of the 10 | queries. (There's actually a [whole industry](http://en.wikipedia.org/wiki/Media_monitoring) that 11 | specialises in doing this.) A similar use case is Twitter search, when you want to see a stream of 12 | tweets matching a search query. 13 | 14 | We're exploring ways of making such streaming search queries scalable (scaling both to large 15 | throughput of documents and large numbers of queries), using the following tools: 16 | 17 | * [Luwak](https://github.com/flaxsearch/luwak) is a wrapper library around 18 | [Lucene](http://lucene.apache.org/). Lucene does the document indexing and querying, and Luwak 19 | adds some optimisations to improve performance when each document needs to be matched against 20 | a large number (hundreds of thousands) of queries. 21 | * [Samza](http://samza.incubator.apache.org/) is a distributed stream processing framework 22 | based on [Kafka](http://kafka.apache.org/) and 23 | [Hadoop YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html). 24 | It provides fault-tolerant message processing and a cluster deployment mechanism. 25 | 26 | What we're doing is a bit similar to 27 | [ElasticSearch's percolator](http://www.elasticsearch.org/guide/en/elasticsearch/reference/current/search-percolate.html), 28 | but we think it has the potential to perform better and be more scalable. 29 | 30 | 31 | High-level architecture 32 | ----------------------- 33 | 34 | There are two input streams and one output stream, which are implemented as Kafka topics: 35 | 36 | * **Queries** (input). When a user wants to start being alerted about documents matching some 37 | search query, they send a message to the queries stream. A unique ID is used as the key of the 38 | message. When the user wants to modify a query, or stop alerts for a particular query, they can 39 | update it by sending another message with the same key. 40 | * **Documents** (input). Every document is published to a stream, and Luwak matches each document 41 | against the current set of stored queries. 42 | * **Matches** (output). For each document that comes in, the job produces an output message that 43 | lists the IDs of the queries that matched the document. This can then be used by downstream 44 | consumers to alert the owners of those queries. 45 | 46 | Note that there is currently no persistent index of documents. This is pure stream search: search 47 | results include only documents that are published after the query is registered, but not historical 48 | documents. 49 | 50 | 51 | Partitioning 52 | ------------ 53 | 54 | How do we make this fast and scalable? Evaluating a single search query is fast, but if you're 55 | matching each document against hundreds of thousands of queries (some of which can be very complex), 56 | it can take a significant time to process every document. We are investigating several approaches: 57 | 58 | * **Co-partitioning queries and documents**. In some cases, you know in advance that some queries 59 | can only apply to certain documents, for example English-language queries are only applied to 60 | English-language documents. This creates a natural partitioning scheme, but requires that you 61 | define the partitioning manually. 62 | * **Pre-searching**. If a query requires an unusual word to appear in the document, then that 63 | query only needs to be attempted on documents containing that word. Luwak already does this to 64 | some degree, and perhaps we can take it further. 65 | * **Partitioning the query set**. We can have several partitions of the stream processor, and store 66 | each query on one of the partitions. Then each partition has a smaller number of queries to try, 67 | so each document can be matched faster. However, each document must be sent to all partitions of 68 | the stream processor, unless we have some prior knowledge about which queries may apply (using 69 | pre-searching). 70 | * **Partitioning the document stream**. We can keep the entire list of queries on each stream 71 | processing node, and parallelise the processing by sending each document to only one of the 72 | nodes. This does not make the processing of any individual document any faster, but allows the 73 | process to scale to a higher throughput of documents. 74 | 75 | At the moment, we are exploring partitioning of the query set. This requires a multi-stage pipeline: 76 | 77 | 1. Documents need to be published to *all* partitions of the documents stream. 78 | Queries need to be partitioned by query ID (and only published to one partition). 79 | 2. The matching job matches each documents against the queries within its partition, and emits a 80 | message to the "matches" stream, indicating which query IDs matched that document ID. 81 | 3. A second job consumes the "matches" stream, and waits for the matches from all the partitions 82 | to arrive. When all partitions have processed a particular document, this job emits a message 83 | to the "combined-matches" stream, which now includes the matching query IDs from all 84 | partitions. 85 | 86 | 87 | Building 88 | -------- 89 | 90 | This project is very hacky and experimental, and may not work at all. It's also a bit convoluted 91 | to build right now, because it depends on various unreleased components. 92 | 93 | Check out [Samza](https://github.com/apache/incubator-samza) (master branch), build it and install 94 | it to your local Maven repository: 95 | 96 | ```bash 97 | git clone https://github.com/apache/incubator-samza.git samza 98 | cd samza 99 | ./gradlew -PscalaVersion=2.10 clean publishToMavenLocal 100 | ``` 101 | 102 | Check out [hello-samza](http://samza.incubator.apache.org/startup/hello-samza/latest/index.html) 103 | (latest branch), and use it to launch Zookeeper, Kafka and YARN locally: 104 | 105 | ```bash 106 | git clone https://github.com/apache/incubator-samza-hello-samza.git hello-samza 107 | cd hello-samza 108 | git checkout latest 109 | bin/grid bootstrap 110 | bin/grid start all 111 | for topic in documents queries matches1 combinedmatches matches-combiner-changelog; do 112 | deploy/kafka/bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic $topic --partitions 2 --replication-factor 1 113 | done 114 | ``` 115 | 116 | Check out [Flax](http://www.flax.co.uk/)'s 117 | [fork of Lucene](https://github.com/flaxsearch/lucene-solr-intervals) (positions-5x branch), 118 | which has a new feature that Luwak needs, but is not yet upstream 119 | (it should be committed upstream eventually): 120 | 121 | ```bash 122 | git clone https://github.com/flaxsearch/lucene-solr-intervals.git 123 | cd lucene-solr-internals 124 | git checkout positions-5x 125 | mvn -DskipTests install 126 | ``` 127 | 128 | Check out [Luwak](https://github.com/flaxsearch/luwak) (1.1.x branch), build it and install it 129 | to your local Maven repository (note this currently doesn't work with JDK8): 130 | 131 | ```bash 132 | git clone https://github.com/flaxsearch/luwak.git 133 | cd luwak 134 | git checkout 1.1.x 135 | mvn install 136 | ``` 137 | 138 | Build and run this project: 139 | 140 | ```bash 141 | git clone https://github.com/romseygeek/samza-luwak.git 142 | cd samza-luwak 143 | mvn clean package 144 | mkdir deploy 145 | tar -xzvf target/samza-luwak-1.0-SNAPSHOT-dist.tar.gz -C deploy/ 146 | deploy/bin/run-job.sh --config-path=file://$PWD/deploy/config/luwak.properties 147 | deploy/bin/run-job.sh --config-path=file://$PWD/deploy/config/combiner.properties 148 | ``` 149 | 150 | Now you can try adding some test documents and queries to the system, and observe the output: 151 | 152 | ```bash 153 | hello-samza/deploy/kafka/bin/kafka-console-consumer.sh --zookeeper localhost:2181 --topic combinedmatches --from-beginning & 154 | java -jar samza-luwak/target/samza-luwak-1.0-SNAPSHOT.jar 155 | q query1 foo AND bar 156 | q query2 bar AND baz 157 | d doc1 this document contains the words foo and bar only 158 | d doc2 this document, on the other hand, mentions bar and baz. 159 | d doc3 this one goes nuts and mentions foo, bar and baz -- all three! 160 | d doc4 finally, this one mentions none of those words. 161 | quit 162 | ``` 163 | 164 | In that command-line tool, queries are defined with "q", followed by the query ID, followed by a 165 | Lucene query string. Documents are defined with "d", followed by the document ID, followed by the 166 | text of the document. 167 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | uk.co.flax 8 | samza-luwak 9 | 1.0-SNAPSHOT 10 | 11 | 12 | 13 | org.apache.samza 14 | samza-api 15 | ${samza.version} 16 | 17 | 18 | org.apache.samza 19 | samza-core_2.10 20 | ${samza.version} 21 | 22 | 23 | org.apache.samza 24 | samza-log4j 25 | ${samza.version} 26 | 27 | 28 | org.apache.samza 29 | samza-serializers_2.10 30 | ${samza.version} 31 | 32 | 33 | org.apache.samza 34 | samza-shell 35 | dist 36 | tgz 37 | ${samza.version} 38 | 39 | 40 | org.apache.samza 41 | samza-yarn_2.10 42 | ${samza.version} 43 | 44 | 45 | org.apache.samza 46 | samza-kv_2.10 47 | ${samza.version} 48 | 49 | 50 | org.apache.samza 51 | samza-kv-leveldb_2.10 52 | ${samza.version} 53 | 54 | 55 | org.apache.samza 56 | samza-kafka_2.10 57 | ${samza.version} 58 | 59 | 60 | org.apache.kafka 61 | kafka_2.10 62 | 0.8.1 63 | 64 | 65 | org.schwering 66 | irclib 67 | 1.10 68 | 69 | 70 | org.slf4j 71 | slf4j-api 72 | 1.6.2 73 | 74 | 75 | org.slf4j 76 | slf4j-log4j12 77 | 1.6.2 78 | 79 | 80 | org.codehaus.jackson 81 | jackson-jaxrs 82 | 1.8.5 83 | 84 | 85 | org.apache.hadoop 86 | hadoop-hdfs 87 | 2.4.0 88 | 89 | 90 | uk.co.flax 91 | luwak 92 | 1.1.0-rc4-SNAPSHOT 93 | 94 | 95 | junit 96 | junit 97 | 4.11 98 | test 99 | 100 | 101 | org.mockito 102 | mockito-core 103 | 1.9.5 104 | test 105 | 106 | 107 | 108 | 109 | 110 | UTF-8 111 | 0.8.0-SNAPSHOT 112 | 113 | 114 | 115 | 116 | my-local-repo 117 | file://${user.home}/.m2/repository 118 | 119 | 120 | apache-releases 121 | https://repository.apache.org/content/groups/public 122 | 123 | 124 | scala-tools.org 125 | Scala-tools Maven2 Repository 126 | https://oss.sonatype.org/content/groups/scala-tools 127 | 128 | 129 | 130 | 131 | 132 | scala-tools.org 133 | Scala-tools Maven2 Repository 134 | http://scala-tools.org/repo-releases 135 | 136 | 137 | 138 | 139 | 140 | 141 | 142 | org.apache.maven.plugins 143 | maven-compiler-plugin 144 | 3.1 145 | 146 | 1.7 147 | 1.7 148 | 149 | 150 | 151 | 152 | 153 | maven-assembly-plugin 154 | 2.3 155 | 156 | 157 | src/main/assembly/src.xml 158 | 159 | 160 | 161 | 162 | make-assembly 163 | package 164 | 165 | single 166 | 167 | 168 | 169 | 170 | 171 | 172 | org.apache.maven.plugins 173 | maven-shade-plugin 174 | 1.6 175 | 176 | true 177 | 178 | 179 | *:* 180 | 181 | META-INF/*.SF 182 | META-INF/*.DSA 183 | META-INF/*.RSA 184 | 185 | 186 | 187 | 188 | 189 | org.apache.samza:samza-shell 190 | 191 | 192 | 193 | 194 | 195 | package 196 | 197 | shade 198 | 199 | 200 | 201 | 202 | 203 | uk.co.flax.samzaluwak.Interact 204 | 205 | 206 | 207 | 208 | 209 | 210 | 211 | 212 | 213 | --------------------------------------------------------------------------------