mirror of https://github.com/apache/druid.git
remove FirehoseV2 and realtime node extensions (#8020)
* remove firehosev2 and realtime node extensions * revert intellij stuff * rat exclusion
This commit is contained in:
parent
613f09b45a
commit
42a7b8849a
|
@ -654,40 +654,6 @@ BINARY/EXTENSIONS/druid-datasketches
|
|||
* com.yahoo.datasketches:memory
|
||||
|
||||
|
||||
BINARY/EXTENSIONS/druid-examples
|
||||
|
||||
This product bundles IRC API version 1.0-0014.
|
||||
* com.ircclouds.irc:irc-api
|
||||
|
||||
This product bundles MaxMind GeoIP2 API version 0.4.0.
|
||||
* com.maxmind.geoip2:geoip2
|
||||
|
||||
This product bundles the following Apache Commons libraries:
|
||||
* commons-beanutils 1.8.3
|
||||
* commons-validator 1.4.0
|
||||
|
||||
This product bundles Twitter4J version 3.0.3.
|
||||
* org.twitter4j:twitter4j-async
|
||||
* org.twitter4j:twitter4j-core
|
||||
* org.twitter4j:twitter4j-stream
|
||||
|
||||
|
||||
BINARY/EXTENSIONS/druid-kafka-eight
|
||||
|
||||
This product bundles Apache Kafka version 0.8.2.1.
|
||||
* org.apache.kafka:kafka_2.10
|
||||
* org.apache.kafka:kafka-clients
|
||||
|
||||
This product bundles ZkClient version 0.3.
|
||||
* com.101tec:zkclient
|
||||
|
||||
This product bundles Yammer Metrics version 2.2.0.
|
||||
* com.yammer.metrics:metrics-core
|
||||
|
||||
This product bundles snappy-java version 1.1.1.6.
|
||||
* org.xerial.snappy:snappy-java
|
||||
|
||||
|
||||
BINARY/EXTENSIONS/druid-kafka-indexing-service
|
||||
This product bundles Apache Kafka version 0.10.2.2.
|
||||
* org.apache.kafka:kafka-clients
|
||||
|
@ -785,12 +751,6 @@ BINARY/HADOOP-CLIENT
|
|||
* org.slf4j:slf4j-log4j12
|
||||
|
||||
|
||||
BINARY/EXTENSIONS/druid-kafka-eight
|
||||
This product bundles JOpt Simple version 3.2., copyright Paul R. Holser, Jr.,
|
||||
which is available under an MIT license. For details, see licenses/bin/jopt-simple.MIT.
|
||||
* net.sf.jopt-simple:jopt-simple
|
||||
|
||||
|
||||
BINARY/WEB-CONSOLE
|
||||
The following dependency names are NPM package names (https://www.npmjs.com).
|
||||
|
||||
|
@ -1052,13 +1012,6 @@ BINARY/EXTENSIONS/druid-kerberos
|
|||
which is available under a BSD-3-Clause License. For details, see licenses/bin/jsch.BSD3.
|
||||
* com.jcraft:jsch
|
||||
|
||||
|
||||
BINARY/EXTENSIONS/druid-kafka-eight
|
||||
This product bundles Scala Library version 2.10.4, copyright EPFL, Lightbend Inc.,
|
||||
which is available under a BSD-3-Clause License. For details, see licenses/bin/scala-lang.BSD3.
|
||||
* org.scala-lang:scala-library
|
||||
|
||||
|
||||
BINARY/EXTENSIONS/druid-lookups-cached-single
|
||||
This product bundles StringTemplate version 3.2, copyright Terrence Parr,
|
||||
which is available under a BSD-3-Clause License. For details, see licenses/bin/antlr-stringtemplate.BSD3.
|
||||
|
|
|
@ -1719,44 +1719,6 @@ http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html
|
|||
|
||||
|
||||
|
||||
############ BINARY/EXTENSIONS/druid-kafka-eight ############
|
||||
|
||||
================= metrics-core-2.2.0.jar =================
|
||||
Metrics
|
||||
Copyright 2010-2012 Coda Hale and Yammer, Inc.
|
||||
|
||||
This product includes software developed by Coda Hale and Yammer, Inc.
|
||||
|
||||
This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
|
||||
with the following comments:
|
||||
|
||||
Written by Doug Lea with assistance from members of JCP JSR-166
|
||||
Expert Group and released to the public domain, as explained at
|
||||
http://creativecommons.org/publicdomain/zero/1.0/
|
||||
|
||||
|
||||
|
||||
|
||||
================= snappy-1.1.1.6.jar =================
|
||||
This product includes software developed by Google
|
||||
Snappy: http://code.google.com/p/snappy/ (New BSD License)
|
||||
|
||||
This product includes software developed by Apache
|
||||
PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/
|
||||
(Apache 2.0 license)
|
||||
|
||||
This library containd statically linked libstdc++. This inclusion is allowed by
|
||||
"GCC RUntime Library Exception"
|
||||
http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html
|
||||
|
||||
== Contributors ==
|
||||
* Tatu Saloranta
|
||||
* Providing benchmark suite
|
||||
* Alec Wysoker
|
||||
* Performance and memory usage improvement
|
||||
|
||||
|
||||
|
||||
|
||||
############ BINARY/EXTENSIONS/druid-kafka-indexing-service ############
|
||||
|
||||
|
|
|
@ -1,51 +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.druid.data.input;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
import org.apache.druid.guice.annotations.ExtensionPoint;
|
||||
import org.apache.druid.java.util.common.parsers.ParseException;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Initialization method that connects up the FirehoseV2. If this method returns successfully it should be safe to
|
||||
* call start() on the returned FirehoseV2 (which might subsequently block).
|
||||
*
|
||||
* In contrast to V1 version, FirehoseFactoryV2 is able to pass an additional json-serialized object to FirehoseV2,
|
||||
* which contains last commit metadata
|
||||
*
|
||||
* <p/>
|
||||
* If this method returns null, then any attempt to call start(), advance(), currRow(), makeCommitter() and close() on the return
|
||||
* value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on
|
||||
* invalid configuration is preferred over returning null.
|
||||
*/
|
||||
@ExtensionPoint
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
public interface FirehoseFactoryV2<T extends InputRowParser>
|
||||
{
|
||||
/**
|
||||
* This method is declared to throw {@link IOException}, although it's not thrown in the implementations in Druid
|
||||
* code, for compatibility with third-party extensions.
|
||||
*/
|
||||
@SuppressWarnings("RedundantThrows")
|
||||
FirehoseV2 connect(T parser, Object lastCommit) throws IOException, ParseException;
|
||||
}
|
|
@ -1,91 +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.druid.data.input;
|
||||
|
||||
import org.apache.druid.guice.annotations.ExtensionPoint;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
/**
|
||||
* This is an interface that holds onto the stream of incoming data. Realtime data ingestion is built around this
|
||||
* abstraction. In order to add a new type of source for realtime data ingestion, all you need to do is implement
|
||||
* one of these and register it with the Main.
|
||||
*
|
||||
* In contrast to Firehose v1 version, FirehoseV2 will always operate in a "peek, then advance" manner.
|
||||
* And the intended usage patttern is
|
||||
* 1. Call start()
|
||||
* 2. Read currRow()
|
||||
* 3. Call advance()
|
||||
* 4. If index should be committed: commit()
|
||||
* 5. GOTO 2
|
||||
*
|
||||
* Note that commit() is being called *after* advance.
|
||||
*
|
||||
* This object acts a lot like an Iterator, but it doesn't extend the Iterator interface because it extends
|
||||
* Closeable and it is very important that the close() method doesn't get forgotten, which is easy to do if this
|
||||
* gets passed around as an Iterator.
|
||||
*
|
||||
* The implementation of this interface only needs to be minimally thread-safe. The methods {@link #start()}, {@link
|
||||
* #advance()}, {@link #currRow()} and {@link #makeCommitter()} are all called from the same thread. {@link
|
||||
* #makeCommitter()}, however, returns a callback which will be called on another thread, so the operations inside of
|
||||
* that callback must be thread-safe.
|
||||
*/
|
||||
@ExtensionPoint
|
||||
public interface FirehoseV2 extends Closeable
|
||||
{
|
||||
/**
|
||||
* For initial start
|
||||
*/
|
||||
void start();
|
||||
|
||||
/**
|
||||
* Advance the firehose to the next offset. Implementations of this interface should make sure that
|
||||
* if advance() is called and throws out an exception, the next call to currRow() should return a
|
||||
* null value.
|
||||
*
|
||||
* @return true if and when there is another row available, false if the stream has dried up
|
||||
*/
|
||||
boolean advance();
|
||||
|
||||
/**
|
||||
* @return The current row
|
||||
*/
|
||||
InputRow currRow();
|
||||
|
||||
/**
|
||||
* Returns a Committer that will "commit" everything read up to the point at which makeCommitter() is called.
|
||||
*
|
||||
* This method is called when the main processing loop starts to persist its current batch of things to process.
|
||||
* The returned committer will be run when the current batch has been successfully persisted
|
||||
* and the metadata the committer carries can also be persisted along with segment data. There is usually
|
||||
* some time lag between when this method is called and when the runnable is run. The Runnable is also run on
|
||||
* a separate thread so its operation should be thread-safe.
|
||||
*
|
||||
* Note that "correct" usage of this interface will always call advance() before commit() if the current row
|
||||
* is considered in the commit.
|
||||
*
|
||||
* The Runnable is essentially just a lambda/closure that is run() after data supplied by this instance has
|
||||
* been committed on the writer side of this interface protocol.
|
||||
*
|
||||
* A simple implementation of this interface might do nothing when run() is called,
|
||||
* and save proper commit information in metadata
|
||||
*/
|
||||
Committer makeCommitter();
|
||||
}
|
|
@ -168,8 +168,6 @@
|
|||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-histogram</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-kafka-eight</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-kafka-extraction-namespace</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-kafka-indexing-service</argument>
|
||||
|
@ -200,8 +198,6 @@
|
|||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-stats</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-examples</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:simple-client-sslcontext</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions:druid-basic-security</argument>
|
||||
|
@ -318,16 +314,12 @@
|
|||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:druid-distinctcount</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:druid-rocketmq</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:graphite-emitter</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:druid-influx-extensions</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:druid-influxdb-emitter</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:druid-kafka-eight-simple-consumer</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:kafka-emitter</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:materialized-view-maintenance</argument>
|
||||
|
@ -336,8 +328,6 @@
|
|||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:druid-opentsdb-emitter</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:druid-rabbitmq</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:druid-redis-cache</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:sqlserver-metadata-storage</argument>
|
||||
|
|
|
@ -166,9 +166,13 @@
|
|||
{"source": "development/community-extensions/rabbitmq.html", "target": "../extensions-contrib/rabbitmq.html"},
|
||||
{"source": "development/extensions-core/namespaced-lookup.html", "target": "lookups-cached-global.html"},
|
||||
{"source": "operations/performance-faq.html", "target": "../operations/basic-cluster-tuning.html"},
|
||||
{"source": "development/extensions-contrib/orc.html", "target": "../extensions-core/orc.html"}
|
||||
{"source": "development/extensions-contrib/orc.html", "target": "../extensions-core/orc.html"},
|
||||
{"source": "operations/performance-faq.html", "target": "../operations/basic-cluster-tuning.html"},
|
||||
{"source": "configuration/realtime.md", "target": "../ingestion/standalone-realtime.html"},
|
||||
{"source": "design/realtime.md", "target": "../ingestion/standalone-realtime.html"},
|
||||
{"source": "ingestion/stream-pull.md", "target": "../ingestion/standalone-realtime.html"}
|
||||
{"source": "ingestion/stream-pull.md", "target": "../ingestion/standalone-realtime.html"},
|
||||
{"source": "development/extensions-core/kafka-eight-firehose.md", "target": "../../ingestion/standalone-realtime.html"},
|
||||
{"source": "development/extensions-contrib/kafka-simple.md", "target": "../../ingestion/standalone-realtime.html"},
|
||||
{"source": "development/extensions-contrib/rabbitmq.md", "target": "../../ingestion/standalone-realtime.html"},
|
||||
{"source": "development/extensions-contrib/rocketmq.md", "target": "../../ingestion/standalone-realtime.html"},
|
||||
]
|
||||
|
|
|
@ -1,56 +0,0 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "Kafka Simple Consumer"
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
# Kafka Simple Consumer
|
||||
|
||||
To use this Apache Druid (incubating) extension, make sure to [include](../../operations/including-extensions.html) `druid-kafka-eight-simpleConsumer` extension.
|
||||
|
||||
## Firehose
|
||||
|
||||
This is an experimental firehose to ingest data from Apache Kafka using the Kafka simple consumer api. Currently, this firehose would only work inside standalone realtime processes.
|
||||
The configuration for KafkaSimpleConsumerFirehose is similar to the Kafka Eight Firehose , except `firehose` should be replaced with `firehoseV2` like this:
|
||||
|
||||
```json
|
||||
"firehoseV2": {
|
||||
"type" : "kafka-0.8-v2",
|
||||
"brokerList" : ["localhost:4443"],
|
||||
"queueBufferLength":10001,
|
||||
"resetOffsetToEarliest":"true",
|
||||
"partitionIdList" : ["0"],
|
||||
"clientId" : "localclient",
|
||||
"feed": "wikipedia"
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|kafka-0.8-v2|yes|
|
||||
|brokerList|list of the kafka brokers|yes|
|
||||
|queueBufferLength|the buffer length for kafka message queue|no default(20000)|
|
||||
|resetOffsetToEarliest|in case of kafkaOffsetOutOfRange error happens, consumer should starts from the earliest or latest message available|true|
|
||||
|partitionIdList|list of kafka partition ids|yes|
|
||||
|clientId|the clientId for kafka SimpleConsumer|yes|
|
||||
|feed|kafka topic|yes|
|
||||
|
||||
For using this firehose at scale and possibly in production, it is recommended to set replication factor to at least three, which means at least three Kafka brokers in the `brokerList`. For a 1*10^4 events per second kafka topic, keeping one partition can work properly, but more partitions could be added if higher throughput is required.
|
|
@ -1,81 +0,0 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "RabbitMQ"
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
# RabbitMQ
|
||||
|
||||
To use this Apache Druid (incubating) extension, make sure to [include](../../operations/including-extensions.html) `druid-rabbitmq` extension.
|
||||
|
||||
## Firehose
|
||||
|
||||
#### RabbitMQFirehose
|
||||
|
||||
This firehose ingests events from a define rabbit-mq queue.
|
||||
|
||||
**Note:** Add **amqp-client-3.2.1.jar** to lib directory of druid to use this firehose.
|
||||
|
||||
A sample spec for rabbitmq firehose:
|
||||
|
||||
```json
|
||||
"firehose" : {
|
||||
"type" : "rabbitmq",
|
||||
"connection" : {
|
||||
"host": "localhost",
|
||||
"port": "5672",
|
||||
"username": "test-dude",
|
||||
"password": "test-word",
|
||||
"virtualHost": "test-vhost",
|
||||
"uri": "amqp://mqserver:1234/vhost"
|
||||
},
|
||||
"config" : {
|
||||
"exchange": "test-exchange",
|
||||
"queue" : "druidtest",
|
||||
"routingKey": "#",
|
||||
"durable": "true",
|
||||
"exclusive": "false",
|
||||
"autoDelete": "false",
|
||||
"maxRetries": "10",
|
||||
"retryIntervalSeconds": "1",
|
||||
"maxDurationSeconds": "300"
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|default|required?|
|
||||
|--------|-----------|-------|---------|
|
||||
|type|This should be "rabbitmq"|N/A|yes|
|
||||
|host|The hostname of the RabbitMQ broker to connect to|localhost|no|
|
||||
|port|The port number to connect to on the RabbitMQ broker|5672|no|
|
||||
|username|The username to use to connect to RabbitMQ|guest|no|
|
||||
|password|The password to use to connect to RabbitMQ|guest|no|
|
||||
|virtualHost|The virtual host to connect to|/|no|
|
||||
|uri|The URI string to use to connect to RabbitMQ| |no|
|
||||
|exchange|The exchange to connect to| |yes|
|
||||
|queue|The queue to connect to or create| |yes|
|
||||
|routingKey|The routing key to use to bind the queue to the exchange| |yes|
|
||||
|durable|Whether the queue should be durable|false|no|
|
||||
|exclusive|Whether the queue should be exclusive|false|no|
|
||||
|autoDelete|Whether the queue should auto-delete on disconnect|false|no|
|
||||
|maxRetries|The max number of reconnection retry attempts| |yes|
|
||||
|retryIntervalSeconds|The reconnection interval| |yes|
|
||||
|maxDurationSeconds|The max duration of trying to reconnect| |yes|
|
|
@ -1,29 +0,0 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "RocketMQ"
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
# RocketMQ
|
||||
|
||||
To use this Apache Druid (incubating) extension, make sure to [include](../../operations/including-extensions.html) `druid-rocketmq` extension.
|
||||
|
||||
Original author: [https://github.com/lizhanhui](https://github.com/lizhanhui).
|
|
@ -1,54 +0,0 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "Apache Kafka Eight Firehose"
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
# Kafka Eight Firehose
|
||||
|
||||
To use this Apache Druid (incubating) extension, make sure to [include](../../operations/including-extensions.html) `druid-kafka-eight` as an extension.
|
||||
|
||||
This firehose acts as a Kafka 0.8.x consumer and ingests data from Kafka.
|
||||
|
||||
Sample spec:
|
||||
|
||||
```json
|
||||
"firehose": {
|
||||
"type": "kafka-0.8",
|
||||
"consumerProps": {
|
||||
"zookeeper.connect": "localhost:2181",
|
||||
"zookeeper.connection.timeout.ms" : "15000",
|
||||
"zookeeper.session.timeout.ms" : "15000",
|
||||
"zookeeper.sync.time.ms" : "5000",
|
||||
"group.id": "druid-example",
|
||||
"fetch.message.max.bytes" : "1048586",
|
||||
"auto.offset.reset": "largest",
|
||||
"auto.commit.enable": "false"
|
||||
},
|
||||
"feed": "wikipedia"
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|This should be "kafka-0.8"|yes|
|
||||
|consumerProps|The full list of consumer configs can be [here](https://kafka.apache.org/08/configuration.html).|yes|
|
||||
|feed|Kafka maintains feeds of messages in categories called topics. This is the topic name.|yes|
|
|
@ -48,7 +48,6 @@ Core extensions are maintained by Druid committers.
|
|||
|druid-datasketches|Support for approximate counts and set operations with [DataSketches](https://datasketches.github.io/).|[link](../development/extensions-core/datasketches-extension.html)|
|
||||
|druid-hdfs-storage|HDFS deep storage.|[link](../development/extensions-core/hdfs.html)|
|
||||
|druid-histogram|Approximate histograms and quantiles aggregator. Deprecated, please use the [DataSketches quantiles aggregator](../development/extensions-core/datasketches-quantiles.html) from the `druid-datasketches` extension instead.|[link](../development/extensions-core/approximate-histograms.html)|
|
||||
|druid-kafka-eight|Kafka ingest firehose (high level consumer) for realtime nodes(deprecated).|[link](../development/extensions-core/kafka-eight-firehose.html)|
|
||||
|druid-kafka-extraction-namespace|Kafka-based namespaced lookup. Requires namespace lookup extension.|[link](../development/extensions-core/kafka-extraction-namespace.html)|
|
||||
|druid-kafka-indexing-service|Supervised exactly-once Kafka ingestion for the indexing service.|[link](../development/extensions-core/kafka-ingestion.html)|
|
||||
|druid-kinesis-indexing-service|Supervised exactly-once Kinesis ingestion for the indexing service.|[link](../development/extensions-core/kinesis-ingestion.html)|
|
||||
|
@ -83,10 +82,7 @@ All of these community extensions can be downloaded using [pull-deps](../operati
|
|||
|druid-cassandra-storage|Apache Cassandra deep storage.|[link](../development/extensions-contrib/cassandra.html)|
|
||||
|druid-cloudfiles-extensions|Rackspace Cloudfiles deep storage and firehose.|[link](../development/extensions-contrib/cloudfiles.html)|
|
||||
|druid-distinctcount|DistinctCount aggregator|[link](../development/extensions-contrib/distinctcount.html)|
|
||||
|druid-kafka-eight-simpleConsumer|Kafka ingest firehose (low level consumer)(deprecated).|[link](../development/extensions-contrib/kafka-simple.html)|
|
||||
|druid-rabbitmq|RabbitMQ firehose.|[link](../development/extensions-contrib/rabbitmq.html)|
|
||||
|druid-redis-cache|A cache implementation for Druid based on Redis.|[link](../development/extensions-contrib/redis-cache.html)|
|
||||
|druid-rocketmq|RocketMQ firehose.|[link](../development/extensions-contrib/rocketmq.html)|
|
||||
|druid-time-min-max|Min/Max aggregator for timestamp.|[link](../development/extensions-contrib/time-min-max.html)|
|
||||
|druid-google-extensions|Google Cloud Storage deep storage.|[link](../development/extensions-contrib/google.html)|
|
||||
|sqlserver-metadata-storage|Microsoft SqlServer deep storage.|[link](../development/extensions-contrib/sqlserver.html)|
|
||||
|
|
|
@ -39,5 +39,8 @@ removed completely in Druid 0.16.0. Operationally, realtime nodes were difficult
|
|||
each node required an unique configuration. The design of the stream pull ingestion system for realtime nodes also
|
||||
suffered from limitations which made it not possible to achieve exactly once ingestion.
|
||||
|
||||
The extensions `druid-kafka-eight`, `druid-kafka-eight-simpleConsumer`, `druid-rabbitmq`, and `druid-rocketmq` were also
|
||||
removed at this time, since they were built to operate on the realtime nodes.
|
||||
|
||||
Please consider using the [Kafka Indexing Service](../development/extensions-core/kafka-ingestion.html) or
|
||||
[Kinesis Indexing Service](../development/extensions-core/kinesis-ingestion.md) for stream pull ingestion instead.
|
||||
|
|
|
@ -92,10 +92,10 @@ To run `pull-deps`, you should
|
|||
|
||||
Example:
|
||||
|
||||
Suppose you want to download ```druid-rabbitmq```, ```mysql-metadata-storage``` and ```hadoop-client```(both 2.3.0 and 2.4.0) with a specific version, you can run `pull-deps` command with `-c org.apache.druid.extensions:druid-examples:#{DRUIDVERSION}`, `-c org.apache.druid.extensions:mysql-metadata-storage:#{DRUIDVERSION}`, `-h org.apache.hadoop:hadoop-client:2.3.0` and `-h org.apache.hadoop:hadoop-client:2.4.0`, an example command would be:
|
||||
Suppose you want to download ```mysql-metadata-storage``` and ```hadoop-client```(both 2.3.0 and 2.4.0) with a specific version, you can run `pull-deps` command with `-c org.apache.druid.extensions:mysql-metadata-storage:#{DRUIDVERSION}`, `-h org.apache.hadoop:hadoop-client:2.3.0` and `-h org.apache.hadoop:hadoop-client:2.4.0`, an example command would be:
|
||||
|
||||
```
|
||||
java -classpath "/my/druid/lib/*" org.apache.druid.cli.Main tools pull-deps --clean -c org.apache.druid.extensions:mysql-metadata-storage:#{DRUIDVERSION} -c org.apache.druid.extensions.contrib:druid-rabbitmq:#{DRUIDVERSION} -h org.apache.hadoop:hadoop-client:2.3.0 -h org.apache.hadoop:hadoop-client:2.4.0
|
||||
java -classpath "/my/druid/lib/*" org.apache.druid.cli.Main tools pull-deps --clean -c org.apache.druid.extensions:mysql-metadata-storage:#{DRUIDVERSION} -h org.apache.hadoop:hadoop-client:2.3.0 -h org.apache.hadoop:hadoop-client:2.4.0
|
||||
```
|
||||
|
||||
Because `--clean` is supplied, this command will first remove the directories specified at `druid.extensions.directory` and `druid.extensions.hadoopDependenciesDir`, then recreate them and start downloading the extensions there. After finishing downloading, if you go to the extension directories you specified, you will see
|
||||
|
@ -103,15 +103,6 @@ Because `--clean` is supplied, this command will first remove the directories sp
|
|||
```
|
||||
tree extensions
|
||||
extensions
|
||||
├── druid-examples
|
||||
│ ├── commons-beanutils-1.8.3.jar
|
||||
│ ├── commons-digester-1.8.jar
|
||||
│ ├── commons-logging-1.1.1.jar
|
||||
│ ├── commons-validator-1.4.0.jar
|
||||
│ ├── druid-examples-#{DRUIDVERSION}.jar
|
||||
│ ├── twitter4j-async-3.0.3.jar
|
||||
│ ├── twitter4j-core-3.0.3.jar
|
||||
│ └── twitter4j-stream-3.0.3.jar
|
||||
└── mysql-metadata-storage
|
||||
└── mysql-metadata-storage-#{DRUIDVERSION}.jar
|
||||
```
|
||||
|
@ -138,10 +129,10 @@ hadoop-dependencies/
|
|||
..... lots of jars
|
||||
```
|
||||
|
||||
Note that if you specify `--defaultVersion`, you don't have to put version information in the coordinate. For example, if you want both `druid-rabbitmq` and `mysql-metadata-storage` to use version `#{DRUIDVERSION}`, you can change the command above to
|
||||
Note that if you specify `--defaultVersion`, you don't have to put version information in the coordinate. For example, if you want `mysql-metadata-storage` to use version `#{DRUIDVERSION}`, you can change the command above to
|
||||
|
||||
```
|
||||
java -classpath "/my/druid/lib/*" org.apache.druid.cli.Main tools pull-deps --defaultVersion #{DRUIDVERSION} --clean -c org.apache.druid.extensions:mysql-metadata-storage -c org.apache.druid.extensions.contrib:druid-rabbitmq -h org.apache.hadoop:hadoop-client:2.3.0 -h org.apache.hadoop:hadoop-client:2.4.0
|
||||
java -classpath "/my/druid/lib/*" org.apache.druid.cli.Main tools pull-deps --defaultVersion #{DRUIDVERSION} --clean -c org.apache.druid.extensions:mysql-metadata-storage -h org.apache.hadoop:hadoop-client:2.3.0 -h org.apache.hadoop:hadoop-client:2.4.0
|
||||
```
|
||||
|
||||
<div class="note info">
|
||||
|
|
134
examples/pom.xml
134
examples/pom.xml
|
@ -1,134 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ 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/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-examples</artifactId>
|
||||
<name>druid-examples</name>
|
||||
<description>druid-examples</description>
|
||||
|
||||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.16.0-incubating-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-core</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.twitter4j</groupId>
|
||||
<artifactId>twitter4j-core</artifactId>
|
||||
<version>3.0.3</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.twitter4j</groupId>
|
||||
<artifactId>twitter4j-async</artifactId>
|
||||
<version>3.0.3</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.twitter4j</groupId>
|
||||
<artifactId>twitter4j-stream</artifactId>
|
||||
<version>3.0.3</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-validator</groupId>
|
||||
<artifactId>commons-validator</artifactId>
|
||||
<version>1.5.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.ircclouds.irc</groupId>
|
||||
<artifactId>irc-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.maxmind.geoip2</groupId>
|
||||
<artifactId>geoip2</artifactId>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!-- For tests! -->
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-shade-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<phase>package</phase>
|
||||
<goals>
|
||||
<goal>shade</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<outputFile>
|
||||
${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar
|
||||
</outputFile>
|
||||
<filters>
|
||||
<filter>
|
||||
<artifact>*:*</artifact>
|
||||
<excludes>
|
||||
<exclude>META-INF/*.SF</exclude>
|
||||
<exclude>META-INF/*.DSA</exclude>
|
||||
<exclude>META-INF/*.RSA</exclude>
|
||||
</excludes>
|
||||
</filter>
|
||||
</filters>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-jar-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<goals>
|
||||
<goal>test-jar</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</project>
|
|
@ -1,56 +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.druid.examples;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.inject.Binder;
|
||||
import org.apache.druid.examples.twitter.TwitterSpritzerFirehoseFactory;
|
||||
import org.apache.druid.examples.wikipedia.IrcFirehoseFactory;
|
||||
import org.apache.druid.examples.wikipedia.IrcInputRowParser;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ExamplesDruidModule implements DruidModule
|
||||
{
|
||||
@Override
|
||||
public List<? extends Module> getJacksonModules()
|
||||
{
|
||||
return Collections.<Module>singletonList(
|
||||
new SimpleModule("ExamplesModule")
|
||||
.registerSubtypes(
|
||||
new NamedType(TwitterSpritzerFirehoseFactory.class, "twitzer"),
|
||||
new NamedType(IrcFirehoseFactory.class, "irc"),
|
||||
new NamedType(IrcInputRowParser.class, "irc")
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
|
||||
}
|
||||
}
|
|
@ -1,387 +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.druid.examples.twitter;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.data.input.Firehose;
|
||||
import org.apache.druid.data.input.FirehoseFactory;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.MapBasedInputRow;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import twitter4j.ConnectionLifeCycleListener;
|
||||
import twitter4j.GeoLocation;
|
||||
import twitter4j.HashtagEntity;
|
||||
import twitter4j.StallWarning;
|
||||
import twitter4j.Status;
|
||||
import twitter4j.StatusDeletionNotice;
|
||||
import twitter4j.StatusListener;
|
||||
import twitter4j.TwitterStream;
|
||||
import twitter4j.TwitterStreamFactory;
|
||||
import twitter4j.User;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.ArrayBlockingQueue;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
/**
|
||||
* Twitter "spritzer" Firehose Factory named "twitzer".
|
||||
* Builds a Firehose that emits a stream of
|
||||
* ??
|
||||
* with timestamps along with ??.
|
||||
* The generated tuples have the form (timestamp, ????)
|
||||
* where the timestamp is from the twitter event.
|
||||
* <p/>
|
||||
* Example spec file:
|
||||
* <p/>
|
||||
* Example query using POST to /druid/v2/?w (where w is an arbitrary parameter and the date and time
|
||||
* is UTC):
|
||||
* <p/>
|
||||
* Notes on twitter.com HTTP (REST) API: v1.0 will be disabled around 2013-03 so v1.1 should be used;
|
||||
* twitter4j 3.0 (not yet released) will support the v1.1 api.
|
||||
* Specifically, we should be using https://stream.twitter.com/1.1/statuses/sample.json
|
||||
* See: http://jira.twitter4j.org/browse/TFJ-186
|
||||
* <p/>
|
||||
* Notes on JSON parsing: as of twitter4j 2.2.x, the json parser has some bugs (ex: Status.toString()
|
||||
* can have number format exceptions), so it might be necessary to extract raw json and process it
|
||||
* separately. If so, set twitter4.jsonStoreEnabled=true and look at DataObjectFactory#getRawJSON();
|
||||
* com.fasterxml.jackson.databind.ObjectMapper should be used to parse.
|
||||
*/
|
||||
@JsonTypeName("twitzer")
|
||||
public class TwitterSpritzerFirehoseFactory implements FirehoseFactory<InputRowParser>
|
||||
{
|
||||
private static final Logger log = new Logger(TwitterSpritzerFirehoseFactory.class);
|
||||
private static final Pattern sourcePattern = Pattern.compile("<a[^>]*>(.*?)</a>", Pattern.CASE_INSENSITIVE);
|
||||
private static final int DEFAULT_QUEUE_SIZE = 2000;
|
||||
|
||||
/**
|
||||
* max events to receive, -1 is infinite, 0 means nothing is delivered; use this to prevent
|
||||
* infinite space consumption or to prevent getting throttled at an inconvenient time
|
||||
* or to see what happens when a Firehose stops delivering
|
||||
* values, or to have hasMore() return false. The Twitter Spritzer can deliver about
|
||||
* 1000 events per minute.
|
||||
*/
|
||||
private final int maxEventCount;
|
||||
/**
|
||||
* maximum number of minutes to fetch Twitter events. Use this to prevent getting
|
||||
* throttled at an inconvenient time. If zero or less, no time limit for run.
|
||||
*/
|
||||
private final int maxRunMinutes;
|
||||
|
||||
@JsonCreator
|
||||
public TwitterSpritzerFirehoseFactory(
|
||||
@JsonProperty("maxEventCount") Integer maxEventCount,
|
||||
@JsonProperty("maxRunMinutes") Integer maxRunMinutes
|
||||
)
|
||||
{
|
||||
this.maxEventCount = maxEventCount;
|
||||
this.maxRunMinutes = maxRunMinutes;
|
||||
log.info("maxEventCount=" + ((maxEventCount <= 0) ? "no limit" : maxEventCount));
|
||||
log.info("maxRunMinutes=" + ((maxRunMinutes <= 0) ? "no limit" : maxRunMinutes));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect(InputRowParser parser, File temporaryDirectory)
|
||||
{
|
||||
final ConnectionLifeCycleListener connectionLifeCycleListener = new ConnectionLifeCycleListener()
|
||||
{
|
||||
@Override
|
||||
public void onConnect()
|
||||
{
|
||||
log.info("Connected_to_Twitter");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onDisconnect()
|
||||
{
|
||||
log.info("Disconnect_from_Twitter");
|
||||
}
|
||||
|
||||
/**
|
||||
* called before thread gets cleaned up
|
||||
*/
|
||||
@Override
|
||||
public void onCleanUp()
|
||||
{
|
||||
log.info("Cleanup_twitter_stream");
|
||||
}
|
||||
}; // ConnectionLifeCycleListener
|
||||
|
||||
final TwitterStream twitterStream;
|
||||
final StatusListener statusListener;
|
||||
/** This queue is used to move twitter events from the twitter4j thread to the druid ingest thread. */
|
||||
final BlockingQueue<Status> queue = new ArrayBlockingQueue<Status>(DEFAULT_QUEUE_SIZE);
|
||||
final long startMsec = System.currentTimeMillis();
|
||||
|
||||
//
|
||||
// set up Twitter Spritzer
|
||||
//
|
||||
twitterStream = new TwitterStreamFactory().getInstance();
|
||||
twitterStream.addConnectionLifeCycleListener(connectionLifeCycleListener);
|
||||
statusListener = new StatusListener()
|
||||
{ // This is what really gets called to deliver stuff from twitter4j
|
||||
@Override
|
||||
public void onStatus(Status status)
|
||||
{
|
||||
// time to stop?
|
||||
if (Thread.currentThread().isInterrupted()) {
|
||||
throw new RuntimeException("Interrupted, time to stop");
|
||||
}
|
||||
try {
|
||||
boolean success = queue.offer(status, 15L, TimeUnit.SECONDS);
|
||||
if (!success) {
|
||||
log.warn("queue too slow!");
|
||||
}
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RuntimeException("InterruptedException", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onDeletionNotice(StatusDeletionNotice statusDeletionNotice)
|
||||
{
|
||||
//log.info("Got a status deletion notice id:" + statusDeletionNotice.getStatusId());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrackLimitationNotice(int numberOfLimitedStatuses)
|
||||
{
|
||||
// This notice will be sent each time a limited stream becomes unlimited.
|
||||
// If this number is high and or rapidly increasing, it is an indication that your predicate is too broad, and you should consider a predicate with higher selectivity.
|
||||
log.warn("Got track limitation notice:" + numberOfLimitedStatuses);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onScrubGeo(long userId, long upToStatusId)
|
||||
{
|
||||
//log.info("Got scrub_geo event userId:" + userId + " upToStatusId:" + upToStatusId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onException(Exception ex)
|
||||
{
|
||||
log.error(ex, "Got exception");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onStallWarning(StallWarning warning)
|
||||
{
|
||||
log.warn("Got stall warning: %s", warning);
|
||||
}
|
||||
};
|
||||
|
||||
twitterStream.addListener(statusListener);
|
||||
twitterStream.sample(); // creates a generic StatusStream
|
||||
log.info("returned from sample()");
|
||||
|
||||
return new Firehose()
|
||||
{
|
||||
|
||||
private final Runnable doNothingRunnable = new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
private long rowCount = 0L;
|
||||
private boolean waitIfmax = (getMaxEventCount() < 0L);
|
||||
private final Map<String, Object> theMap = new TreeMap<>();
|
||||
// DIY json parsing // private final ObjectMapper omapper = new ObjectMapper();
|
||||
|
||||
private boolean maxTimeReached()
|
||||
{
|
||||
if (getMaxRunMinutes() <= 0) {
|
||||
return false;
|
||||
} else {
|
||||
return (System.currentTimeMillis() - startMsec) / 60000L >= getMaxRunMinutes();
|
||||
}
|
||||
}
|
||||
|
||||
private boolean maxCountReached()
|
||||
{
|
||||
return getMaxEventCount() >= 0 && rowCount >= getMaxEventCount();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMore()
|
||||
{
|
||||
if (maxCountReached() || maxTimeReached()) {
|
||||
return waitIfmax;
|
||||
} else {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public InputRow nextRow()
|
||||
{
|
||||
// Interrupted to stop?
|
||||
if (Thread.currentThread().isInterrupted()) {
|
||||
throw new RuntimeException("Interrupted, time to stop");
|
||||
}
|
||||
|
||||
// all done?
|
||||
if (maxCountReached() || maxTimeReached()) {
|
||||
if (waitIfmax) {
|
||||
// sleep a long time instead of terminating
|
||||
try {
|
||||
log.info("reached limit, sleeping a long time...");
|
||||
Thread.sleep(2000000000L);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RuntimeException("InterruptedException", e);
|
||||
}
|
||||
} else {
|
||||
// allow this event through, and the next hasMore() call will be false
|
||||
}
|
||||
}
|
||||
if (++rowCount % 1000 == 0) {
|
||||
log.info("nextRow() has returned %,d InputRows", rowCount);
|
||||
}
|
||||
|
||||
Status status;
|
||||
try {
|
||||
status = queue.take();
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RuntimeException("InterruptedException", e);
|
||||
}
|
||||
|
||||
theMap.clear();
|
||||
|
||||
HashtagEntity[] hts = status.getHashtagEntities();
|
||||
String text = status.getText();
|
||||
theMap.put("text", (null == text) ? "" : text);
|
||||
theMap.put(
|
||||
"htags", (hts.length > 0) ? Lists.transform(
|
||||
Arrays.asList(hts), new Function<HashtagEntity, String>()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public String apply(HashtagEntity input)
|
||||
{
|
||||
return input.getText();
|
||||
}
|
||||
}
|
||||
) : ImmutableList.<String>of()
|
||||
);
|
||||
|
||||
long[] lcontrobutors = status.getContributors();
|
||||
List<String> contributors = new ArrayList<>();
|
||||
for (long contrib : lcontrobutors) {
|
||||
contributors.add(StringUtils.format("%d", contrib));
|
||||
}
|
||||
theMap.put("contributors", contributors);
|
||||
|
||||
GeoLocation geoLocation = status.getGeoLocation();
|
||||
if (null != geoLocation) {
|
||||
double lat = status.getGeoLocation().getLatitude();
|
||||
double lon = status.getGeoLocation().getLongitude();
|
||||
theMap.put("lat", lat);
|
||||
theMap.put("lon", lon);
|
||||
} else {
|
||||
theMap.put("lat", null);
|
||||
theMap.put("lon", null);
|
||||
}
|
||||
|
||||
if (status.getSource() != null) {
|
||||
Matcher m = sourcePattern.matcher(status.getSource());
|
||||
theMap.put("source", m.find() ? m.group(1) : status.getSource());
|
||||
}
|
||||
|
||||
theMap.put("retweet", status.isRetweet());
|
||||
|
||||
if (status.isRetweet()) {
|
||||
Status original = status.getRetweetedStatus();
|
||||
theMap.put("retweet_count", original.getRetweetCount());
|
||||
|
||||
User originator = original.getUser();
|
||||
theMap.put("originator_screen_name", originator != null ? originator.getScreenName() : "");
|
||||
theMap.put("originator_follower_count", originator != null ? originator.getFollowersCount() : "");
|
||||
theMap.put("originator_friends_count", originator != null ? originator.getFriendsCount() : "");
|
||||
theMap.put("originator_verified", originator != null ? originator.isVerified() : "");
|
||||
}
|
||||
|
||||
User user = status.getUser();
|
||||
final boolean hasUser = (null != user);
|
||||
theMap.put("follower_count", hasUser ? user.getFollowersCount() : 0);
|
||||
theMap.put("friends_count", hasUser ? user.getFriendsCount() : 0);
|
||||
theMap.put("lang", hasUser ? user.getLang() : "");
|
||||
theMap.put("utc_offset", hasUser ? user.getUtcOffset() : -1); // resolution in seconds, -1 if not available?
|
||||
theMap.put("statuses_count", hasUser ? user.getStatusesCount() : 0);
|
||||
theMap.put("user_id", hasUser ? StringUtils.format("%d", user.getId()) : "");
|
||||
theMap.put("screen_name", hasUser ? user.getScreenName() : "");
|
||||
theMap.put("location", hasUser ? user.getLocation() : "");
|
||||
theMap.put("verified", hasUser ? user.isVerified() : "");
|
||||
|
||||
theMap.put("ts", status.getCreatedAt().getTime());
|
||||
|
||||
List<String> dimensions = Lists.newArrayList(theMap.keySet());
|
||||
|
||||
return new MapBasedInputRow(status.getCreatedAt().getTime(), dimensions, theMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Runnable commit()
|
||||
{
|
||||
// ephemera in, ephemera out.
|
||||
return doNothingRunnable; // reuse the same object each time
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
log.info("CLOSE twitterstream");
|
||||
twitterStream.shutdown(); // invokes twitterStream.cleanUp()
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getMaxEventCount()
|
||||
{
|
||||
return maxEventCount;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getMaxRunMinutes()
|
||||
{
|
||||
return maxRunMinutes;
|
||||
}
|
||||
}
|
|
@ -1,34 +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.druid.examples.wikipedia;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes({
|
||||
@JsonSubTypes.Type(name = "wikipedia", value = WikipediaIrcDecoder.class)
|
||||
})
|
||||
public interface IrcDecoder
|
||||
{
|
||||
InputRow decodeMessage(DateTime timestamp, String channel, String msg);
|
||||
}
|
|
@ -1,256 +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.druid.examples.wikipedia;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.ircclouds.irc.api.Callback;
|
||||
import com.ircclouds.irc.api.IRCApi;
|
||||
import com.ircclouds.irc.api.IRCApiImpl;
|
||||
import com.ircclouds.irc.api.IServerParameters;
|
||||
import com.ircclouds.irc.api.domain.IRCServer;
|
||||
import com.ircclouds.irc.api.domain.messages.ChannelPrivMsg;
|
||||
import com.ircclouds.irc.api.listeners.VariousMessageListenerAdapter;
|
||||
import com.ircclouds.irc.api.state.IIRCState;
|
||||
import org.apache.druid.data.input.Firehose;
|
||||
import org.apache.druid.data.input.FirehoseFactory;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* <p><b>Example code:</b></p>
|
||||
* <pre>{@code
|
||||
* <p/>
|
||||
* IrcFirehoseFactory factory = new IrcFirehoseFactory(
|
||||
* "wiki123",
|
||||
* "irc.wikimedia.org",
|
||||
* Lists.newArrayList(
|
||||
* "#en.wikipedia",
|
||||
* "#fr.wikipedia",
|
||||
* "#de.wikipedia",
|
||||
* "#ja.wikipedia"
|
||||
* )
|
||||
* );
|
||||
* }</pre>
|
||||
*/
|
||||
public class IrcFirehoseFactory implements FirehoseFactory<InputRowParser<Pair<DateTime, ChannelPrivMsg>>>
|
||||
{
|
||||
private static final Logger log = new Logger(IrcFirehoseFactory.class);
|
||||
|
||||
private final String nick;
|
||||
private final String host;
|
||||
private final List<String> channels;
|
||||
private volatile boolean closed = false;
|
||||
|
||||
@JsonCreator
|
||||
public IrcFirehoseFactory(
|
||||
@JsonProperty("nick") String nick,
|
||||
@JsonProperty("host") String host,
|
||||
@JsonProperty("channels") List<String> channels
|
||||
)
|
||||
{
|
||||
this.nick = nick;
|
||||
this.host = host;
|
||||
this.channels = channels;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getNick()
|
||||
{
|
||||
return nick;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getHost()
|
||||
{
|
||||
return host;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<String> getChannels()
|
||||
{
|
||||
return channels;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect(
|
||||
final InputRowParser<Pair<DateTime, ChannelPrivMsg>> firehoseParser,
|
||||
final File temporaryDirectory
|
||||
)
|
||||
{
|
||||
final IRCApi irc = new IRCApiImpl(false);
|
||||
final LinkedBlockingQueue<Pair<DateTime, ChannelPrivMsg>> queue = new LinkedBlockingQueue<Pair<DateTime, ChannelPrivMsg>>();
|
||||
|
||||
irc.addListener(
|
||||
new VariousMessageListenerAdapter()
|
||||
{
|
||||
@Override
|
||||
public void onChannelMessage(ChannelPrivMsg aMsg)
|
||||
{
|
||||
try {
|
||||
queue.put(Pair.of(DateTimes.nowUtc(), aMsg));
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RuntimeException("interrupted adding message to queue", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
log.info("connecting to irc server [%s]", host);
|
||||
irc.connect(
|
||||
new IServerParameters()
|
||||
{
|
||||
@Override
|
||||
public String getNickname()
|
||||
{
|
||||
return nick;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getAlternativeNicknames()
|
||||
{
|
||||
return Lists.newArrayList(nick + UUID.randomUUID(), nick + UUID.randomUUID(), nick + UUID.randomUUID());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getIdent()
|
||||
{
|
||||
return "druid";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getRealname()
|
||||
{
|
||||
return nick;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IRCServer getServer()
|
||||
{
|
||||
return new IRCServer(host, false);
|
||||
}
|
||||
},
|
||||
new Callback<IIRCState>()
|
||||
{
|
||||
@Override
|
||||
public void onSuccess(IIRCState aObject)
|
||||
{
|
||||
log.info("irc connection to server [%s] established", host);
|
||||
for (String chan : channels) {
|
||||
log.info("Joining channel %s", chan);
|
||||
irc.joinChannel(chan);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e)
|
||||
{
|
||||
log.error(e, "Unable to connect to irc server [%s]", host);
|
||||
throw new RuntimeException("Unable to connect to server", e);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
closed = false;
|
||||
|
||||
return new Firehose()
|
||||
{
|
||||
InputRow nextRow = null;
|
||||
Iterator<InputRow> nextIterator = Collections.emptyIterator();
|
||||
|
||||
@Override
|
||||
public boolean hasMore()
|
||||
{
|
||||
try {
|
||||
while (true) {
|
||||
if (closed) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (nextIterator.hasNext()) {
|
||||
nextRow = nextIterator.next();
|
||||
if (nextRow != null) {
|
||||
return true;
|
||||
}
|
||||
} else {
|
||||
Pair<DateTime, ChannelPrivMsg> nextMsg = queue.poll(100, TimeUnit.MILLISECONDS);
|
||||
if (nextMsg == null) {
|
||||
continue;
|
||||
}
|
||||
try {
|
||||
nextIterator = firehoseParser.parseBatch(nextMsg).iterator();
|
||||
}
|
||||
catch (IllegalArgumentException iae) {
|
||||
log.debug("ignoring invalid message in channel [%s]", nextMsg.rhs.getChannelName());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
Thread.interrupted();
|
||||
throw new RuntimeException("interrupted retrieving elements from queue", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public InputRow nextRow()
|
||||
{
|
||||
return nextRow;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Runnable commit()
|
||||
{
|
||||
return () -> {
|
||||
// nothing to see here
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
try {
|
||||
log.info("disconnecting from irc server [%s]", host);
|
||||
irc.disconnect("");
|
||||
}
|
||||
finally {
|
||||
closed = true;
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
@ -1,107 +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.druid.examples.wikipedia;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.ircclouds.irc.api.domain.messages.ChannelPrivMsg;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
import org.apache.druid.data.input.impl.ParseSpec;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* <p><b>Example Usage</b></p>
|
||||
* <p/>
|
||||
* <p>Decoder definition: <code>wikipedia-decoder.json</code></p>
|
||||
* <pre>{@code
|
||||
* <p/>
|
||||
* {
|
||||
* "type": "wikipedia",
|
||||
* "namespaces": {
|
||||
* "#en.wikipedia": {
|
||||
* "": "main",
|
||||
* "Category": "category",
|
||||
* "Template talk": "template talk",
|
||||
* "Help talk": "help talk",
|
||||
* "Media": "media",
|
||||
* "MediaWiki talk": "mediawiki talk",
|
||||
* "File talk": "file talk",
|
||||
* "MediaWiki": "mediawiki",
|
||||
* "User": "user",
|
||||
* "File": "file",
|
||||
* "User talk": "user talk",
|
||||
* "Template": "template",
|
||||
* "Help": "help",
|
||||
* "Special": "special",
|
||||
* "Talk": "talk",
|
||||
* "Category talk": "category talk"
|
||||
* }
|
||||
* },
|
||||
* "geoIpDatabase": "path/to/GeoLite2-City.mmdb"
|
||||
* }
|
||||
* }</pre>
|
||||
*/
|
||||
@JsonTypeName("irc")
|
||||
public class IrcInputRowParser implements InputRowParser<Pair<DateTime, ChannelPrivMsg>>
|
||||
{
|
||||
private final ParseSpec parseSpec;
|
||||
private final IrcDecoder decoder;
|
||||
|
||||
@JsonCreator
|
||||
public IrcInputRowParser(
|
||||
@JsonProperty("parseSpec") ParseSpec parseSpec,
|
||||
@JsonProperty("decoder") IrcDecoder decoder
|
||||
)
|
||||
{
|
||||
this.parseSpec = parseSpec;
|
||||
this.decoder = decoder;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public IrcDecoder getDecoder()
|
||||
{
|
||||
return decoder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<InputRow> parseBatch(Pair<DateTime, ChannelPrivMsg> msg)
|
||||
{
|
||||
return ImmutableList.of(decoder.decodeMessage(msg.lhs, msg.rhs.getChannelName(), msg.rhs.getText()));
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public ParseSpec getParseSpec()
|
||||
{
|
||||
return parseSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRowParser withParseSpec(ParseSpec parseSpec)
|
||||
{
|
||||
return new IrcInputRowParser(parseSpec, decoder);
|
||||
}
|
||||
}
|
|
@ -1,307 +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.druid.examples.wikipedia;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.maxmind.geoip2.DatabaseReader;
|
||||
import com.maxmind.geoip2.exception.GeoIp2Exception;
|
||||
import com.maxmind.geoip2.model.Omni;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.net.URL;
|
||||
import java.net.UnknownHostException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.zip.GZIPInputStream;
|
||||
|
||||
class WikipediaIrcDecoder implements IrcDecoder
|
||||
{
|
||||
static final Logger LOG = new Logger(WikipediaIrcDecoder.class);
|
||||
|
||||
private static final Pattern PATTERN = Pattern.compile(
|
||||
".*\\x0314\\[\\[\\x0307(.+?)\\x0314\\]\\]\\x034 (.*?)\\x0310.*\\x0302(http.+?)" +
|
||||
"\\x03.+\\x0303(.+?)\\x03.+\\x03 (\\(([+-]\\d+)\\).*|.+) \\x0310(.+)\\x03.*"
|
||||
);
|
||||
|
||||
private static final Pattern IP_PATTERN = Pattern.compile("\\d+.\\d+.\\d+.\\d+");
|
||||
private static final Pattern SHORTNAME_PATTERN = Pattern.compile("#(\\w\\w)\\..*");
|
||||
private static final Pattern SINGLE_SPACE_PATTERN = Pattern.compile("\\s");
|
||||
|
||||
static final List<String> dimensionList = Lists.newArrayList(
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city"
|
||||
);
|
||||
|
||||
final DatabaseReader geoLookup;
|
||||
final Map<String, Map<String, String>> namespaces;
|
||||
final String geoIpDatabase;
|
||||
|
||||
public WikipediaIrcDecoder(Map<String, Map<String, String>> namespaces)
|
||||
{
|
||||
this(namespaces, null);
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public WikipediaIrcDecoder(
|
||||
@JsonProperty("namespaces") Map<String, Map<String, String>> namespaces,
|
||||
@JsonProperty("geoIpDatabase") String geoIpDatabase
|
||||
)
|
||||
{
|
||||
if (namespaces == null) {
|
||||
namespaces = new HashMap<>();
|
||||
}
|
||||
this.namespaces = namespaces;
|
||||
this.geoIpDatabase = geoIpDatabase;
|
||||
|
||||
if (geoIpDatabase != null) {
|
||||
this.geoLookup = openGeoIpDb(new File(geoIpDatabase));
|
||||
} else {
|
||||
this.geoLookup = openDefaultGeoIpDb();
|
||||
}
|
||||
}
|
||||
|
||||
private DatabaseReader openDefaultGeoIpDb()
|
||||
{
|
||||
File geoDb = new File(System.getProperty("java.io.tmpdir"),
|
||||
this.getClass().getCanonicalName() + ".GeoLite2-City.mmdb");
|
||||
try {
|
||||
return openDefaultGeoIpDb(geoDb);
|
||||
}
|
||||
catch (RuntimeException e) {
|
||||
LOG.warn(e.getMessage() + " Attempting to re-download.", e);
|
||||
if (geoDb.exists() && !geoDb.delete()) {
|
||||
throw new RuntimeException("Could not delete geo db file [" + geoDb.getAbsolutePath() + "].");
|
||||
}
|
||||
// local download may be corrupt, will retry once.
|
||||
return openDefaultGeoIpDb(geoDb);
|
||||
}
|
||||
}
|
||||
|
||||
private DatabaseReader openDefaultGeoIpDb(File geoDb)
|
||||
{
|
||||
downloadGeoLiteDbToFile(geoDb);
|
||||
return openGeoIpDb(geoDb);
|
||||
}
|
||||
|
||||
private DatabaseReader openGeoIpDb(File geoDb)
|
||||
{
|
||||
try {
|
||||
DatabaseReader reader = new DatabaseReader(geoDb);
|
||||
LOG.info("Using geo ip database at [%s].", geoDb);
|
||||
return reader;
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException("Could not open geo db at [" + geoDb.getAbsolutePath() + "].", e);
|
||||
}
|
||||
}
|
||||
|
||||
private void downloadGeoLiteDbToFile(File geoDb)
|
||||
{
|
||||
if (geoDb.exists()) {
|
||||
return;
|
||||
}
|
||||
|
||||
try {
|
||||
LOG.info("Downloading geo ip database to [%s]. This may take a few minutes.", geoDb.getAbsolutePath());
|
||||
|
||||
File tmpFile = File.createTempFile("druid", "geo");
|
||||
|
||||
FileUtils.copyInputStreamToFile(
|
||||
new GZIPInputStream(
|
||||
new URL("http://geolite.maxmind.com/download/geoip/database/GeoLite2-City.mmdb.gz").openStream()
|
||||
),
|
||||
tmpFile
|
||||
);
|
||||
|
||||
if (!tmpFile.renameTo(geoDb)) {
|
||||
throw new RuntimeException("Unable to move geo file to [" + geoDb.getAbsolutePath() + "]!");
|
||||
}
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException("Unable to download geo ip database.", e);
|
||||
}
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Map<String, Map<String, String>> getNamespaces()
|
||||
{
|
||||
return namespaces;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getGeoIpDatabase()
|
||||
{
|
||||
return geoIpDatabase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRow decodeMessage(final DateTime timestamp, String channel, String msg)
|
||||
{
|
||||
final Map<String, String> dimensions = new HashMap<>();
|
||||
final Map<String, Float> metrics = new HashMap<>();
|
||||
|
||||
Matcher m = PATTERN.matcher(msg);
|
||||
if (!m.matches()) {
|
||||
throw new IllegalArgumentException("Invalid input format");
|
||||
}
|
||||
|
||||
Matcher shortname = SHORTNAME_PATTERN.matcher(channel);
|
||||
if (shortname.matches()) {
|
||||
dimensions.put("language", shortname.group(1));
|
||||
}
|
||||
|
||||
String page = m.group(1);
|
||||
String pageUrl = SINGLE_SPACE_PATTERN.matcher(page).replaceAll("_");
|
||||
|
||||
dimensions.put("page", pageUrl);
|
||||
|
||||
String user = m.group(4);
|
||||
Matcher ipMatch = IP_PATTERN.matcher(user);
|
||||
boolean anonymous = ipMatch.matches();
|
||||
if (anonymous) {
|
||||
try {
|
||||
final InetAddress ip = InetAddress.getByName(ipMatch.group());
|
||||
final Omni lookup = geoLookup.omni(ip);
|
||||
|
||||
dimensions.put("continent", lookup.getContinent().getName());
|
||||
dimensions.put("country", lookup.getCountry().getName());
|
||||
dimensions.put("region", lookup.getMostSpecificSubdivision().getName());
|
||||
dimensions.put("city", lookup.getCity().getName());
|
||||
}
|
||||
catch (UnknownHostException e) {
|
||||
LOG.error(e, "invalid ip [%s]", ipMatch.group());
|
||||
}
|
||||
catch (IOException e) {
|
||||
LOG.error(e, "error looking up geo ip");
|
||||
}
|
||||
catch (GeoIp2Exception e) {
|
||||
LOG.error(e, "error looking up geo ip");
|
||||
}
|
||||
}
|
||||
dimensions.put("user", user);
|
||||
|
||||
final String flags = m.group(2);
|
||||
dimensions.put("unpatrolled", Boolean.toString(flags.contains("!")));
|
||||
dimensions.put("newPage", Boolean.toString(flags.contains("N")));
|
||||
dimensions.put("robot", Boolean.toString(flags.contains("B")));
|
||||
|
||||
dimensions.put("anonymous", Boolean.toString(anonymous));
|
||||
|
||||
String[] parts = page.split(":");
|
||||
if (parts.length > 1 && !parts[1].startsWith(" ")) {
|
||||
Map<String, String> channelNamespaces = namespaces.get(channel);
|
||||
if (channelNamespaces != null && channelNamespaces.containsKey(parts[0])) {
|
||||
dimensions.put("namespace", channelNamespaces.get(parts[0]));
|
||||
} else {
|
||||
dimensions.put("namespace", "wikipedia");
|
||||
}
|
||||
} else {
|
||||
dimensions.put("namespace", "article");
|
||||
}
|
||||
|
||||
float delta = m.group(6) != null ? Float.parseFloat(m.group(6)) : 0;
|
||||
metrics.put("delta", delta);
|
||||
metrics.put("added", Math.max(delta, 0));
|
||||
metrics.put("deleted", Math.min(delta, 0));
|
||||
|
||||
return new InputRow()
|
||||
{
|
||||
@Override
|
||||
public List<String> getDimensions()
|
||||
{
|
||||
return dimensionList;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTimestampFromEpoch()
|
||||
{
|
||||
return timestamp.getMillis();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DateTime getTimestamp()
|
||||
{
|
||||
return timestamp;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getDimension(String dimension)
|
||||
{
|
||||
final String value = dimensions.get(dimension);
|
||||
if (value != null) {
|
||||
return ImmutableList.of(value);
|
||||
} else {
|
||||
return ImmutableList.of();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getRaw(String dimension)
|
||||
{
|
||||
return dimensions.get(dimension);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Number getMetric(String metric)
|
||||
{
|
||||
return metrics.get(metric);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(Row o)
|
||||
{
|
||||
return timestamp.compareTo(o.getTimestamp());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "WikipediaRow{" +
|
||||
"timestamp=" + timestamp +
|
||||
", dimensions=" + dimensions +
|
||||
", metrics=" + metrics +
|
||||
'}';
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -1,16 +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.
|
||||
|
||||
org.apache.druid.examples.ExamplesDruidModule
|
|
@ -1,58 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ 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>druid</artifactId>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<version>0.16.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
<groupId>org.apache.druid.extensions.contrib</groupId>
|
||||
<artifactId>druid-rocketmq</artifactId>
|
||||
|
||||
<properties>
|
||||
<rocketmq.version>3.2.6</rocketmq.version>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.alibaba.rocketmq</groupId>
|
||||
<artifactId>rocketmq-client</artifactId>
|
||||
<version>${rocketmq.version}</version>
|
||||
<exclusions>
|
||||
<!-- Druid uses its own netty version -->
|
||||
<exclusion>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty-all</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-core</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
|
@ -1,50 +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.druid.firehose.rocketmq;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class RocketMQDruidModule implements DruidModule
|
||||
{
|
||||
|
||||
@Override
|
||||
public List<? extends Module> getJacksonModules()
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new SimpleModule("RocketMQFirehoseModule")
|
||||
.registerSubtypes(
|
||||
new NamedType(RocketMQFirehoseFactory.class, "rocketMQ")
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
|
||||
}
|
||||
}
|
|
@ -1,584 +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.druid.firehose.rocketmq;
|
||||
|
||||
import com.alibaba.rocketmq.client.Validators;
|
||||
import com.alibaba.rocketmq.client.consumer.DefaultMQPullConsumer;
|
||||
import com.alibaba.rocketmq.client.consumer.MessageQueueListener;
|
||||
import com.alibaba.rocketmq.client.consumer.PullResult;
|
||||
import com.alibaba.rocketmq.client.consumer.store.OffsetStore;
|
||||
import com.alibaba.rocketmq.client.consumer.store.ReadOffsetType;
|
||||
import com.alibaba.rocketmq.client.exception.MQBrokerException;
|
||||
import com.alibaba.rocketmq.client.exception.MQClientException;
|
||||
import com.alibaba.rocketmq.common.ServiceThread;
|
||||
import com.alibaba.rocketmq.common.message.MessageExt;
|
||||
import com.alibaba.rocketmq.common.message.MessageQueue;
|
||||
import com.alibaba.rocketmq.common.protocol.heartbeat.MessageModel;
|
||||
import com.alibaba.rocketmq.remoting.exception.RemotingException;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.druid.data.input.Firehose;
|
||||
import org.apache.druid.data.input.FirehoseFactory;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.java.util.common.parsers.ParseException;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
public class RocketMQFirehoseFactory implements FirehoseFactory<InputRowParser<ByteBuffer>>
|
||||
{
|
||||
|
||||
private static final Logger LOGGER = new Logger(RocketMQFirehoseFactory.class);
|
||||
|
||||
/**
|
||||
* Passed in configuration for consumer client.
|
||||
* This provides an approach to override default values defined in {@link com.alibaba.rocketmq.common.MixAll}.
|
||||
*/
|
||||
@JsonProperty
|
||||
private final Properties consumerProps;
|
||||
|
||||
/**
|
||||
* Consumer group. It's required.
|
||||
*/
|
||||
@JsonProperty(required = true)
|
||||
private final String consumerGroup;
|
||||
|
||||
/**
|
||||
* Topics to consume. It's required.
|
||||
*/
|
||||
@JsonProperty(required = true)
|
||||
private final List<String> feed;
|
||||
|
||||
/**
|
||||
* Pull batch size. It's optional.
|
||||
*/
|
||||
@JsonProperty
|
||||
private final String pullBatchSize;
|
||||
|
||||
/**
|
||||
* Store messages that are fetched from brokers but not yet delivered to druid via fire hose.
|
||||
*/
|
||||
private final ConcurrentHashMap<MessageQueue, ConcurrentSkipListSet<MessageExt>> messageQueueTreeSetMap =
|
||||
new ConcurrentHashMap<>();
|
||||
|
||||
/**
|
||||
* Store message consuming status.
|
||||
*/
|
||||
private final ConcurrentHashMap<MessageQueue, ConcurrentSkipListSet<Long>> windows = new ConcurrentHashMap<>();
|
||||
|
||||
/**
|
||||
* Default pull batch size.
|
||||
*/
|
||||
private static final int DEFAULT_PULL_BATCH_SIZE = 32;
|
||||
|
||||
@JsonCreator
|
||||
public RocketMQFirehoseFactory(
|
||||
@JsonProperty("consumerProps") Properties consumerProps,
|
||||
@JsonProperty("consumerGroup") String consumerGroup,
|
||||
@JsonProperty("feed") List<String> feed,
|
||||
@JsonProperty("pullBatchSize") String pullBatchSize
|
||||
)
|
||||
{
|
||||
this.consumerProps = consumerProps;
|
||||
this.pullBatchSize = pullBatchSize;
|
||||
for (Map.Entry<Object, Object> configItem : this.consumerProps.entrySet()) {
|
||||
System.setProperty(configItem.getKey().toString(), configItem.getValue().toString());
|
||||
}
|
||||
this.consumerGroup = consumerGroup;
|
||||
this.feed = feed;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if there are locally pending messages to consume.
|
||||
*
|
||||
* @return true if there are some; false otherwise.
|
||||
*/
|
||||
private boolean hasMessagesPending()
|
||||
{
|
||||
|
||||
for (Map.Entry<MessageQueue, ConcurrentSkipListSet<MessageExt>> entry : messageQueueTreeSetMap.entrySet()) {
|
||||
if (!entry.getValue().isEmpty()) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect(
|
||||
InputRowParser<ByteBuffer> byteBufferInputRowParser,
|
||||
File temporaryDirectory
|
||||
) throws IOException, ParseException
|
||||
{
|
||||
|
||||
Set<String> newDimExclus = Sets.union(
|
||||
byteBufferInputRowParser.getParseSpec().getDimensionsSpec().getDimensionExclusions(),
|
||||
Sets.newHashSet("feed")
|
||||
);
|
||||
|
||||
final InputRowParser<ByteBuffer> theParser = byteBufferInputRowParser.withParseSpec(
|
||||
byteBufferInputRowParser.getParseSpec()
|
||||
.withDimensionsSpec(
|
||||
byteBufferInputRowParser.getParseSpec()
|
||||
.getDimensionsSpec()
|
||||
.withDimensionExclusions(
|
||||
newDimExclus
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
/**
|
||||
* Topic-Queue mapping.
|
||||
*/
|
||||
final ConcurrentHashMap<String, Set<MessageQueue>> topicQueueMap;
|
||||
|
||||
/**
|
||||
* Default Pull-style client for RocketMQ.
|
||||
*/
|
||||
final DefaultMQPullConsumer defaultMQPullConsumer;
|
||||
final DruidPullMessageService pullMessageService;
|
||||
|
||||
messageQueueTreeSetMap.clear();
|
||||
windows.clear();
|
||||
|
||||
try {
|
||||
defaultMQPullConsumer = new DefaultMQPullConsumer(this.consumerGroup);
|
||||
defaultMQPullConsumer.setMessageModel(MessageModel.CLUSTERING);
|
||||
topicQueueMap = new ConcurrentHashMap<>();
|
||||
|
||||
pullMessageService = new DruidPullMessageService(defaultMQPullConsumer);
|
||||
for (String topic : feed) {
|
||||
Validators.checkTopic(topic);
|
||||
topicQueueMap.put(topic, defaultMQPullConsumer.fetchSubscribeMessageQueues(topic));
|
||||
}
|
||||
DruidMessageQueueListener druidMessageQueueListener =
|
||||
new DruidMessageQueueListener(Sets.newHashSet(feed), topicQueueMap, defaultMQPullConsumer);
|
||||
defaultMQPullConsumer.setMessageQueueListener(druidMessageQueueListener);
|
||||
defaultMQPullConsumer.start();
|
||||
pullMessageService.start();
|
||||
}
|
||||
catch (MQClientException e) {
|
||||
LOGGER.error(e, "Failed to start DefaultMQPullConsumer");
|
||||
throw new IOException("Failed to start RocketMQ client", e);
|
||||
}
|
||||
|
||||
return new Firehose()
|
||||
{
|
||||
private Iterator<InputRow> nextIterator = Collections.emptyIterator();
|
||||
|
||||
@Override
|
||||
public boolean hasMore()
|
||||
{
|
||||
if (nextIterator.hasNext()) {
|
||||
return true;
|
||||
}
|
||||
boolean hasMore = false;
|
||||
DruidPullRequest earliestPullRequest = null;
|
||||
|
||||
for (Map.Entry<String, Set<MessageQueue>> entry : topicQueueMap.entrySet()) {
|
||||
for (MessageQueue messageQueue : entry.getValue()) {
|
||||
ConcurrentSkipListSet<MessageExt> messages = messageQueueTreeSetMap.get(messageQueue);
|
||||
if (messages != null && !messages.isEmpty()) {
|
||||
hasMore = true;
|
||||
} else {
|
||||
try {
|
||||
long offset = defaultMQPullConsumer.fetchConsumeOffset(messageQueue, false);
|
||||
int batchSize = (null == pullBatchSize || pullBatchSize.isEmpty()) ?
|
||||
DEFAULT_PULL_BATCH_SIZE : Integer.parseInt(pullBatchSize);
|
||||
|
||||
DruidPullRequest newPullRequest = new DruidPullRequest(messageQueue, null, offset,
|
||||
batchSize, !hasMessagesPending()
|
||||
);
|
||||
|
||||
// notify pull message service to pull messages from brokers.
|
||||
pullMessageService.putRequest(newPullRequest);
|
||||
|
||||
// set the earliest pull in case we need to block.
|
||||
if (null == earliestPullRequest) {
|
||||
earliestPullRequest = newPullRequest;
|
||||
}
|
||||
}
|
||||
catch (MQClientException e) {
|
||||
LOGGER.error("Failed to fetch consume offset for queue: %s", entry.getKey());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Block only when there is no locally pending messages.
|
||||
if (!hasMore && null != earliestPullRequest) {
|
||||
try {
|
||||
earliestPullRequest.getCountDownLatch().await();
|
||||
hasMore = true;
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
LOGGER.error(e, "CountDownLatch await got interrupted");
|
||||
}
|
||||
}
|
||||
return hasMore;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public InputRow nextRow()
|
||||
{
|
||||
if (nextIterator.hasNext()) {
|
||||
return nextIterator.next();
|
||||
}
|
||||
|
||||
for (Map.Entry<MessageQueue, ConcurrentSkipListSet<MessageExt>> entry : messageQueueTreeSetMap.entrySet()) {
|
||||
if (!entry.getValue().isEmpty()) {
|
||||
MessageExt message = entry.getValue().pollFirst();
|
||||
nextIterator = theParser.parseBatch(ByteBuffer.wrap(message.getBody())).iterator();
|
||||
|
||||
windows
|
||||
.computeIfAbsent(entry.getKey(), k -> new ConcurrentSkipListSet<>())
|
||||
.add(message.getQueueOffset());
|
||||
return nextIterator.next();
|
||||
}
|
||||
}
|
||||
|
||||
// should never happen.
|
||||
throw new RuntimeException("Unexpected Fatal Error! There should have been one row available.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Runnable commit()
|
||||
{
|
||||
return new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
OffsetStore offsetStore = defaultMQPullConsumer.getOffsetStore();
|
||||
Set<MessageQueue> updated = new HashSet<>();
|
||||
// calculate offsets according to consuming windows.
|
||||
for (Map.Entry<MessageQueue, ConcurrentSkipListSet<Long>> entry : windows.entrySet()) {
|
||||
while (!entry.getValue().isEmpty()) {
|
||||
|
||||
long offset = offsetStore.readOffset(entry.getKey(), ReadOffsetType.MEMORY_FIRST_THEN_STORE);
|
||||
if (offset + 1 > entry.getValue().first()) {
|
||||
entry.getValue().pollFirst();
|
||||
} else if (offset + 1 == entry.getValue().first()) {
|
||||
entry.getValue().pollFirst();
|
||||
offsetStore.updateOffset(entry.getKey(), offset + 1, true);
|
||||
updated.add(entry.getKey());
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
offsetStore.persistAll(updated);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
defaultMQPullConsumer.shutdown();
|
||||
pullMessageService.shutdown(false);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Pull request.
|
||||
*/
|
||||
static final class DruidPullRequest
|
||||
{
|
||||
private final MessageQueue messageQueue;
|
||||
private final String tag;
|
||||
private final long nextBeginOffset;
|
||||
private final int pullBatchSize;
|
||||
private final boolean longPull;
|
||||
private final CountDownLatch countDownLatch;
|
||||
|
||||
public DruidPullRequest(
|
||||
final MessageQueue messageQueue,
|
||||
final String tag,
|
||||
final long nextBeginOffset,
|
||||
final int pullBatchSize,
|
||||
final boolean useLongPull
|
||||
)
|
||||
{
|
||||
this.messageQueue = messageQueue;
|
||||
this.tag = (null == tag ? "*" : tag);
|
||||
this.nextBeginOffset = nextBeginOffset;
|
||||
this.pullBatchSize = pullBatchSize;
|
||||
this.longPull = useLongPull;
|
||||
countDownLatch = new CountDownLatch(1);
|
||||
}
|
||||
|
||||
public MessageQueue getMessageQueue()
|
||||
{
|
||||
return messageQueue;
|
||||
}
|
||||
|
||||
public long getNextBeginOffset()
|
||||
{
|
||||
return nextBeginOffset;
|
||||
}
|
||||
|
||||
public String getTag()
|
||||
{
|
||||
return tag;
|
||||
}
|
||||
|
||||
public int getPullBatchSize()
|
||||
{
|
||||
return pullBatchSize;
|
||||
}
|
||||
|
||||
public boolean isLongPull()
|
||||
{
|
||||
return longPull;
|
||||
}
|
||||
|
||||
public CountDownLatch getCountDownLatch()
|
||||
{
|
||||
return countDownLatch;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Pull message service for druid.
|
||||
* <p/>
|
||||
* <strong>Note: this is a single thread service.</strong>
|
||||
*/
|
||||
final class DruidPullMessageService extends ServiceThread
|
||||
{
|
||||
|
||||
private volatile List<DruidPullRequest> requestsWrite = new ArrayList<>();
|
||||
private volatile List<DruidPullRequest> requestsRead = new ArrayList<>();
|
||||
|
||||
private final DefaultMQPullConsumer defaultMQPullConsumer;
|
||||
|
||||
public DruidPullMessageService(final DefaultMQPullConsumer defaultMQPullConsumer)
|
||||
{
|
||||
this.defaultMQPullConsumer = defaultMQPullConsumer;
|
||||
}
|
||||
|
||||
public void putRequest(final DruidPullRequest request)
|
||||
{
|
||||
synchronized (this) {
|
||||
this.requestsWrite.add(request);
|
||||
if (!hasNotified) {
|
||||
hasNotified = true;
|
||||
// No need to use notifyAll here since extended class com.alibaba.rocketmq.common.ServiceThread handles it
|
||||
notify();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void swapRequests()
|
||||
{
|
||||
List<DruidPullRequest> tmp = requestsWrite;
|
||||
requestsWrite = requestsRead;
|
||||
requestsRead = tmp;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getServiceName()
|
||||
{
|
||||
return getClass().getSimpleName();
|
||||
}
|
||||
|
||||
/**
|
||||
* Core message pulling logic code goes here.
|
||||
*/
|
||||
private void doPull()
|
||||
{
|
||||
for (DruidPullRequest pullRequest : requestsRead) {
|
||||
PullResult pullResult;
|
||||
try {
|
||||
if (!pullRequest.isLongPull()) {
|
||||
pullResult = defaultMQPullConsumer.pull(
|
||||
pullRequest.getMessageQueue(),
|
||||
pullRequest.getTag(),
|
||||
pullRequest.getNextBeginOffset(),
|
||||
pullRequest.getPullBatchSize()
|
||||
);
|
||||
} else {
|
||||
pullResult = defaultMQPullConsumer.pullBlockIfNotFound(
|
||||
pullRequest.getMessageQueue(),
|
||||
pullRequest.getTag(),
|
||||
pullRequest.getNextBeginOffset(),
|
||||
pullRequest.getPullBatchSize()
|
||||
);
|
||||
}
|
||||
|
||||
switch (pullResult.getPullStatus()) {
|
||||
case FOUND:
|
||||
// Handle pull result.
|
||||
messageQueueTreeSetMap
|
||||
.computeIfAbsent(pullRequest.getMessageQueue(), k -> new ConcurrentSkipListSet<>(MESSAGE_COMPARATOR))
|
||||
.addAll(pullResult.getMsgFoundList());
|
||||
break;
|
||||
|
||||
case NO_NEW_MSG:
|
||||
case NO_MATCHED_MSG:
|
||||
break;
|
||||
|
||||
case OFFSET_ILLEGAL:
|
||||
LOGGER.error(
|
||||
"Bad Pull Request: Offset is illegal. Offset used: %d",
|
||||
pullRequest.getNextBeginOffset()
|
||||
);
|
||||
break;
|
||||
|
||||
default:
|
||||
break;
|
||||
}
|
||||
}
|
||||
catch (MQClientException | RemotingException | MQBrokerException | InterruptedException e) {
|
||||
LOGGER.error(e, "Failed to pull message from broker.");
|
||||
}
|
||||
finally {
|
||||
pullRequest.getCountDownLatch().countDown();
|
||||
}
|
||||
|
||||
}
|
||||
requestsRead.clear();
|
||||
}
|
||||
|
||||
/**
|
||||
* Thread looping entry.
|
||||
*/
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
LOGGER.info(getServiceName() + " starts.");
|
||||
while (!isStoped()) {
|
||||
waitForRunning(0);
|
||||
doPull();
|
||||
}
|
||||
|
||||
// in case this service is shutdown gracefully without interruption.
|
||||
try {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
LOGGER.error(e, "");
|
||||
}
|
||||
|
||||
synchronized (this) {
|
||||
swapRequests();
|
||||
}
|
||||
|
||||
doPull();
|
||||
LOGGER.info(getServiceName() + " terminated.");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void onWaitEnd()
|
||||
{
|
||||
swapRequests();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Compare messages pulled from same message queue according to queue offset.
|
||||
*/
|
||||
private static final Comparator<MessageExt> MESSAGE_COMPARATOR = Comparator.comparingLong(MessageExt::getQueueOffset);
|
||||
|
||||
|
||||
/**
|
||||
* Handle message queues re-balance operations.
|
||||
*/
|
||||
final class DruidMessageQueueListener implements MessageQueueListener
|
||||
{
|
||||
|
||||
private final Set<String> topics;
|
||||
|
||||
private final ConcurrentHashMap<String, Set<MessageQueue>> topicQueueMap;
|
||||
|
||||
private final DefaultMQPullConsumer defaultMQPullConsumer;
|
||||
|
||||
public DruidMessageQueueListener(
|
||||
final Set<String> topics,
|
||||
final ConcurrentHashMap<String, Set<MessageQueue>> topicQueueMap,
|
||||
final DefaultMQPullConsumer defaultMQPullConsumer
|
||||
)
|
||||
{
|
||||
this.topics = topics;
|
||||
this.topicQueueMap = topicQueueMap;
|
||||
this.defaultMQPullConsumer = defaultMQPullConsumer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void messageQueueChanged(String topic, Set<MessageQueue> mqAll, Set<MessageQueue> mqDivided)
|
||||
{
|
||||
if (topics.contains(topic)) {
|
||||
topicQueueMap.put(topic, mqDivided);
|
||||
|
||||
// Remove message queues that are re-assigned to other clients.
|
||||
Iterator<Map.Entry<MessageQueue, ConcurrentSkipListSet<MessageExt>>> it =
|
||||
messageQueueTreeSetMap.entrySet().iterator();
|
||||
while (it.hasNext()) {
|
||||
if (!mqDivided.contains(it.next().getKey())) {
|
||||
it.remove();
|
||||
}
|
||||
}
|
||||
|
||||
StringBuilder stringBuilder = new StringBuilder();
|
||||
for (MessageQueue messageQueue : mqDivided) {
|
||||
stringBuilder.append(messageQueue.getBrokerName())
|
||||
.append("#")
|
||||
.append(messageQueue.getQueueId())
|
||||
.append(", ");
|
||||
}
|
||||
|
||||
if (LOGGER.isDebugEnabled() && stringBuilder.length() > 2) {
|
||||
LOGGER.debug(StringUtils.format(
|
||||
"%s@%s is consuming the following message queues: %s",
|
||||
defaultMQPullConsumer.getClientIP(),
|
||||
defaultMQPullConsumer.getInstanceName(),
|
||||
stringBuilder.substring(0, stringBuilder.length() - 2) /*Remove the trailing comma*/
|
||||
));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,89 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ 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/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>org.apache.druid.extensions.contrib</groupId>
|
||||
<artifactId>druid-kafka-eight-simple-consumer</artifactId>
|
||||
<name>druid-kafka-eight-simple-consumer</name>
|
||||
<description>druid-kafka-eight-simple-consumer</description>
|
||||
|
||||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.16.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-core</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.kafka</groupId>
|
||||
<artifactId>kafka_2.10</artifactId>
|
||||
<version>0.8.2.1</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>log4j</groupId>
|
||||
<artifactId>log4j</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.zookeeper</groupId>
|
||||
<artifactId>zookeeper</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-api</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>net.jpountz.lz4</groupId>
|
||||
<artifactId>lz4</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<!-- Tests -->
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<artifactId>maven-jar-plugin</artifactId>
|
||||
<configuration>
|
||||
<archive>
|
||||
<manifest>
|
||||
<addDefaultImplementationEntries>true</addDefaultImplementationEntries>
|
||||
<addDefaultSpecificationEntries>true</addDefaultSpecificationEntries>
|
||||
</manifest>
|
||||
</archive>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</project>
|
|
@ -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.druid.firehose.kafka;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
@Deprecated
|
||||
public class KafkaEightSimpleConsumerDruidModule implements DruidModule
|
||||
{
|
||||
@Override
|
||||
public List<? extends Module> getJacksonModules()
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new SimpleModule("KafkaEightSimpleConsumerFirehoseModule").registerSubtypes(
|
||||
new NamedType(KafkaEightSimpleConsumerFirehoseFactory.class, "kafka-0.8-v2")
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
|
||||
}
|
||||
}
|
|
@ -1,350 +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.druid.firehose.kafka;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.io.Closeables;
|
||||
import org.apache.druid.data.input.ByteBufferInputRowParser;
|
||||
import org.apache.druid.data.input.Committer;
|
||||
import org.apache.druid.data.input.FirehoseFactoryV2;
|
||||
import org.apache.druid.data.input.FirehoseV2;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.firehose.kafka.KafkaSimpleConsumer.BytesMessageWithOffset;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.parsers.ParseException;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
@Deprecated
|
||||
public class KafkaEightSimpleConsumerFirehoseFactory implements
|
||||
FirehoseFactoryV2<ByteBufferInputRowParser>
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(
|
||||
KafkaEightSimpleConsumerFirehoseFactory.class
|
||||
);
|
||||
|
||||
@JsonProperty
|
||||
private final List<String> brokerList;
|
||||
|
||||
@JsonProperty
|
||||
private final List<Integer> partitionIdList;
|
||||
|
||||
@JsonProperty
|
||||
private final String clientId;
|
||||
|
||||
@JsonProperty
|
||||
private final String feed;
|
||||
|
||||
@JsonProperty
|
||||
private final int queueBufferLength;
|
||||
|
||||
@JsonProperty
|
||||
private final boolean earliest;
|
||||
|
||||
private final List<PartitionConsumerWorker> consumerWorkers = new CopyOnWriteArrayList<>();
|
||||
private static final int DEFAULT_QUEUE_BUFFER_LENGTH = 20000;
|
||||
private static final int CONSUMER_FETCH_TIMEOUT_MILLIS = (int) TimeUnit.SECONDS.toMillis(10);
|
||||
|
||||
@JsonCreator
|
||||
public KafkaEightSimpleConsumerFirehoseFactory(
|
||||
@JsonProperty("brokerList") List<String> brokerList,
|
||||
@JsonProperty("partitionIdList") List<Integer> partitionIdList,
|
||||
@JsonProperty("clientId") String clientId,
|
||||
@JsonProperty("feed") String feed,
|
||||
@JsonProperty("queueBufferLength") Integer queueBufferLength,
|
||||
@JsonProperty("resetOffsetToEarliest") Boolean resetOffsetToEarliest
|
||||
)
|
||||
{
|
||||
this.brokerList = brokerList;
|
||||
Preconditions.checkArgument(
|
||||
brokerList != null && brokerList.size() > 0,
|
||||
"brokerList is null/empty"
|
||||
);
|
||||
|
||||
this.partitionIdList = partitionIdList;
|
||||
Preconditions.checkArgument(
|
||||
partitionIdList != null && partitionIdList.size() > 0,
|
||||
"partitionIdList is null/empty"
|
||||
);
|
||||
|
||||
this.clientId = clientId;
|
||||
Preconditions.checkArgument(
|
||||
clientId != null && !clientId.isEmpty(),
|
||||
"clientId is null/empty"
|
||||
);
|
||||
|
||||
this.feed = feed;
|
||||
Preconditions.checkArgument(
|
||||
feed != null && !feed.isEmpty(),
|
||||
"feed is null/empty"
|
||||
);
|
||||
|
||||
this.queueBufferLength = queueBufferLength == null ? DEFAULT_QUEUE_BUFFER_LENGTH : queueBufferLength;
|
||||
Preconditions.checkArgument(this.queueBufferLength > 0, "queueBufferLength must be positive number");
|
||||
log.info("queueBufferLength loaded as[%s]", this.queueBufferLength);
|
||||
|
||||
this.earliest = resetOffsetToEarliest == null ? true : resetOffsetToEarliest.booleanValue();
|
||||
log.info(
|
||||
"if old offsets are not known, data from partition will be read from [%s] available offset.",
|
||||
this.earliest ? "earliest" : "latest"
|
||||
);
|
||||
}
|
||||
|
||||
private Map<Integer, Long> loadOffsetFromPreviousMetaData(Object lastCommit)
|
||||
{
|
||||
Map<Integer, Long> offsetMap = new HashMap<>();
|
||||
if (lastCommit == null) {
|
||||
return offsetMap;
|
||||
}
|
||||
if (lastCommit instanceof Map) {
|
||||
Map<Object, Object> lastCommitMap = (Map) lastCommit;
|
||||
for (Map.Entry<Object, Object> entry : lastCommitMap.entrySet()) {
|
||||
try {
|
||||
int partitionId = Integer.parseInt(entry.getKey().toString());
|
||||
long offset = Long.parseLong(entry.getValue().toString());
|
||||
log.debug("Recover last commit information partitionId [%s], offset [%s]", partitionId, offset);
|
||||
offsetMap.put(partitionId, offset);
|
||||
}
|
||||
catch (NumberFormatException e) {
|
||||
log.error(e, "Fail to load offset from previous meta data [%s]", entry);
|
||||
}
|
||||
}
|
||||
log.info("Loaded offset map[%s]", offsetMap);
|
||||
} else {
|
||||
log.makeAlert("Unable to cast lastCommit to Map for feed [%s]", feed).emit();
|
||||
}
|
||||
return offsetMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FirehoseV2 connect(final ByteBufferInputRowParser firehoseParser, Object lastCommit)
|
||||
{
|
||||
final Map<Integer, Long> lastOffsets = loadOffsetFromPreviousMetaData(lastCommit);
|
||||
|
||||
for (Integer partition : partitionIdList) {
|
||||
final KafkaSimpleConsumer kafkaSimpleConsumer =
|
||||
new KafkaSimpleConsumer(feed, partition, clientId, brokerList, earliest);
|
||||
Long startOffset = lastOffsets.get(partition);
|
||||
PartitionConsumerWorker worker = new PartitionConsumerWorker(
|
||||
feed,
|
||||
kafkaSimpleConsumer,
|
||||
partition,
|
||||
startOffset == null ? -1 : startOffset
|
||||
);
|
||||
consumerWorkers.add(worker);
|
||||
}
|
||||
|
||||
final LinkedBlockingQueue<BytesMessageWithOffset> messageQueue = new LinkedBlockingQueue<BytesMessageWithOffset>(
|
||||
queueBufferLength
|
||||
);
|
||||
log.info("Kicking off all consumers");
|
||||
for (PartitionConsumerWorker worker : consumerWorkers) {
|
||||
worker.go(messageQueue);
|
||||
}
|
||||
log.info("All consumer started");
|
||||
|
||||
return new FirehoseV2()
|
||||
{
|
||||
private Map<Integer, Long> lastOffsetPartitions;
|
||||
private volatile boolean stopped;
|
||||
private volatile BytesMessageWithOffset msg = null;
|
||||
private volatile InputRow row = null;
|
||||
private volatile Iterator<InputRow> nextIterator = Collections.emptyIterator();
|
||||
|
||||
{
|
||||
lastOffsetPartitions = new HashMap<>();
|
||||
lastOffsetPartitions.putAll(lastOffsets);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start()
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean advance()
|
||||
{
|
||||
if (stopped) {
|
||||
return false;
|
||||
}
|
||||
|
||||
nextMessage();
|
||||
return true;
|
||||
}
|
||||
|
||||
private void nextMessage()
|
||||
{
|
||||
try {
|
||||
row = null;
|
||||
while (row == null) {
|
||||
if (!nextIterator.hasNext()) {
|
||||
if (msg != null) {
|
||||
lastOffsetPartitions.put(msg.getPartition(), msg.offset());
|
||||
}
|
||||
msg = messageQueue.take();
|
||||
final byte[] message = msg.message();
|
||||
nextIterator = message == null
|
||||
? Collections.emptyIterator()
|
||||
: firehoseParser.parseBatch(ByteBuffer.wrap(message)).iterator();
|
||||
continue;
|
||||
}
|
||||
row = nextIterator.next();
|
||||
}
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
//Let the caller decide whether to stop or continue when thread is interrupted.
|
||||
log.warn(e, "Thread Interrupted while taking from queue, propagating the interrupt");
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputRow currRow()
|
||||
{
|
||||
if (stopped) {
|
||||
return null;
|
||||
}
|
||||
// currRow will be called before the first advance
|
||||
if (row == null) {
|
||||
try {
|
||||
nextMessage();
|
||||
}
|
||||
catch (ParseException e) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
return row;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Committer makeCommitter()
|
||||
{
|
||||
final Map<Integer, Long> offsets = new HashMap<>(lastOffsetPartitions);
|
||||
|
||||
return new Committer()
|
||||
{
|
||||
@Override
|
||||
public Object getMetadata()
|
||||
{
|
||||
return offsets;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
log.info("Stopping kafka 0.8 simple firehose");
|
||||
stopped = true;
|
||||
for (PartitionConsumerWorker t : consumerWorkers) {
|
||||
Closeables.close(t, true);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private static class PartitionConsumerWorker implements Closeable
|
||||
{
|
||||
private final String topic;
|
||||
private final KafkaSimpleConsumer consumer;
|
||||
private final int partitionId;
|
||||
private final long startOffset;
|
||||
|
||||
private final AtomicBoolean stopped = new AtomicBoolean(false);
|
||||
private volatile Thread thread = null;
|
||||
|
||||
PartitionConsumerWorker(String topic, KafkaSimpleConsumer consumer, int partitionId, long startOffset)
|
||||
{
|
||||
this.topic = topic;
|
||||
this.consumer = consumer;
|
||||
this.partitionId = partitionId;
|
||||
this.startOffset = startOffset;
|
||||
}
|
||||
|
||||
public void go(final LinkedBlockingQueue<BytesMessageWithOffset> messageQueue)
|
||||
{
|
||||
thread = new Thread()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
long offset = startOffset;
|
||||
log.info("Start running parition[%s], offset[%s]", partitionId, offset);
|
||||
try {
|
||||
while (!stopped.get()) {
|
||||
try {
|
||||
Iterable<BytesMessageWithOffset> msgs = consumer.fetch(offset, CONSUMER_FETCH_TIMEOUT_MILLIS);
|
||||
int count = 0;
|
||||
for (BytesMessageWithOffset msgWithOffset : msgs) {
|
||||
offset = msgWithOffset.offset();
|
||||
messageQueue.put(msgWithOffset);
|
||||
count++;
|
||||
}
|
||||
log.debug("fetch [%s] msgs for partition [%s] in one time ", count, partitionId);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
log.info("Interrupted when fetching data, shutting down.");
|
||||
return;
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.error(e, "Exception happened in fetching data, but will continue consuming");
|
||||
}
|
||||
}
|
||||
}
|
||||
finally {
|
||||
consumer.stop();
|
||||
}
|
||||
}
|
||||
};
|
||||
thread.setDaemon(true);
|
||||
thread.setName(StringUtils.format("kafka-%s-%s", topic, partitionId));
|
||||
thread.start();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void close()
|
||||
{
|
||||
if (stopped.compareAndSet(false, true)) {
|
||||
thread.interrupt();
|
||||
thread = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,390 +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.druid.firehose.kafka;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.net.HostAndPort;
|
||||
import kafka.api.FetchRequest;
|
||||
import kafka.api.FetchRequestBuilder;
|
||||
import kafka.api.PartitionOffsetRequestInfo;
|
||||
import kafka.cluster.Broker;
|
||||
import kafka.common.ErrorMapping;
|
||||
import kafka.common.TopicAndPartition;
|
||||
import kafka.javaapi.FetchResponse;
|
||||
import kafka.javaapi.OffsetRequest;
|
||||
import kafka.javaapi.OffsetResponse;
|
||||
import kafka.javaapi.PartitionMetadata;
|
||||
import kafka.javaapi.TopicMetadata;
|
||||
import kafka.javaapi.TopicMetadataRequest;
|
||||
import kafka.javaapi.TopicMetadataResponse;
|
||||
import kafka.javaapi.consumer.SimpleConsumer;
|
||||
import kafka.message.MessageAndOffset;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.guava.FunctionalIterable;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* refer @{link https://cwiki.apache.org/confluence/display/KAFKA/0.8.0+SimpleConsumer+Example}
|
||||
* <p>
|
||||
* This class is not thread safe, the caller must ensure all the methods be
|
||||
* called from single thread
|
||||
*/
|
||||
@Deprecated
|
||||
public class KafkaSimpleConsumer
|
||||
{
|
||||
|
||||
private static final Logger log = new Logger(KafkaSimpleConsumer.class);
|
||||
|
||||
private final List<HostAndPort> allBrokers;
|
||||
private final String topic;
|
||||
private final int partitionId;
|
||||
private final String clientId;
|
||||
private final String leaderLookupClientId;
|
||||
private final boolean earliest;
|
||||
|
||||
private volatile Broker leaderBroker;
|
||||
private List<HostAndPort> replicaBrokers;
|
||||
private SimpleConsumer consumer = null;
|
||||
|
||||
private static final int SO_TIMEOUT_MILLIS = (int) TimeUnit.SECONDS.toMillis(30);
|
||||
private static final int BUFFER_SIZE = 65536;
|
||||
private static final long RETRY_INTERVAL_MILLIS = TimeUnit.MINUTES.toMillis(1);
|
||||
private static final int FETCH_SIZE = 100_000_000;
|
||||
|
||||
public KafkaSimpleConsumer(String topic, int partitionId, String clientId, List<String> brokers, boolean earliest)
|
||||
{
|
||||
List<HostAndPort> brokerList = new ArrayList<>();
|
||||
for (String broker : brokers) {
|
||||
HostAndPort brokerHostAndPort = HostAndPort.fromString(broker);
|
||||
Preconditions.checkArgument(
|
||||
brokerHostAndPort.getHostText() != null &&
|
||||
!brokerHostAndPort.getHostText().isEmpty() &&
|
||||
brokerHostAndPort.hasPort(),
|
||||
"kafka broker [%s] is not valid, must be <host>:<port>",
|
||||
broker
|
||||
);
|
||||
brokerList.add(brokerHostAndPort);
|
||||
}
|
||||
|
||||
this.allBrokers = Collections.unmodifiableList(brokerList);
|
||||
this.topic = topic;
|
||||
this.partitionId = partitionId;
|
||||
this.clientId = StringUtils.format("%s_%d_%s", topic, partitionId, clientId);
|
||||
this.leaderLookupClientId = clientId + "leaderLookup";
|
||||
this.replicaBrokers = new ArrayList<>();
|
||||
this.replicaBrokers.addAll(this.allBrokers);
|
||||
this.earliest = earliest;
|
||||
log.info(
|
||||
"KafkaSimpleConsumer initialized with clientId [%s] for message consumption and clientId [%s] for leader lookup",
|
||||
this.clientId,
|
||||
this.leaderLookupClientId
|
||||
);
|
||||
}
|
||||
|
||||
private void ensureConsumer(Broker leader) throws InterruptedException
|
||||
{
|
||||
if (consumer == null) {
|
||||
while (leaderBroker == null) {
|
||||
leaderBroker = findNewLeader(leader);
|
||||
}
|
||||
log.info(
|
||||
"making SimpleConsumer[%s][%s], leader broker[%s:%s]",
|
||||
topic, partitionId, leaderBroker.host(), leaderBroker.port()
|
||||
);
|
||||
|
||||
consumer = new SimpleConsumer(
|
||||
leaderBroker.host(), leaderBroker.port(), SO_TIMEOUT_MILLIS, BUFFER_SIZE, clientId
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
public static class BytesMessageWithOffset
|
||||
{
|
||||
final byte[] msg;
|
||||
final long offset;
|
||||
final int partition;
|
||||
|
||||
public BytesMessageWithOffset(byte[] msg, long offset, int partition)
|
||||
{
|
||||
this.msg = msg;
|
||||
this.offset = offset;
|
||||
this.partition = partition;
|
||||
}
|
||||
|
||||
public int getPartition()
|
||||
{
|
||||
return partition;
|
||||
}
|
||||
|
||||
public byte[] message()
|
||||
{
|
||||
return msg;
|
||||
}
|
||||
|
||||
public long offset()
|
||||
{
|
||||
return offset;
|
||||
}
|
||||
}
|
||||
|
||||
private Iterable<BytesMessageWithOffset> filterAndDecode(Iterable<MessageAndOffset> kafkaMessages, final long offset)
|
||||
{
|
||||
return FunctionalIterable
|
||||
.create(kafkaMessages)
|
||||
.filter(
|
||||
new Predicate<MessageAndOffset>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(MessageAndOffset msgAndOffset)
|
||||
{
|
||||
return msgAndOffset.offset() >= offset;
|
||||
}
|
||||
}
|
||||
)
|
||||
.transform(
|
||||
new Function<MessageAndOffset, BytesMessageWithOffset>()
|
||||
{
|
||||
|
||||
@Override
|
||||
public BytesMessageWithOffset apply(MessageAndOffset msgAndOffset)
|
||||
{
|
||||
ByteBuffer bb = msgAndOffset.message().payload();
|
||||
byte[] payload = new byte[bb.remaining()];
|
||||
bb.get(payload);
|
||||
// add nextOffset here, thus next fetch will use nextOffset instead of current offset
|
||||
return new BytesMessageWithOffset(payload, msgAndOffset.nextOffset(), partitionId);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private long getOffset(boolean earliest) throws InterruptedException
|
||||
{
|
||||
TopicAndPartition topicAndPartition = new TopicAndPartition(topic, partitionId);
|
||||
Map<TopicAndPartition, PartitionOffsetRequestInfo> requestInfo = new HashMap<TopicAndPartition, PartitionOffsetRequestInfo>();
|
||||
requestInfo.put(
|
||||
topicAndPartition,
|
||||
new PartitionOffsetRequestInfo(
|
||||
earliest ? kafka.api.OffsetRequest.EarliestTime() : kafka.api.OffsetRequest.LatestTime(), 1
|
||||
)
|
||||
);
|
||||
OffsetRequest request = new OffsetRequest(requestInfo, kafka.api.OffsetRequest.CurrentVersion(), clientId);
|
||||
OffsetResponse response;
|
||||
try {
|
||||
response = consumer.getOffsetsBefore(request);
|
||||
}
|
||||
catch (Exception e) {
|
||||
ensureNotInterrupted(e);
|
||||
log.error(e, "caught exception in getOffsetsBefore [%s] - [%s]", topic, partitionId);
|
||||
return -1;
|
||||
}
|
||||
if (response.hasError()) {
|
||||
log.error(
|
||||
"error fetching data Offset from the Broker [%s]. reason: [%s]", leaderBroker.host(),
|
||||
response.errorCode(topic, partitionId)
|
||||
);
|
||||
return -1;
|
||||
}
|
||||
long[] offsets = response.offsets(topic, partitionId);
|
||||
return earliest ? offsets[0] : offsets[offsets.length - 1];
|
||||
}
|
||||
|
||||
public Iterable<BytesMessageWithOffset> fetch(long offset, int timeoutMs) throws InterruptedException
|
||||
{
|
||||
FetchResponse response;
|
||||
Broker previousLeader = leaderBroker;
|
||||
while (true) {
|
||||
ensureConsumer(previousLeader);
|
||||
|
||||
FetchRequest request = new FetchRequestBuilder()
|
||||
.clientId(clientId)
|
||||
.addFetch(topic, partitionId, offset, FETCH_SIZE)
|
||||
.maxWait(timeoutMs)
|
||||
.minBytes(1)
|
||||
.build();
|
||||
|
||||
log.debug("fetch offset %s", offset);
|
||||
|
||||
try {
|
||||
response = consumer.fetch(request);
|
||||
}
|
||||
catch (Exception e) {
|
||||
ensureNotInterrupted(e);
|
||||
log.warn(e, "caught exception in fetch %s - %d", topic, partitionId);
|
||||
response = null;
|
||||
}
|
||||
|
||||
if (response == null || response.hasError()) {
|
||||
short errorCode = response != null ? response.errorCode(topic, partitionId) : ErrorMapping.UnknownCode();
|
||||
log.warn("fetch %s - %s with offset %s encounters error: [%s]", topic, partitionId, offset, errorCode);
|
||||
|
||||
boolean needNewLeader = false;
|
||||
if (errorCode == ErrorMapping.RequestTimedOutCode()) {
|
||||
log.info("kafka request timed out, response[%s]", response);
|
||||
} else if (errorCode == ErrorMapping.OffsetOutOfRangeCode()) {
|
||||
long newOffset = getOffset(earliest);
|
||||
log.info("got [%s] offset[%s] for [%s][%s]", earliest ? "earliest" : "latest", newOffset, topic, partitionId);
|
||||
if (newOffset < 0) {
|
||||
needNewLeader = true;
|
||||
} else {
|
||||
offset = newOffset;
|
||||
continue;
|
||||
}
|
||||
} else {
|
||||
needNewLeader = true;
|
||||
}
|
||||
|
||||
if (needNewLeader) {
|
||||
stopConsumer();
|
||||
previousLeader = leaderBroker;
|
||||
leaderBroker = null;
|
||||
continue;
|
||||
}
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return filterAndDecode(response.messageSet(topic, partitionId), offset);
|
||||
}
|
||||
|
||||
private void stopConsumer()
|
||||
{
|
||||
if (consumer != null) {
|
||||
try {
|
||||
consumer.close();
|
||||
log.info("stop consumer[%s][%s], leaderBroker[%s]", topic, partitionId, leaderBroker);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.warn(e, "stop consumer[%s][%s] failed", topic, partitionId);
|
||||
}
|
||||
finally {
|
||||
consumer = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void stop()
|
||||
{
|
||||
stopConsumer();
|
||||
log.info("KafkaSimpleConsumer[%s][%s] stopped", topic, partitionId);
|
||||
}
|
||||
|
||||
private PartitionMetadata findLeader() throws InterruptedException
|
||||
{
|
||||
for (HostAndPort broker : replicaBrokers) {
|
||||
SimpleConsumer consumer = null;
|
||||
try {
|
||||
log.info("Finding new leader from Kafka brokers, try broker [%s]", broker.toString());
|
||||
consumer = new SimpleConsumer(broker.getHostText(), broker.getPort(), SO_TIMEOUT_MILLIS, BUFFER_SIZE, leaderLookupClientId);
|
||||
TopicMetadataResponse resp = consumer.send(new TopicMetadataRequest(Collections.singletonList(topic)));
|
||||
|
||||
List<TopicMetadata> metaData = resp.topicsMetadata();
|
||||
for (TopicMetadata item : metaData) {
|
||||
if (topic.equals(item.topic())) {
|
||||
for (PartitionMetadata part : item.partitionsMetadata()) {
|
||||
if (part.partitionId() == partitionId) {
|
||||
return part;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
ensureNotInterrupted(e);
|
||||
log.warn(
|
||||
e,
|
||||
"error communicating with Kafka Broker [%s] to find leader for [%s] - [%s]",
|
||||
broker,
|
||||
topic,
|
||||
partitionId
|
||||
);
|
||||
}
|
||||
finally {
|
||||
if (consumer != null) {
|
||||
consumer.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
private Broker findNewLeader(Broker oldLeader) throws InterruptedException
|
||||
{
|
||||
long retryCnt = 0;
|
||||
while (true) {
|
||||
PartitionMetadata metadata = findLeader();
|
||||
if (metadata != null) {
|
||||
replicaBrokers.clear();
|
||||
for (Broker replica : metadata.replicas()) {
|
||||
replicaBrokers.add(
|
||||
HostAndPort.fromParts(replica.host(), replica.port())
|
||||
);
|
||||
}
|
||||
|
||||
log.debug("Got new Kafka leader metadata : [%s], previous leader : [%s]", metadata, oldLeader);
|
||||
Broker newLeader = metadata.leader();
|
||||
if (newLeader != null) {
|
||||
// We check the retryCnt here as well to make sure that we have slept a little bit
|
||||
// if we don't notice a change in leadership
|
||||
// just in case if Zookeeper doesn't get updated fast enough
|
||||
if (oldLeader == null || isValidNewLeader(newLeader) || retryCnt != 0) {
|
||||
return newLeader;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Thread.sleep(RETRY_INTERVAL_MILLIS);
|
||||
retryCnt++;
|
||||
// if could not find the leader for current replicaBrokers, let's try to
|
||||
// find one via allBrokers
|
||||
if (retryCnt >= 3 && (retryCnt - 3) % 5 == 0) {
|
||||
log.warn("cannot find leader for [%s] - [%s] after [%s] retries", topic, partitionId, retryCnt);
|
||||
replicaBrokers.clear();
|
||||
replicaBrokers.addAll(allBrokers);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isValidNewLeader(Broker broker)
|
||||
{
|
||||
// broker is considered valid new leader if it is not the same as old leaderBroker
|
||||
return !(leaderBroker.host().equalsIgnoreCase(broker.host()) && leaderBroker.port() == broker.port());
|
||||
}
|
||||
|
||||
private void ensureNotInterrupted(Exception e) throws InterruptedException
|
||||
{
|
||||
if (Thread.interrupted()) {
|
||||
log.error(e, "Interrupted during fetching for %s - %s", topic, partitionId);
|
||||
throw new InterruptedException();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,16 +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.
|
||||
|
||||
org.apache.druid.firehose.kafka.KafkaEightSimpleConsumerDruidModule
|
|
@ -1,83 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ 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>
|
||||
|
||||
<groupId>org.apache.druid.extensions.contrib</groupId>
|
||||
<artifactId>druid-rabbitmq</artifactId>
|
||||
<name>druid-rabbitmq</name>
|
||||
<description>druid-rabbitmq</description>
|
||||
|
||||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.16.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-core</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.rabbitmq</groupId>
|
||||
<artifactId>amqp-client</artifactId>
|
||||
<version>3.2.1</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>net.jodah</groupId>
|
||||
<artifactId>lyra</artifactId>
|
||||
<version>0.3.1</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-api</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>com.rabbitmq</groupId>
|
||||
<artifactId>amqp-client</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!-- Tests -->
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-cli</groupId>
|
||||
<artifactId>commons-cli</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
</project>
|
|
@ -1,280 +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.druid.firehose.rabbitmq;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.rabbitmq.client.ConnectionFactory;
|
||||
import com.rabbitmq.client.LongString;
|
||||
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.security.KeyManagementException;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A Jacksonified version of the RabbitMQ ConnectionFactory for better integration
|
||||
* into the realtime.spec configuration file format.
|
||||
*/
|
||||
public class JacksonifiedConnectionFactory extends ConnectionFactory
|
||||
{
|
||||
public static JacksonifiedConnectionFactory makeDefaultConnectionFactory() throws Exception
|
||||
{
|
||||
return new JacksonifiedConnectionFactory(null, 0, null, null, null, null, 0, 0, 0, 0, null);
|
||||
}
|
||||
|
||||
private static Map<String, Object> getSerializableClientProperties(final Map<String, Object> clientProperties)
|
||||
{
|
||||
return Maps.transformEntries(
|
||||
clientProperties,
|
||||
new Maps.EntryTransformer<String, Object, Object>()
|
||||
{
|
||||
@Override
|
||||
public Object transformEntry(String key, Object value)
|
||||
{
|
||||
if (value instanceof LongString) {
|
||||
return value.toString();
|
||||
}
|
||||
return value;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private final String host;
|
||||
private final int port;
|
||||
private final String username;
|
||||
private final String password;
|
||||
private final String virtualHost;
|
||||
private final String uri;
|
||||
private final int requestedChannelMax;
|
||||
private final int requestedFrameMax;
|
||||
private final int requestedHeartbeat;
|
||||
private final int connectionTimeout;
|
||||
private final Map<String, Object> clientProperties;
|
||||
|
||||
@JsonCreator
|
||||
public JacksonifiedConnectionFactory(
|
||||
@JsonProperty("host") String host,
|
||||
@JsonProperty("port") int port,
|
||||
@JsonProperty("username") String username,
|
||||
@JsonProperty("password") String password,
|
||||
@JsonProperty("virtualHost") String virtualHost,
|
||||
@JsonProperty("uri") String uri,
|
||||
@JsonProperty("requestedChannelMax") int requestedChannelMax,
|
||||
@JsonProperty("requestedFrameMax") int requestedFrameMax,
|
||||
@JsonProperty("requestedHeartbeat") int requestedHeartbeat,
|
||||
@JsonProperty("connectionTimeout") int connectionTimeout,
|
||||
@JsonProperty("clientProperties") Map<String, Object> clientProperties
|
||||
) throws Exception
|
||||
{
|
||||
super();
|
||||
|
||||
this.host = host == null ? super.getHost() : host;
|
||||
this.port = port == 0 ? super.getPort() : port;
|
||||
this.username = username == null ? super.getUsername() : username;
|
||||
this.password = password == null ? super.getPassword() : password;
|
||||
this.virtualHost = virtualHost == null ? super.getVirtualHost() : virtualHost;
|
||||
this.uri = uri;
|
||||
this.requestedChannelMax = requestedChannelMax == 0 ? super.getRequestedChannelMax() : requestedChannelMax;
|
||||
this.requestedFrameMax = requestedFrameMax == 0 ? super.getRequestedFrameMax() : requestedFrameMax;
|
||||
this.requestedHeartbeat = requestedHeartbeat == 0 ? super.getRequestedHeartbeat() : requestedHeartbeat;
|
||||
this.connectionTimeout = connectionTimeout == 0 ? super.getConnectionTimeout() : connectionTimeout;
|
||||
this.clientProperties = clientProperties == null ? super.getClientProperties() : clientProperties;
|
||||
|
||||
super.setHost(this.host);
|
||||
super.setPort(this.port);
|
||||
super.setUsername(this.username);
|
||||
super.setPassword(this.password);
|
||||
super.setVirtualHost(this.virtualHost);
|
||||
if (this.uri != null) {
|
||||
super.setUri(this.uri);
|
||||
}
|
||||
super.setRequestedChannelMax(this.requestedChannelMax);
|
||||
super.setRequestedFrameMax(this.requestedFrameMax);
|
||||
super.setRequestedHeartbeat(this.requestedHeartbeat);
|
||||
super.setConnectionTimeout(this.connectionTimeout);
|
||||
super.setClientProperties(this.clientProperties);
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getHost()
|
||||
{
|
||||
return host;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public int getPort()
|
||||
{
|
||||
return port;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getUsername()
|
||||
{
|
||||
return username;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getPassword()
|
||||
{
|
||||
return password;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getVirtualHost()
|
||||
{
|
||||
return virtualHost;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getUri()
|
||||
{
|
||||
return uri;
|
||||
}
|
||||
|
||||
// we are only overriding this to help Jackson not be confused about the two setURI methods
|
||||
@JsonIgnore
|
||||
@Override
|
||||
public void setUri(URI uri) throws URISyntaxException, NoSuchAlgorithmException, KeyManagementException
|
||||
{
|
||||
super.setUri(uri);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setUri(String uriString) throws URISyntaxException, NoSuchAlgorithmException, KeyManagementException
|
||||
{
|
||||
super.setUri(uriString);
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public int getRequestedChannelMax()
|
||||
{
|
||||
return requestedChannelMax;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public int getRequestedFrameMax()
|
||||
{
|
||||
return requestedFrameMax;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public int getRequestedHeartbeat()
|
||||
{
|
||||
return requestedHeartbeat;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public int getConnectionTimeout()
|
||||
{
|
||||
return connectionTimeout;
|
||||
}
|
||||
|
||||
@JsonProperty("clientProperties")
|
||||
public Map<String, Object> getSerializableClientProperties()
|
||||
{
|
||||
return getSerializableClientProperties(clientProperties);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
JacksonifiedConnectionFactory that = (JacksonifiedConnectionFactory) o;
|
||||
|
||||
if (connectionTimeout != that.connectionTimeout) {
|
||||
return false;
|
||||
}
|
||||
if (port != that.port) {
|
||||
return false;
|
||||
}
|
||||
if (requestedChannelMax != that.requestedChannelMax) {
|
||||
return false;
|
||||
}
|
||||
if (requestedFrameMax != that.requestedFrameMax) {
|
||||
return false;
|
||||
}
|
||||
if (requestedHeartbeat != that.requestedHeartbeat) {
|
||||
return false;
|
||||
}
|
||||
if (clientProperties != null
|
||||
? !Maps.difference(
|
||||
getSerializableClientProperties(clientProperties),
|
||||
getSerializableClientProperties(that.clientProperties)
|
||||
).areEqual()
|
||||
: that.clientProperties != null) {
|
||||
return false;
|
||||
}
|
||||
if (host != null ? !host.equals(that.host) : that.host != null) {
|
||||
return false;
|
||||
}
|
||||
if (password != null ? !password.equals(that.password) : that.password != null) {
|
||||
return false;
|
||||
}
|
||||
if (uri != null ? !uri.equals(that.uri) : that.uri != null) {
|
||||
return false;
|
||||
}
|
||||
if (username != null ? !username.equals(that.username) : that.username != null) {
|
||||
return false;
|
||||
}
|
||||
if (virtualHost != null ? !virtualHost.equals(that.virtualHost) : that.virtualHost != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = host != null ? host.hashCode() : 0;
|
||||
result = 31 * result + port;
|
||||
result = 31 * result + (username != null ? username.hashCode() : 0);
|
||||
result = 31 * result + (password != null ? password.hashCode() : 0);
|
||||
result = 31 * result + (virtualHost != null ? virtualHost.hashCode() : 0);
|
||||
result = 31 * result + (uri != null ? uri.hashCode() : 0);
|
||||
result = 31 * result + requestedChannelMax;
|
||||
result = 31 * result + requestedFrameMax;
|
||||
result = 31 * result + requestedHeartbeat;
|
||||
result = 31 * result + connectionTimeout;
|
||||
result = 31 * result + (clientProperties != null ? clientProperties.hashCode() : 0);
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -1,51 +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.druid.firehose.rabbitmq;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class RabbitMQDruidModule implements DruidModule
|
||||
{
|
||||
@Override
|
||||
public List<? extends Module> getJacksonModules()
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new SimpleModule("RabbitMQFirehoseModule")
|
||||
.registerSubtypes(
|
||||
new NamedType(RabbitMQFirehoseFactory.class, "rabbitmq")
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
|
||||
}
|
||||
}
|
|
@ -1,186 +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.druid.firehose.rabbitmq;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
/**
|
||||
* A configuration object for a RabbitMQ connection.
|
||||
*/
|
||||
public class RabbitMQFirehoseConfig
|
||||
{
|
||||
// Lyra (auto reconnect) properties
|
||||
private static final int defaultMaxRetries = 100;
|
||||
private static final int defaultRetryIntervalSeconds = 2;
|
||||
private static final long defaultMaxDurationSeconds = 5 * 60;
|
||||
|
||||
public static RabbitMQFirehoseConfig makeDefaultConfig()
|
||||
{
|
||||
return new RabbitMQFirehoseConfig(null, null, null, false, false, false, 0, 0, 0);
|
||||
}
|
||||
|
||||
private final String queue;
|
||||
private final String exchange;
|
||||
private final String routingKey;
|
||||
private final boolean durable;
|
||||
private final boolean exclusive;
|
||||
private final boolean autoDelete;
|
||||
private final int maxRetries;
|
||||
private final int retryIntervalSeconds;
|
||||
private final long maxDurationSeconds;
|
||||
|
||||
@JsonCreator
|
||||
public RabbitMQFirehoseConfig(
|
||||
@JsonProperty("queue") String queue,
|
||||
@JsonProperty("exchange") String exchange,
|
||||
@JsonProperty("routingKey") String routingKey,
|
||||
@JsonProperty("durable") boolean durable,
|
||||
@JsonProperty("exclusive") boolean exclusive,
|
||||
@JsonProperty("autoDelete") boolean autoDelete,
|
||||
@JsonProperty("maxRetries") int maxRetries,
|
||||
@JsonProperty("retryIntervalSeconds") int retryIntervalSeconds,
|
||||
@JsonProperty("maxDurationSeconds") long maxDurationSeconds
|
||||
)
|
||||
{
|
||||
this.queue = queue;
|
||||
this.exchange = exchange;
|
||||
this.routingKey = routingKey;
|
||||
this.durable = durable;
|
||||
this.exclusive = exclusive;
|
||||
this.autoDelete = autoDelete;
|
||||
|
||||
this.maxRetries = maxRetries == 0 ? defaultMaxRetries : maxRetries;
|
||||
this.retryIntervalSeconds = retryIntervalSeconds == 0 ? defaultRetryIntervalSeconds : retryIntervalSeconds;
|
||||
this.maxDurationSeconds = maxDurationSeconds == 0 ? defaultMaxDurationSeconds : maxDurationSeconds;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getQueue()
|
||||
{
|
||||
return queue;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getExchange()
|
||||
{
|
||||
return exchange;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getRoutingKey()
|
||||
{
|
||||
return routingKey;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isDurable()
|
||||
{
|
||||
return durable;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isExclusive()
|
||||
{
|
||||
return exclusive;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isAutoDelete()
|
||||
{
|
||||
return autoDelete;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getMaxRetries()
|
||||
{
|
||||
return maxRetries;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getRetryIntervalSeconds()
|
||||
{
|
||||
return retryIntervalSeconds;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getMaxDurationSeconds()
|
||||
{
|
||||
return maxDurationSeconds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
RabbitMQFirehoseConfig that = (RabbitMQFirehoseConfig) o;
|
||||
|
||||
if (autoDelete != that.autoDelete) {
|
||||
return false;
|
||||
}
|
||||
if (durable != that.durable) {
|
||||
return false;
|
||||
}
|
||||
if (exclusive != that.exclusive) {
|
||||
return false;
|
||||
}
|
||||
if (maxDurationSeconds != that.maxDurationSeconds) {
|
||||
return false;
|
||||
}
|
||||
if (maxRetries != that.maxRetries) {
|
||||
return false;
|
||||
}
|
||||
if (retryIntervalSeconds != that.retryIntervalSeconds) {
|
||||
return false;
|
||||
}
|
||||
if (exchange != null ? !exchange.equals(that.exchange) : that.exchange != null) {
|
||||
return false;
|
||||
}
|
||||
if (queue != null ? !queue.equals(that.queue) : that.queue != null) {
|
||||
return false;
|
||||
}
|
||||
if (routingKey != null ? !routingKey.equals(that.routingKey) : that.routingKey != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = queue != null ? queue.hashCode() : 0;
|
||||
result = 31 * result + (exchange != null ? exchange.hashCode() : 0);
|
||||
result = 31 * result + (routingKey != null ? routingKey.hashCode() : 0);
|
||||
result = 31 * result + (durable ? 1 : 0);
|
||||
result = 31 * result + (exclusive ? 1 : 0);
|
||||
result = 31 * result + (autoDelete ? 1 : 0);
|
||||
result = 31 * result + maxRetries;
|
||||
result = 31 * result + retryIntervalSeconds;
|
||||
result = 31 * result + (int) (maxDurationSeconds ^ (maxDurationSeconds >>> 32));
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -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.druid.firehose.rabbitmq;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.rabbitmq.client.AMQP;
|
||||
import com.rabbitmq.client.Channel;
|
||||
import com.rabbitmq.client.Connection;
|
||||
import com.rabbitmq.client.ConsumerCancelledException;
|
||||
import com.rabbitmq.client.DefaultConsumer;
|
||||
import com.rabbitmq.client.Envelope;
|
||||
import com.rabbitmq.client.QueueingConsumer.Delivery;
|
||||
import com.rabbitmq.client.ShutdownListener;
|
||||
import com.rabbitmq.client.ShutdownSignalException;
|
||||
import net.jodah.lyra.ConnectionOptions;
|
||||
import net.jodah.lyra.Connections;
|
||||
import net.jodah.lyra.config.Config;
|
||||
import net.jodah.lyra.retry.RetryPolicy;
|
||||
import net.jodah.lyra.util.Duration;
|
||||
import org.apache.druid.data.input.Firehose;
|
||||
import org.apache.druid.data.input.FirehoseFactory;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
||||
/**
|
||||
* A FirehoseFactory for RabbitMQ.
|
||||
* <p/>
|
||||
* It will receive it's configuration through the realtime.spec file and expects to find a
|
||||
* consumerProps element in the firehose definition with values for a number of configuration options.
|
||||
* Below is a complete example for a RabbitMQ firehose configuration with some explanation. Options
|
||||
* that have defaults can be skipped but options with no defaults must be specified with the exception
|
||||
* of the URI property. If the URI property is set, it will override any other property that was also
|
||||
* set.
|
||||
* <p/>
|
||||
* File: <em>realtime.spec</em>
|
||||
* <pre>
|
||||
* "firehose" : {
|
||||
* "type" : "rabbitmq",
|
||||
* "connection" : {
|
||||
* "host": "localhost", # The hostname of the RabbitMQ broker to connect to. Default: 'localhost'
|
||||
* "port": "5672", # The port number to connect to on the RabbitMQ broker. Default: '5672'
|
||||
* "username": "test-dude", # The username to use to connect to RabbitMQ. Default: 'guest'
|
||||
* "password": "test-word", # The password to use to connect to RabbitMQ. Default: 'guest'
|
||||
* "virtualHost": "test-vhost", # The virtual host to connect to. Default: '/'
|
||||
* "uri": "amqp://mqserver:1234/vhost", # The URI string to use to connect to RabbitMQ. No default and not needed
|
||||
* },
|
||||
* "config" : {
|
||||
* "exchange": "test-exchange", # The exchange to connect to. No default
|
||||
* "queue" : "druidtest", # The queue to connect to or create. No default
|
||||
* "routingKey": "#", # The routing key to use to bind the queue to the exchange. No default
|
||||
* "durable": "true", # Whether the queue should be durable. Default: 'false'
|
||||
* "exclusive": "false", # Whether the queue should be exclusive. Default: 'false'
|
||||
* "autoDelete": "false", # Whether the queue should auto-delete on disconnect. Default: 'false'
|
||||
*
|
||||
* "maxRetries": "10", # The max number of reconnection retry attempts
|
||||
* "retryIntervalSeconds": "1", # The reconnection interval
|
||||
* "maxDurationSeconds": "300" # The max duration of trying to reconnect
|
||||
* },
|
||||
* "parser" : {
|
||||
* "timestampSpec" : { "column" : "utcdt", "format" : "iso" },
|
||||
* "data" : { "format" : "json" },
|
||||
* "dimensionExclusions" : ["wp"]
|
||||
* }
|
||||
* },
|
||||
* </pre>
|
||||
* <p/>
|
||||
* <b>Limitations:</b> This implementation will not attempt to reconnect to the MQ broker if the
|
||||
* connection to it is lost. Furthermore it does not support any automatic failover on high availability
|
||||
* RabbitMQ clusters. This is not supported by the underlying AMQP client library and while the behavior
|
||||
* could be "faked" to some extent we haven't implemented that yet. However, if a policy is defined in
|
||||
* the RabbitMQ cluster that sets the "ha-mode" and "ha-sync-mode" properly on the queue that this
|
||||
* Firehose connects to, messages should survive an MQ broker node failure and be delivered once a
|
||||
* connection to another node is set up.
|
||||
* <p/>
|
||||
* For more information on RabbitMQ high availability please see:
|
||||
* <a href="http://www.rabbitmq.com/ha.html">http://www.rabbitmq.com/ha.html</a>.
|
||||
*/
|
||||
public class RabbitMQFirehoseFactory implements FirehoseFactory<InputRowParser<ByteBuffer>>
|
||||
{
|
||||
private static final Logger log = new Logger(RabbitMQFirehoseFactory.class);
|
||||
|
||||
private final RabbitMQFirehoseConfig config;
|
||||
private final JacksonifiedConnectionFactory connectionFactory;
|
||||
|
||||
@JsonCreator
|
||||
public RabbitMQFirehoseFactory(
|
||||
@JsonProperty("connection") JacksonifiedConnectionFactory connectionFactory,
|
||||
@JsonProperty("config") RabbitMQFirehoseConfig config,
|
||||
// See https://github.com/apache/incubator-druid/pull/1922
|
||||
@JsonProperty("connectionFactory") JacksonifiedConnectionFactory connectionFactoryCOMPAT
|
||||
) throws Exception
|
||||
{
|
||||
this.connectionFactory = connectionFactory == null
|
||||
? connectionFactoryCOMPAT == null
|
||||
? JacksonifiedConnectionFactory.makeDefaultConnectionFactory()
|
||||
: connectionFactoryCOMPAT
|
||||
: connectionFactory;
|
||||
this.config = config == null ? RabbitMQFirehoseConfig.makeDefaultConfig() : config;
|
||||
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public RabbitMQFirehoseConfig getConfig()
|
||||
{
|
||||
return config;
|
||||
}
|
||||
|
||||
@JsonProperty("connection")
|
||||
public JacksonifiedConnectionFactory getConnectionFactory()
|
||||
{
|
||||
return connectionFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect(final InputRowParser<ByteBuffer> firehoseParser, File temporaryDirectory) throws IOException
|
||||
{
|
||||
ConnectionOptions lyraOptions = new ConnectionOptions(this.connectionFactory);
|
||||
Config lyraConfig = new Config()
|
||||
.withRecoveryPolicy(
|
||||
new RetryPolicy()
|
||||
.withMaxRetries(config.getMaxRetries())
|
||||
.withRetryInterval(Duration.seconds(config.getRetryIntervalSeconds()))
|
||||
.withMaxDuration(Duration.seconds(config.getMaxDurationSeconds()))
|
||||
);
|
||||
|
||||
String queue = config.getQueue();
|
||||
String exchange = config.getExchange();
|
||||
String routingKey = config.getRoutingKey();
|
||||
|
||||
boolean durable = config.isDurable();
|
||||
boolean exclusive = config.isExclusive();
|
||||
boolean autoDelete = config.isAutoDelete();
|
||||
|
||||
final Connection connection = Connections.create(lyraOptions, lyraConfig);
|
||||
|
||||
connection.addShutdownListener(
|
||||
new ShutdownListener()
|
||||
{
|
||||
@Override
|
||||
public void shutdownCompleted(ShutdownSignalException cause)
|
||||
{
|
||||
log.warn(cause, "Connection closed!");
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
final Channel channel = connection.createChannel();
|
||||
channel.queueDeclare(queue, durable, exclusive, autoDelete, null);
|
||||
channel.queueBind(queue, exchange, routingKey);
|
||||
channel.addShutdownListener(
|
||||
new ShutdownListener()
|
||||
{
|
||||
@Override
|
||||
public void shutdownCompleted(ShutdownSignalException cause)
|
||||
{
|
||||
log.warn(cause, "Channel closed!");
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
// We create a QueueingConsumer that will not auto-acknowledge messages since that
|
||||
// happens on commit().
|
||||
final QueueingConsumer consumer = new QueueingConsumer(channel);
|
||||
channel.basicConsume(queue, false, consumer);
|
||||
|
||||
return new Firehose()
|
||||
{
|
||||
/**
|
||||
* Storing the latest row as a member variable should be safe since this will only be run
|
||||
* by a single thread.
|
||||
*/
|
||||
private InputRow nextRow;
|
||||
|
||||
/**
|
||||
* Store the latest delivery tag to be able to commit (acknowledge) the message delivery up to
|
||||
* and including this tag. See commit() for more detail.
|
||||
*/
|
||||
private long lastDeliveryTag;
|
||||
|
||||
private Iterator<InputRow> nextIterator = Collections.emptyIterator();
|
||||
|
||||
@Override
|
||||
public boolean hasMore()
|
||||
{
|
||||
nextRow = null;
|
||||
try {
|
||||
if (nextIterator.hasNext()) {
|
||||
nextRow = nextIterator.next();
|
||||
return true;
|
||||
}
|
||||
// Wait for the next delivery. This will block until something is available.
|
||||
final Delivery delivery = consumer.nextDelivery();
|
||||
if (delivery != null) {
|
||||
lastDeliveryTag = delivery.getEnvelope().getDeliveryTag();
|
||||
nextIterator = firehoseParser.parseBatch(ByteBuffer.wrap(delivery.getBody())).iterator();
|
||||
if (nextIterator.hasNext()) {
|
||||
nextRow = nextIterator.next();
|
||||
// If delivery is non-null, we report that there is something more to process.
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
// A little unclear on how we should handle this.
|
||||
|
||||
// At any rate, we're in an unknown state now so let's log something and return false.
|
||||
log.wtf(e, "Got interrupted while waiting for next delivery. Doubt this should ever happen.");
|
||||
}
|
||||
|
||||
// This means that delivery is null or we caught the exception above so we report that we have
|
||||
// nothing more to process.
|
||||
return false;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public InputRow nextRow()
|
||||
{
|
||||
if (nextRow == null) {
|
||||
//Just making sure.
|
||||
log.wtf("I have nothing in delivery. Method hasMore() should have returned false.");
|
||||
return null;
|
||||
}
|
||||
|
||||
return nextRow;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Runnable commit()
|
||||
{
|
||||
// This method will be called from the same thread that calls the other methods of
|
||||
// this Firehose. However, the returned Runnable will be called by a different thread.
|
||||
//
|
||||
// It should be (thread) safe to copy the lastDeliveryTag like we do below and then
|
||||
// acknowledge values up to and including that value.
|
||||
return new Runnable()
|
||||
{
|
||||
// Store (copy) the last delivery tag to "become" thread safe.
|
||||
final long deliveryTag = lastDeliveryTag;
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
try {
|
||||
log.info("Acknowledging delivery of messages up to tag: " + deliveryTag);
|
||||
|
||||
// Acknowledge all messages up to and including the stored delivery tag.
|
||||
channel.basicAck(deliveryTag, true);
|
||||
}
|
||||
catch (IOException e) {
|
||||
log.error(e, "Unable to acknowledge message reception to message queue.");
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
log.info("Closing connection to RabbitMQ");
|
||||
channel.close();
|
||||
connection.close();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private static class QueueingConsumer extends DefaultConsumer
|
||||
{
|
||||
private final BlockingQueue<Delivery> _queue;
|
||||
|
||||
public QueueingConsumer(Channel ch)
|
||||
{
|
||||
this(ch, new LinkedBlockingQueue<Delivery>());
|
||||
}
|
||||
|
||||
public QueueingConsumer(Channel ch, BlockingQueue<Delivery> q)
|
||||
{
|
||||
super(ch);
|
||||
this._queue = q;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handleShutdownSignal(String consumerTag, ShutdownSignalException sig)
|
||||
{
|
||||
_queue.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handleCancel(String consumerTag)
|
||||
{
|
||||
_queue.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handleDelivery(
|
||||
String consumerTag,
|
||||
Envelope envelope,
|
||||
AMQP.BasicProperties properties,
|
||||
byte[] body
|
||||
)
|
||||
{
|
||||
this._queue.add(new Delivery(envelope, properties, body));
|
||||
}
|
||||
|
||||
public Delivery nextDelivery()
|
||||
throws InterruptedException, ShutdownSignalException, ConsumerCancelledException
|
||||
{
|
||||
return _queue.take();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,16 +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.
|
||||
|
||||
org.apache.druid.firehose.rabbitmq.RabbitMQDruidModule
|
|
@ -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.druid.examples.rabbitmq;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.rabbitmq.client.ConnectionFactory;
|
||||
import org.apache.druid.firehose.rabbitmq.JacksonifiedConnectionFactory;
|
||||
import org.apache.druid.firehose.rabbitmq.RabbitMQFirehoseConfig;
|
||||
import org.apache.druid.firehose.rabbitmq.RabbitMQFirehoseFactory;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class RabbitMQFirehoseFactoryTest
|
||||
{
|
||||
private static final ObjectMapper mapper = new DefaultObjectMapper();
|
||||
|
||||
@Test
|
||||
public void testSerde() throws Exception
|
||||
{
|
||||
RabbitMQFirehoseConfig config = new RabbitMQFirehoseConfig(
|
||||
"test",
|
||||
"test2",
|
||||
"test3",
|
||||
true,
|
||||
true,
|
||||
true,
|
||||
5,
|
||||
10,
|
||||
20
|
||||
);
|
||||
|
||||
JacksonifiedConnectionFactory connectionFactory = new JacksonifiedConnectionFactory(
|
||||
"foo",
|
||||
9978,
|
||||
"user",
|
||||
"pw",
|
||||
"host",
|
||||
null,
|
||||
5,
|
||||
10,
|
||||
11,
|
||||
12,
|
||||
ImmutableMap.of("hi", "bye")
|
||||
);
|
||||
|
||||
RabbitMQFirehoseFactory factory = new RabbitMQFirehoseFactory(
|
||||
connectionFactory,
|
||||
config,
|
||||
null
|
||||
);
|
||||
|
||||
byte[] bytes = mapper.writeValueAsBytes(factory);
|
||||
RabbitMQFirehoseFactory factory2 = mapper.readValue(bytes, RabbitMQFirehoseFactory.class);
|
||||
byte[] bytes2 = mapper.writeValueAsBytes(factory2);
|
||||
|
||||
Assert.assertArrayEquals(bytes, bytes2);
|
||||
|
||||
Assert.assertEquals(factory.getConfig(), factory2.getConfig());
|
||||
Assert.assertEquals(factory.getConnectionFactory(), factory2.getConnectionFactory());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDefaultSerde() throws Exception
|
||||
{
|
||||
RabbitMQFirehoseConfig config = RabbitMQFirehoseConfig.makeDefaultConfig();
|
||||
|
||||
JacksonifiedConnectionFactory connectionFactory = JacksonifiedConnectionFactory.makeDefaultConnectionFactory();
|
||||
|
||||
RabbitMQFirehoseFactory factory = new RabbitMQFirehoseFactory(
|
||||
connectionFactory,
|
||||
config,
|
||||
null
|
||||
);
|
||||
|
||||
byte[] bytes = mapper.writeValueAsBytes(factory);
|
||||
RabbitMQFirehoseFactory factory2 = mapper.readValue(bytes, RabbitMQFirehoseFactory.class);
|
||||
byte[] bytes2 = mapper.writeValueAsBytes(factory2);
|
||||
|
||||
Assert.assertArrayEquals(bytes, bytes2);
|
||||
|
||||
Assert.assertEquals(factory.getConfig(), factory2.getConfig());
|
||||
Assert.assertEquals(factory.getConnectionFactory(), factory2.getConnectionFactory());
|
||||
|
||||
Assert.assertEquals(300, factory2.getConfig().getMaxDurationSeconds());
|
||||
|
||||
Assert.assertEquals(ConnectionFactory.DEFAULT_HOST, factory2.getConnectionFactory().getHost());
|
||||
Assert.assertEquals(ConnectionFactory.DEFAULT_USER, factory2.getConnectionFactory().getUsername());
|
||||
Assert.assertEquals(ConnectionFactory.DEFAULT_AMQP_PORT, factory2.getConnectionFactory().getPort());
|
||||
}
|
||||
}
|
|
@ -1,212 +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.druid.examples.rabbitmq;
|
||||
|
||||
import com.rabbitmq.client.Channel;
|
||||
import com.rabbitmq.client.Connection;
|
||||
import com.rabbitmq.client.ConnectionFactory;
|
||||
import org.apache.commons.cli.BasicParser;
|
||||
import org.apache.commons.cli.CommandLine;
|
||||
import org.apache.commons.cli.HelpFormatter;
|
||||
import org.apache.commons.cli.Option;
|
||||
import org.apache.commons.cli.OptionBuilder;
|
||||
import org.apache.commons.cli.Options;
|
||||
import org.apache.commons.cli.ParseException;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Calendar;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Random;
|
||||
import java.util.TimeZone;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class RabbitMQProducerMain
|
||||
{
|
||||
public static void main(String[] args)
|
||||
throws Exception
|
||||
{
|
||||
// We use a List to keep track of option insertion order. See below.
|
||||
final List<Option> optionList = new ArrayList<Option>();
|
||||
|
||||
optionList.add(OptionBuilder.withLongOpt("help")
|
||||
.withDescription("display this help message")
|
||||
.create("h"));
|
||||
optionList.add(OptionBuilder.withLongOpt("hostname")
|
||||
.hasArg()
|
||||
.withDescription("the hostname of the AMQP broker [defaults to AMQP library default]")
|
||||
.create("b"));
|
||||
optionList.add(OptionBuilder.withLongOpt("port")
|
||||
.hasArg()
|
||||
.withDescription("the port of the AMQP broker [defaults to AMQP library default]")
|
||||
.create("n"));
|
||||
optionList.add(OptionBuilder.withLongOpt("username")
|
||||
.hasArg()
|
||||
.withDescription("username to connect to the AMQP broker [defaults to AMQP library default]")
|
||||
.create("u"));
|
||||
optionList.add(OptionBuilder.withLongOpt("password")
|
||||
.hasArg()
|
||||
.withDescription("password to connect to the AMQP broker [defaults to AMQP library default]")
|
||||
.create("p"));
|
||||
optionList.add(OptionBuilder.withLongOpt("vhost")
|
||||
.hasArg()
|
||||
.withDescription("name of virtual host on the AMQP broker [defaults to AMQP library default]")
|
||||
.create("v"));
|
||||
optionList.add(OptionBuilder.withLongOpt("exchange")
|
||||
.isRequired()
|
||||
.hasArg()
|
||||
.withDescription("name of the AMQP exchange [required - no default]")
|
||||
.create("e"));
|
||||
optionList.add(OptionBuilder.withLongOpt("key")
|
||||
.hasArg()
|
||||
.withDescription("the routing key to use when sending messages [default: 'default.routing.key']")
|
||||
.create("k"));
|
||||
optionList.add(OptionBuilder.withLongOpt("type")
|
||||
.hasArg()
|
||||
.withDescription("the type of exchange to create [default: 'topic']")
|
||||
.create("t"));
|
||||
optionList.add(OptionBuilder.withLongOpt("durable")
|
||||
.withDescription("if set, a durable exchange will be declared [default: not set]")
|
||||
.create("d"));
|
||||
optionList.add(OptionBuilder.withLongOpt("autodelete")
|
||||
.withDescription("if set, an auto-delete exchange will be declared [default: not set]")
|
||||
.create("a"));
|
||||
optionList.add(OptionBuilder.withLongOpt("single")
|
||||
.withDescription("if set, only a single message will be sent [default: not set]")
|
||||
.create("s"));
|
||||
optionList.add(OptionBuilder.withLongOpt("start")
|
||||
.hasArg()
|
||||
.withDescription("time to use to start sending messages from [default: 2010-01-01T00:00:00]")
|
||||
.create());
|
||||
optionList.add(OptionBuilder.withLongOpt("stop")
|
||||
.hasArg()
|
||||
.withDescription("time to use to send messages until (format: '2013-07-18T23:45:59') [default: current time]")
|
||||
.create());
|
||||
optionList.add(OptionBuilder.withLongOpt("interval")
|
||||
.hasArg()
|
||||
.withDescription("the interval to add to the timestamp between messages in seconds [default: 10]")
|
||||
.create());
|
||||
optionList.add(OptionBuilder.withLongOpt("delay")
|
||||
.hasArg()
|
||||
.withDescription("the delay between sending messages in milliseconds [default: 100]")
|
||||
.create());
|
||||
|
||||
// An extremely silly hack to maintain the above order in the help formatting.
|
||||
HelpFormatter formatter = new HelpFormatter();
|
||||
// Add a comparator to the HelpFormatter using the ArrayList above to sort by insertion order.
|
||||
//noinspection ComparatorCombinators -- don't replace with comparingInt() to preserve comments
|
||||
formatter.setOptionComparator((o1, o2) -> {
|
||||
// I know this isn't fast, but who cares! The list is short.
|
||||
//noinspection SuspiciousMethodCalls
|
||||
return Integer.compare(optionList.indexOf(o1), optionList.indexOf(o2));
|
||||
});
|
||||
|
||||
// Now we can add all the options to an Options instance. This is dumb!
|
||||
Options options = new Options();
|
||||
for (Option option : optionList) {
|
||||
options.addOption(option);
|
||||
}
|
||||
|
||||
CommandLine cmd = null;
|
||||
|
||||
try {
|
||||
cmd = new BasicParser().parse(options, args);
|
||||
}
|
||||
catch (ParseException e) {
|
||||
formatter.printHelp("RabbitMQProducerMain", e.getMessage(), options, null);
|
||||
System.exit(1);
|
||||
}
|
||||
|
||||
if (cmd.hasOption("h")) {
|
||||
formatter.printHelp("RabbitMQProducerMain", options);
|
||||
System.exit(2);
|
||||
}
|
||||
|
||||
ConnectionFactory factory = new ConnectionFactory();
|
||||
|
||||
if (cmd.hasOption("b")) {
|
||||
factory.setHost(cmd.getOptionValue("b"));
|
||||
}
|
||||
if (cmd.hasOption("u")) {
|
||||
factory.setUsername(cmd.getOptionValue("u"));
|
||||
}
|
||||
if (cmd.hasOption("p")) {
|
||||
factory.setPassword(cmd.getOptionValue("p"));
|
||||
}
|
||||
if (cmd.hasOption("v")) {
|
||||
factory.setVirtualHost(cmd.getOptionValue("v"));
|
||||
}
|
||||
if (cmd.hasOption("n")) {
|
||||
factory.setPort(Integer.parseInt(cmd.getOptionValue("n")));
|
||||
}
|
||||
|
||||
String exchange = cmd.getOptionValue("e");
|
||||
String routingKey = "default.routing.key";
|
||||
if (cmd.hasOption("k")) {
|
||||
routingKey = cmd.getOptionValue("k");
|
||||
}
|
||||
|
||||
boolean durable = cmd.hasOption("d");
|
||||
boolean autoDelete = cmd.hasOption("a");
|
||||
String type = cmd.getOptionValue("t", "topic");
|
||||
boolean single = cmd.hasOption("single");
|
||||
int interval = Integer.parseInt(cmd.getOptionValue("interval", "10"));
|
||||
int delay = Integer.parseInt(cmd.getOptionValue("delay", "100"));
|
||||
|
||||
SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss", Locale.ENGLISH);
|
||||
Date stop = sdf.parse(cmd.getOptionValue("stop", sdf.format(new Date())));
|
||||
|
||||
Random r = ThreadLocalRandom.current();
|
||||
Calendar timer = Calendar.getInstance(TimeZone.getTimeZone("UTC"), Locale.ENGLISH);
|
||||
timer.setTime(sdf.parse(cmd.getOptionValue("start", "2010-01-01T00:00:00")));
|
||||
|
||||
String msg_template = "{\"utcdt\": \"%s\", \"wp\": %d, \"gender\": \"%s\", \"age\": %d}";
|
||||
|
||||
Connection connection = factory.newConnection();
|
||||
Channel channel = connection.createChannel();
|
||||
|
||||
channel.exchangeDeclare(exchange, type, durable, autoDelete, null);
|
||||
|
||||
do {
|
||||
int wp = (10 + r.nextInt(90)) * 100;
|
||||
String gender = r.nextBoolean() ? "male" : "female";
|
||||
int age = 20 + r.nextInt(70);
|
||||
|
||||
String line = StringUtils.format(msg_template, sdf.format(timer.getTime()), wp, gender, age);
|
||||
|
||||
channel.basicPublish(exchange, routingKey, null, StringUtils.toUtf8(line));
|
||||
|
||||
System.out.println("Sent message: " + line);
|
||||
|
||||
timer.add(Calendar.SECOND, interval);
|
||||
|
||||
Thread.sleep(delay);
|
||||
} while ((!single && stop.after(timer.getTime())));
|
||||
|
||||
connection.close();
|
||||
}
|
||||
|
||||
}
|
|
@ -1,75 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ 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/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-kafka-eight</artifactId>
|
||||
<name>druid-kafka-eight</name>
|
||||
<description>druid-kafka-eight</description>
|
||||
|
||||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.16.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-core</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.kafka</groupId>
|
||||
<artifactId>kafka_2.10</artifactId>
|
||||
<version>0.8.2.1</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>log4j</groupId>
|
||||
<artifactId>log4j</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.apache.zookeeper</groupId>
|
||||
<artifactId>zookeeper</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-api</artifactId>
|
||||
</exclusion>
|
||||
<exclusion>
|
||||
<groupId>net.jpountz.lz4</groupId>
|
||||
<artifactId>lz4</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
<!-- Tests -->
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
</project>
|
|
@ -1,52 +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.druid.firehose.kafka;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
@Deprecated
|
||||
public class KafkaEightDruidModule implements DruidModule
|
||||
{
|
||||
@Override
|
||||
public List<? extends Module> getJacksonModules()
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new SimpleModule("KafkaEightFirehoseModule")
|
||||
.registerSubtypes(
|
||||
new NamedType(KafkaEightFirehoseFactory.class, "kafka-0.8")
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
|
||||
}
|
||||
}
|
|
@ -1,176 +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.druid.firehose.kafka;
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Sets;
|
||||
import kafka.consumer.Consumer;
|
||||
import kafka.consumer.ConsumerConfig;
|
||||
import kafka.consumer.ConsumerIterator;
|
||||
import kafka.consumer.KafkaStream;
|
||||
import kafka.javaapi.consumer.ConsumerConnector;
|
||||
import kafka.message.InvalidMessageException;
|
||||
import org.apache.druid.data.input.Firehose;
|
||||
import org.apache.druid.data.input.FirehoseFactory;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* This class is deprecated and kafka-eight module should be removed completely
|
||||
*/
|
||||
@Deprecated
|
||||
public class KafkaEightFirehoseFactory implements FirehoseFactory<InputRowParser<ByteBuffer>>
|
||||
{
|
||||
private static final Logger log = new Logger(KafkaEightFirehoseFactory.class);
|
||||
|
||||
@JsonProperty
|
||||
private final Properties consumerProps;
|
||||
|
||||
@JsonProperty
|
||||
private final String feed;
|
||||
|
||||
@JsonCreator
|
||||
public KafkaEightFirehoseFactory(
|
||||
@JsonProperty("consumerProps") Properties consumerProps,
|
||||
@JsonProperty("feed") String feed
|
||||
|
||||
)
|
||||
{
|
||||
this.consumerProps = consumerProps;
|
||||
this.feed = feed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Firehose connect(final InputRowParser<ByteBuffer> firehoseParser, File temporaryDirectory)
|
||||
{
|
||||
Set<String> newDimExclus = Sets.union(
|
||||
firehoseParser.getParseSpec().getDimensionsSpec().getDimensionExclusions(),
|
||||
Sets.newHashSet("feed")
|
||||
);
|
||||
|
||||
final InputRowParser<ByteBuffer> theParser = firehoseParser.withParseSpec(
|
||||
firehoseParser.getParseSpec()
|
||||
.withDimensionsSpec(
|
||||
firehoseParser.getParseSpec()
|
||||
.getDimensionsSpec()
|
||||
.withDimensionExclusions(
|
||||
newDimExclus
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
final ConsumerConnector connector = Consumer.createJavaConsumerConnector(new ConsumerConfig(consumerProps));
|
||||
|
||||
final Map<String, List<KafkaStream<byte[], byte[]>>> streams = connector.createMessageStreams(
|
||||
ImmutableMap.of(
|
||||
feed,
|
||||
1
|
||||
)
|
||||
);
|
||||
|
||||
final List<KafkaStream<byte[], byte[]>> streamList = streams.get(feed);
|
||||
if (streamList == null || streamList.size() != 1) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final KafkaStream<byte[], byte[]> stream = streamList.get(0);
|
||||
final ConsumerIterator<byte[], byte[]> iter = stream.iterator();
|
||||
|
||||
return new Firehose()
|
||||
{
|
||||
Iterator<InputRow> nextIterator = Collections.emptyIterator();
|
||||
|
||||
@Override
|
||||
public boolean hasMore()
|
||||
{
|
||||
return nextIterator.hasNext() || iter.hasNext();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public InputRow nextRow()
|
||||
{
|
||||
try {
|
||||
if (!nextIterator.hasNext()) {
|
||||
final byte[] message = iter.next().message();
|
||||
|
||||
if (message == null) {
|
||||
return null;
|
||||
}
|
||||
nextIterator = theParser.parseBatch(ByteBuffer.wrap(message)).iterator();
|
||||
}
|
||||
|
||||
return nextIterator.next();
|
||||
|
||||
}
|
||||
catch (InvalidMessageException e) {
|
||||
/*
|
||||
IF the CRC is caused within the wire transfer, this is not the best way to handel CRC.
|
||||
Probably it is better to shutdown the fireHose without commit and start it again.
|
||||
*/
|
||||
log.error(e, "Message failed its checksum and it is corrupt, will skip it");
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Runnable commit()
|
||||
{
|
||||
return new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
/*
|
||||
This is actually not going to do exactly what we want, cause it will be called asynchronously
|
||||
after the persist is complete. So, it's going to commit that it's processed more than was actually
|
||||
persisted. This is unfortunate, but good enough for now. Should revisit along with an upgrade
|
||||
of our Kafka version.
|
||||
*/
|
||||
|
||||
log.info("committing offsets");
|
||||
connector.commitOffsets();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
connector.shutdown();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
|
@ -1,16 +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.
|
||||
|
||||
org.apache.druid.firehose.kafka.KafkaEightDruidModule
|
|
@ -672,7 +672,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
null,
|
||||
new FireDepartment(
|
||||
dataSchema,
|
||||
new RealtimeIOConfig(null, null, null),
|
||||
new RealtimeIOConfig(null, null),
|
||||
null
|
||||
),
|
||||
null
|
||||
|
|
|
@ -637,7 +637,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
null,
|
||||
new FireDepartment(
|
||||
dataSchema,
|
||||
new RealtimeIOConfig(null, null, null),
|
||||
new RealtimeIOConfig(null, null),
|
||||
null
|
||||
),
|
||||
null
|
||||
|
|
|
@ -244,7 +244,7 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
|
|||
.withBasePersistDirectory(toolbox.getPersistDir());
|
||||
|
||||
final FireDepartment fireDepartmentForMetrics =
|
||||
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null, null), null);
|
||||
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null), null);
|
||||
|
||||
final TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build(
|
||||
this,
|
||||
|
|
|
@ -842,7 +842,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
|
|||
{
|
||||
final GranularitySpec granularitySpec = dataSchema.getGranularitySpec();
|
||||
final FireDepartment fireDepartmentForMetrics =
|
||||
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null, null), null);
|
||||
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null), null);
|
||||
buildSegmentsFireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
|
||||
|
||||
if (toolbox.getMonitorScheduler() != null) {
|
||||
|
|
|
@ -283,7 +283,7 @@ public class ParallelIndexSubTask extends AbstractTask
|
|||
final DataSchema dataSchema = ingestionSchema.getDataSchema();
|
||||
final GranularitySpec granularitySpec = dataSchema.getGranularitySpec();
|
||||
final FireDepartment fireDepartmentForMetrics =
|
||||
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null, null), null);
|
||||
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null), null);
|
||||
final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
|
||||
|
||||
if (toolbox.getMonitorScheduler() != null) {
|
||||
|
|
|
@ -353,7 +353,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
// Set up FireDepartmentMetrics
|
||||
final FireDepartment fireDepartmentForMetrics = new FireDepartment(
|
||||
task.getDataSchema(),
|
||||
new RealtimeIOConfig(null, null, null),
|
||||
new RealtimeIOConfig(null, null),
|
||||
null
|
||||
);
|
||||
FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
|
||||
|
|
|
@ -58,8 +58,7 @@ public class TestRealtimeTask extends RealtimeIndexTask
|
|||
new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, null, mapper),
|
||||
new RealtimeIOConfig(
|
||||
new LocalFirehoseFactory(new File("lol"), "rofl", null),
|
||||
(schema, config, metrics) -> null,
|
||||
null
|
||||
(schema, config, metrics) -> null
|
||||
),
|
||||
null
|
||||
),
|
||||
|
|
|
@ -1404,7 +1404,6 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
);
|
||||
RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig(
|
||||
new TestFirehoseFactory(),
|
||||
null,
|
||||
null
|
||||
);
|
||||
RealtimeAppenderatorTuningConfig tuningConfig = new RealtimeAppenderatorTuningConfig(
|
||||
|
|
|
@ -822,7 +822,6 @@ public class RealtimeIndexTaskTest
|
|||
);
|
||||
RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig(
|
||||
new TestFirehose.TestFirehoseFactory(),
|
||||
null,
|
||||
null
|
||||
);
|
||||
RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig(
|
||||
|
|
|
@ -378,8 +378,7 @@ public class TaskSerdeTest
|
|||
),
|
||||
new RealtimeIOConfig(
|
||||
new LocalFirehoseFactory(new File("lol"), "rofl", null),
|
||||
(schema, config, metrics) -> null,
|
||||
null
|
||||
(schema, config, metrics) -> null
|
||||
),
|
||||
|
||||
new RealtimeTuningConfig(
|
||||
|
|
|
@ -1275,9 +1275,8 @@ public class TaskLifecycleTest
|
|||
);
|
||||
RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig(
|
||||
new MockFirehoseFactory(true),
|
||||
null,
|
||||
// PlumberSchool - Realtime Index Task always uses RealtimePlumber which is hardcoded in RealtimeIndexTask class
|
||||
null
|
||||
// PlumberSchool - Realtime Index Task always uses RealtimePlumber which is hardcoded in RealtimeIndexTask class
|
||||
);
|
||||
RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig(
|
||||
1000,
|
||||
|
|
|
@ -57,8 +57,7 @@ public class TaskAnnouncementTest
|
|||
new DataSchema("foo", null, new AggregatorFactory[0], null, null, new DefaultObjectMapper()),
|
||||
new RealtimeIOConfig(
|
||||
new LocalFirehoseFactory(new File("lol"), "rofl", null),
|
||||
(schema, config, metrics) -> null,
|
||||
null
|
||||
(schema, config, metrics) -> null
|
||||
),
|
||||
null
|
||||
),
|
||||
|
|
6
pom.xml
6
pom.xml
|
@ -113,7 +113,6 @@
|
|||
|
||||
<modules>
|
||||
<module>core</module>
|
||||
<module>examples</module>
|
||||
<module>indexing-hadoop</module>
|
||||
<module>indexing-service</module>
|
||||
<module>processing</module>
|
||||
|
@ -136,7 +135,6 @@
|
|||
<module>extensions-core/hdfs-storage</module>
|
||||
<module>extensions-core/histogram</module>
|
||||
<module>extensions-core/stats</module>
|
||||
<module>extensions-core/kafka-eight</module>
|
||||
<module>extensions-core/kafka-extraction-namespace</module>
|
||||
<module>extensions-core/kafka-indexing-service</module>
|
||||
<module>extensions-core/kinesis-indexing-service</module>
|
||||
|
@ -156,11 +154,8 @@
|
|||
<module>extensions-contrib/influx-extensions</module>
|
||||
<module>extensions-contrib/azure-extensions</module>
|
||||
<module>extensions-contrib/cassandra-storage</module>
|
||||
<module>extensions-contrib/druid-rocketmq</module>
|
||||
<module>extensions-contrib/cloudfiles-extensions</module>
|
||||
<module>extensions-contrib/graphite-emitter</module>
|
||||
<module>extensions-contrib/kafka-eight-simpleConsumer</module>
|
||||
<module>extensions-contrib/rabbitmq</module>
|
||||
<module>extensions-contrib/distinctcount</module>
|
||||
<module>extensions-contrib/statsd-emitter</module>
|
||||
<module>extensions-contrib/time-min-max</module>
|
||||
|
@ -1614,6 +1609,7 @@
|
|||
<exclude>node_modules/**</exclude>
|
||||
<exclude>coordinator-console/**</exclude>
|
||||
<exclude>src/**/*.snap</exclude>
|
||||
<exclude>examples/conf/**</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.segment.indexing;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.data.input.FirehoseFactory;
|
||||
import org.apache.druid.data.input.FirehoseFactoryV2;
|
||||
import org.apache.druid.segment.realtime.plumber.PlumberSchool;
|
||||
|
||||
/**
|
||||
|
@ -31,22 +30,15 @@ public class RealtimeIOConfig implements IOConfig
|
|||
{
|
||||
private final FirehoseFactory firehoseFactory;
|
||||
private final PlumberSchool plumberSchool;
|
||||
private final FirehoseFactoryV2 firehoseFactoryV2;
|
||||
|
||||
@JsonCreator
|
||||
public RealtimeIOConfig(
|
||||
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
|
||||
@JsonProperty("plumber") PlumberSchool plumberSchool,
|
||||
@JsonProperty("firehoseV2") FirehoseFactoryV2 firehoseFactoryV2
|
||||
@JsonProperty("plumber") PlumberSchool plumberSchool
|
||||
)
|
||||
{
|
||||
if (firehoseFactory != null && firehoseFactoryV2 != null) {
|
||||
throw new IllegalArgumentException("Only provide one of firehose or firehoseV2");
|
||||
}
|
||||
|
||||
this.firehoseFactory = firehoseFactory;
|
||||
this.plumberSchool = plumberSchool;
|
||||
this.firehoseFactoryV2 = firehoseFactoryV2;
|
||||
}
|
||||
|
||||
@JsonProperty("firehose")
|
||||
|
@ -55,12 +47,6 @@ public class RealtimeIOConfig implements IOConfig
|
|||
return firehoseFactory;
|
||||
}
|
||||
|
||||
@JsonProperty("firehoseV2")
|
||||
public FirehoseFactoryV2 getFirehoseFactoryV2()
|
||||
{
|
||||
return firehoseFactoryV2;
|
||||
}
|
||||
|
||||
public PlumberSchool getPlumberSchool()
|
||||
{
|
||||
return plumberSchool;
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.data.input.Firehose;
|
||||
import org.apache.druid.data.input.FirehoseV2;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.IngestionSpec;
|
||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
|
@ -94,21 +93,11 @@ public class FireDepartment extends IngestionSpec<RealtimeIOConfig, RealtimeTuni
|
|||
return ioConfig.getPlumberSchool().findPlumber(dataSchema, tuningConfig, metrics);
|
||||
}
|
||||
|
||||
public boolean checkFirehoseV2()
|
||||
{
|
||||
return ioConfig.getFirehoseFactoryV2() != null;
|
||||
}
|
||||
|
||||
public Firehose connect() throws IOException
|
||||
{
|
||||
return ioConfig.getFirehoseFactory().connect(dataSchema.getParser(), null);
|
||||
}
|
||||
|
||||
public FirehoseV2 connect(Object metaData) throws IOException
|
||||
{
|
||||
return ioConfig.getFirehoseFactoryV2().connect(dataSchema.getParser(), metaData);
|
||||
}
|
||||
|
||||
public FireDepartmentMetrics getMetrics()
|
||||
{
|
||||
return metrics;
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.google.common.base.Supplier;
|
|||
import com.google.common.base.Suppliers;
|
||||
import org.apache.druid.data.input.Committer;
|
||||
import org.apache.druid.data.input.Firehose;
|
||||
import org.apache.druid.data.input.FirehoseV2;
|
||||
|
||||
public class Committers
|
||||
{
|
||||
|
@ -87,18 +86,6 @@ public class Committers
|
|||
};
|
||||
}
|
||||
|
||||
public static Supplier<Committer> supplierFromFirehoseV2(final FirehoseV2 firehose)
|
||||
{
|
||||
return new Supplier<Committer>()
|
||||
{
|
||||
@Override
|
||||
public Committer get()
|
||||
{
|
||||
return firehose.makeCommitter();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public static Committer nil()
|
||||
{
|
||||
return NIL;
|
||||
|
|
|
@ -258,11 +258,9 @@ public class InitializationTest
|
|||
}
|
||||
};
|
||||
final File mysql_metadata_storage = new File(extensionsDir, "mysql-metadata-storage");
|
||||
final File druid_kafka_eight = new File(extensionsDir, "druid-kafka-eight");
|
||||
mysql_metadata_storage.mkdir();
|
||||
druid_kafka_eight.mkdir();
|
||||
|
||||
final File[] expectedFileList = new File[]{druid_kafka_eight, mysql_metadata_storage};
|
||||
final File[] expectedFileList = new File[]{mysql_metadata_storage};
|
||||
final File[] actualFileList = Initialization.getExtensionFilesToLoad(config);
|
||||
Arrays.sort(actualFileList);
|
||||
Assert.assertArrayEquals(expectedFileList, actualFileList);
|
||||
|
@ -284,7 +282,7 @@ public class InitializationTest
|
|||
@Override
|
||||
public LinkedHashSet<String> getLoadList()
|
||||
{
|
||||
return Sets.newLinkedHashSet(Arrays.asList("mysql-metadata-storage", "druid-kafka-eight", absolutePathExtension.getAbsolutePath()));
|
||||
return Sets.newLinkedHashSet(Arrays.asList("mysql-metadata-storage", absolutePathExtension.getAbsolutePath()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -294,14 +292,12 @@ public class InitializationTest
|
|||
}
|
||||
};
|
||||
final File mysql_metadata_storage = new File(extensionsDir, "mysql-metadata-storage");
|
||||
final File druid_kafka_eight = new File(extensionsDir, "druid-kafka-eight");
|
||||
final File random_extension = new File(extensionsDir, "random-extensions");
|
||||
|
||||
mysql_metadata_storage.mkdir();
|
||||
druid_kafka_eight.mkdir();
|
||||
random_extension.mkdir();
|
||||
|
||||
final File[] expectedFileList = new File[]{mysql_metadata_storage, druid_kafka_eight, absolutePathExtension};
|
||||
final File[] expectedFileList = new File[]{mysql_metadata_storage, absolutePathExtension};
|
||||
final File[] actualFileList = Initialization.getExtensionFilesToLoad(config);
|
||||
Assert.assertArrayEquals(expectedFileList, actualFileList);
|
||||
}
|
||||
|
@ -319,7 +315,7 @@ public class InitializationTest
|
|||
@Override
|
||||
public LinkedHashSet<String> getLoadList()
|
||||
{
|
||||
return Sets.newLinkedHashSet(Arrays.asList("mysql-metadata-storage", "druid-kafka-eight"));
|
||||
return Sets.newLinkedHashSet(ImmutableList.of("mysql-metadata-storage"));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -328,9 +324,7 @@ public class InitializationTest
|
|||
return extensionsDir.getAbsolutePath();
|
||||
}
|
||||
};
|
||||
final File druid_kafka_eight = new File(extensionsDir, "druid-kafka-eight");
|
||||
final File random_extension = new File(extensionsDir, "random-extensions");
|
||||
druid_kafka_eight.mkdir();
|
||||
random_extension.mkdir();
|
||||
Initialization.getExtensionFilesToLoad(config);
|
||||
}
|
||||
|
|
|
@ -119,8 +119,7 @@ public class FireDepartmentTest
|
|||
new CachePopulatorStats(),
|
||||
TestHelper.makeJsonMapper()
|
||||
|
||||
),
|
||||
null
|
||||
)
|
||||
),
|
||||
RealtimeTuningConfig.makeDefaultTuningConfig(new File("/tmp/nonexistent"))
|
||||
);
|
||||
|
|
|
@ -152,8 +152,7 @@ public class DruidJsonValidatorTest
|
|||
),
|
||||
new RealtimeIOConfig(
|
||||
new LocalFirehoseFactory(new File("lol"), "rofl", null),
|
||||
(schema, config, metrics) -> null,
|
||||
null
|
||||
(schema, config, metrics) -> null
|
||||
),
|
||||
|
||||
new RealtimeTuningConfig(
|
||||
|
|
Loading…
Reference in New Issue