diff --git a/inlong-dataproxy/bin/flume-ng b/inlong-dataproxy/bin/flume-ng new file mode 100644 index 00000000000..019714dd0c5 --- /dev/null +++ b/inlong-dataproxy/bin/flume-ng @@ -0,0 +1,379 @@ +#!/bin/bash +# +# +# 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. +# + +################################ +# constants +################################ + +FLUME_AGENT_CLASS="org.apache.flume.node.Application" +FLUME_AVRO_CLIENT_CLASS="org.apache.flume.client.avro.AvroCLIClient" +FLUME_VERSION_CLASS="org.apache.flume.tools.VersionInfo" + +CLEAN_FLAG=1 +################################ +# functions +################################ + +info() { + if [ ${CLEAN_FLAG} -ne 0 ]; then + local msg=$1 + echo "Info: $msg" >&2 + fi +} + +warn() { + if [ ${CLEAN_FLAG} -ne 0 ]; then + local msg=$1 + echo "Warning: $msg" >&2 + fi +} + +error() { + local msg=$1 + local exit_code=$2 + + echo "Error: $msg" >&2 + + if [ -n "$exit_code" ] ; then + exit $exit_code + fi +} + +# If avail, add Hadoop paths to the FLUME_CLASSPATH and to the +# FLUME_JAVA_LIBRARY_PATH env vars. +# Requires Flume jars to already be on FLUME_CLASSPATH. +add_hadoop_paths() { + local HADOOP_IN_PATH=$(PATH="${HADOOP_HOME:-${HADOOP_PREFIX}}/bin:$PATH" \ + which hadoop 2>/dev/null) + + if [ -f "${HADOOP_IN_PATH}" ]; then + info "Including Hadoop libraries found via ($HADOOP_IN_PATH) for HDFS access" + + # determine hadoop java.library.path and use that for flume + local HADOOP_CLASSPATH="" + local HADOOP_JAVA_LIBRARY_PATH=$(HADOOP_CLASSPATH="$FLUME_CLASSPATH" \ + ${HADOOP_IN_PATH} org.apache.flume.tools.GetJavaProperty \ + java.library.path) + + # look for the line that has the desired property value + # (considering extraneous output from some GC options that write to stdout) + # IFS = InternalFieldSeparator (set to recognize only newline char as delimiter) + IFS=$'\n' + for line in $HADOOP_JAVA_LIBRARY_PATH; do + if [[ $line =~ ^java\.library\.path=(.*)$ ]]; then + HADOOP_JAVA_LIBRARY_PATH=${BASH_REMATCH[1]} + break + fi + done + unset IFS + + if [ -n "${HADOOP_JAVA_LIBRARY_PATH}" ]; then + FLUME_JAVA_LIBRARY_PATH="$FLUME_JAVA_LIBRARY_PATH:$HADOOP_JAVA_LIBRARY_PATH" + fi + + # determine hadoop classpath + HADOOP_CLASSPATH=$($HADOOP_IN_PATH classpath) + + # hack up and filter hadoop classpath + local ELEMENTS=$(sed -e 's/:/ /g' <<<${HADOOP_CLASSPATH}) + local ELEMENT + for ELEMENT in $ELEMENTS; do + local PIECE + for PIECE in $(echo $ELEMENT); do + if [[ $PIECE =~ slf4j-(api|log4j12).*\.jar ]]; then + info "Excluding $PIECE from classpath" + continue + else + FLUME_CLASSPATH="$FLUME_CLASSPATH:$PIECE" + fi + done + done + + fi +} +add_HBASE_paths() { + local HBASE_IN_PATH=$(PATH="${HBASE_HOME}/bin:$PATH" \ + which hbase 2>/dev/null) + + if [ -f "${HBASE_IN_PATH}" ]; then + info "Including HBASE libraries found via ($HBASE_IN_PATH) for HBASE access" + + # determine HBASE java.library.path and use that for flume + local HBASE_CLASSPATH="" + local HBASE_JAVA_LIBRARY_PATH=$(HBASE_CLASSPATH="$FLUME_CLASSPATH" \ + ${HBASE_IN_PATH} org.apache.flume.tools.GetJavaProperty \ + java.library.path) + + # look for the line that has the desired property value + # (considering extraneous output from some GC options that write to stdout) + # IFS = InternalFieldSeparator (set to recognize only newline char as delimiter) + IFS=$'\n' + for line in $HBASE_JAVA_LIBRARY_PATH; do + if [[ $line =~ ^java\.library\.path=(.*)$ ]]; then + HBASE_JAVA_LIBRARY_PATH=${BASH_REMATCH[1]} + break + fi + done + unset IFS + + if [ -n "${HBASE_JAVA_LIBRARY_PATH}" ]; then + FLUME_JAVA_LIBRARY_PATH="$FLUME_JAVA_LIBRARY_PATH:$HBASE_JAVA_LIBRARY_PATH" + fi + + # determine HBASE classpath + HBASE_CLASSPATH=$($HBASE_IN_PATH classpath) + + # hack up and filter HBASE classpath + local ELEMENTS=$(sed -e 's/:/ /g' <<<${HBASE_CLASSPATH}) + local ELEMENT + for ELEMENT in $ELEMENTS; do + local PIECE + for PIECE in $(echo $ELEMENT); do + if [[ $PIECE =~ slf4j-(api|log4j12).*\.jar ]]; then + info "Excluding $PIECE from classpath" + continue + else + FLUME_CLASSPATH="$FLUME_CLASSPATH:$PIECE" + fi + done + done + FLUME_CLASSPATH="$FLUME_CLASSPATH:$HBASE_HOME/conf" + + fi +} + +set_LD_LIBRARY_PATH(){ +#Append the FLUME_JAVA_LIBRARY_PATH to whatever the user may have specified in +#flume-env.sh + if [ -n "${FLUME_JAVA_LIBRARY_PATH}" ]; then + export LD_LIBRARY_PATH="${LD_LIBRARY_PATH}:${FLUME_JAVA_LIBRARY_PATH}" + fi +} + +display_help() { + cat < [options]... + +commands: + help display this help text + agent run a Flume agent + avro-client run an avro Flume client + version show Flume version info + +global options: + --conf,-c use configs in directory + --classpath,-C append to the classpath + --dryrun,-d do not actually start Flume, just print the command + -Dproperty=value sets a JDK system property value + +agent options: + --conf-file,-f specify a config file (required) + --name,-n the name of this agent (required) + --help,-h display help text + +avro-client options: + --dirname directory to stream to avro source + --host,-H hostname to which events will be sent (required) + --port,-p port of the avro source (required) + --filename,-F text file to stream to avro source [default: std input] + --headerFile,-R headerFile containing headers as key/value pairs on each new line + --help,-h display help text + +Note that if directory is specified, then it is always included first +in the classpath. + +EOF +} + +run_flume() { + local FLUME_APPLICATION_CLASS + + if [ "$#" -gt 0 ]; then + FLUME_APPLICATION_CLASS=$1 + shift + else + error "Must specify flume application class" 1 + fi + + if [ ${CLEAN_FLAG} -ne 0 ]; then + set -x + fi + $EXEC $JAVA_HOME/bin/java $JAVA_OPTS -cp "$FLUME_CLASSPATH" -Djava.util.logging.config.file="/data/work/osgoo/authcentertest/apache-flume-1.6.0-bin/conf/netty.properties" \ + -Djava.library.path=$FLUME_JAVA_LIBRARY_PATH "$FLUME_APPLICATION_CLASS" $* +} + +################################ +# main +################################ + +# set default params +FLUME_CLASSPATH="" +JAVA_OPTS="-Xmx10g -Xms10g -Xmn2g -XX:SurvivorRatio=6" +#LD_LIBRARY_PATH="" + +opt_conf="" +opt_classpath="" +opt_dryrun="" + +mode=$1 +shift + +case "$mode" in + help) + display_help + exit 0 + ;; + agent) + opt_agent=1 + ;; + node) + opt_agent=1 + warn "The \"node\" command is deprecated. Please use \"agent\" instead." + ;; + avro-client) + opt_avro_client=1 + ;; + version) + opt_version=1 + CLEAN_FLAG=0 + ;; + *) + error "Unknown or unspecified command '$mode'" + echo + display_help + exit 1 + ;; +esac + +while [ -n "$*" ] ; do + arg=$1 + shift + + case "$arg" in + --conf|-c) + [ -n "$1" ] || error "Option --conf requires an argument" 1 + opt_conf=$1 + shift + ;; + --classpath|-C) + [ -n "$1" ] || error "Option --classpath requires an argument" 1 + opt_classpath=$1 + shift + ;; + --dryrun|-d) + opt_dryrun="1" + ;; + -D*) + opt_java_props="$opt_java_props $arg" + ;; + -X*) + opt_java_props="$opt_java_props $arg" + ;; + *) + args="$args $arg" + ;; + esac +done + +# make opt_conf absolute +if [[ -n "$opt_conf" && -d "$opt_conf" ]]; then + opt_conf=$(cd $opt_conf; pwd) +fi + +# allow users to override the default env vars via conf/flume-env.sh +if [ -z "$opt_conf" ]; then + warn "No configuration directory set! Use --conf to override." +elif [ -f "$opt_conf/flume-env.sh" ]; then + info "Sourcing environment configuration script $opt_conf/flume-env.sh" + source "$opt_conf/flume-env.sh" +fi + +# append command-line java options to stock or env script JAVA_OPTS +if [ -n "${opt_java_props}" ]; then + JAVA_OPTS="${JAVA_OPTS} ${opt_java_props}" +fi + +# prepend command-line classpath to env script classpath +if [ -n "${opt_classpath}" ]; then + if [ -n "${FLUME_CLASSPATH}" ]; then + FLUME_CLASSPATH="${opt_classpath}:${FLUME_CLASSPATH}" + else + FLUME_CLASSPATH="${opt_classpath}" + fi +fi + +if [ -z "${FLUME_HOME}" ]; then + FLUME_HOME=$(cd $(dirname $0)/..; pwd) +fi + +# prepend $FLUME_HOME/lib jars to the specified classpath (if any) +if [ -n "${FLUME_CLASSPATH}" ] ; then + FLUME_CLASSPATH="${FLUME_HOME}/lib/*:$FLUME_CLASSPATH" +else + FLUME_CLASSPATH="${FLUME_HOME}/lib/*" +fi + +# find java +if [ -z "${JAVA_HOME}" ] ; then + warn "JAVA_HOME is not set!" + # Try to use Bigtop to autodetect JAVA_HOME if it's available + if [ -e /usr/libexec/bigtop-detect-javahome ] ; then + . /usr/libexec/bigtop-detect-javahome + elif [ -e /usr/lib/bigtop-utils/bigtop-detect-javahome ] ; then + . /usr/lib/bigtop-utils/bigtop-detect-javahome + fi + + # Using java from path if bigtop is not installed or couldn't find it + if [ -z "${JAVA_HOME}" ] ; then + JAVA_DEFAULT=$(type -p java) + [ -n "$JAVA_DEFAULT" ] || error "Unable to find java executable. Is it in your PATH?" 1 + JAVA_HOME=$(cd $(dirname $JAVA_DEFAULT)/..; pwd) + fi +fi + +# look for hadoop libs +#add_hadoop_paths +#add_HBASE_paths + +# prepend conf dir to classpath +if [ -n "$opt_conf" ]; then + FLUME_CLASSPATH="$opt_conf:$FLUME_CLASSPATH" +fi + +set_LD_LIBRARY_PATH +# allow dryrun +EXEC="exec" +if [ -n "${opt_dryrun}" ]; then + warn "Dryrun mode enabled (will not actually initiate startup)" + EXEC="echo" +fi +echo "LD_LIBRARY_PATH:$LD_LIBRARY_PATH" +# finally, invoke the appropriate command +if [ -n "$opt_agent" ] ; then + run_flume $FLUME_AGENT_CLASS $args +elif [ -n "$opt_avro_client" ] ; then + run_flume $FLUME_AVRO_CLIENT_CLASS $args +elif [ -n "${opt_version}" ] ; then + run_flume $FLUME_VERSION_CLASS $args +else + error "This message should never appear" 1 +fi +exit 0 diff --git a/inlong-dataproxy/bin/prepare_env.sh b/inlong-dataproxy/bin/prepare_env.sh new file mode 100644 index 00000000000..e0d4cdbf102 --- /dev/null +++ b/inlong-dataproxy/bin/prepare_env.sh @@ -0,0 +1,27 @@ +#!/bin/bash +# +# +# 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. +# + +cd ../conf + +for i in {mx.properties,transfer.properties,weight.properties,common.properties,blacklist.properties,bid_mapping.properties,dc_mapping.properties,topics.properties,tube_switch.properties} + do + touch $i +done \ No newline at end of file diff --git a/inlong-dataproxy/bin/start.sh b/inlong-dataproxy/bin/start.sh new file mode 100644 index 00000000000..8ef68194f81 --- /dev/null +++ b/inlong-dataproxy/bin/start.sh @@ -0,0 +1,22 @@ +#!/bin/bash +# +# +# 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. +# +chmod 777 bin/flume-ng +nohup bin/flume-ng agent --conf conf/ -f conf/flume.conf -n agent1 --no-reload-conf > nohup.out 2>&1 & \ No newline at end of file diff --git a/inlong-dataproxy/conf/flume.conf b/inlong-dataproxy/conf/flume.conf new file mode 100644 index 00000000000..9385608e8cb --- /dev/null +++ b/inlong-dataproxy/conf/flume.conf @@ -0,0 +1,273 @@ +# +# 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. +# + +#6-segment protocol TCP source +agent1.sources.tcp-source.channels = ch-msg1 ch-msg2 ch-msg3 ch-more1 ch-more2 ch-more3 ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9 ch-msg10 ch-back +agent1.sources.tcp-source.type = org.apache.inlong.dataproxy.source.SimpleTcpSource +agent1.sources.tcp-source.msg-factory-name = org.apache.inlong.dataproxy.source.ServerMessageFactory +agent1.sources.tcp-source.message-handler-name = org.apache.inlong.dataproxy.source.ServerMessageHandler +agent1.sources.tcp-source.host = 0.0.0.0 +agent1.sources.tcp-source.port = 46801 +agent1.sources.tcp-source.highWaterMark=2621440 +agent1.sources.tcp-source.enableExceptionReturn=true +agent1.sources.tcp-source.max-msg-length = 524288 +agent1.sources.tcp-source.topic = test_token +agent1.sources.tcp-source.attr = m=9 +agent1.sources.tcp-source.connections = 5000 +agent1.sources.tcp-source.max-threads = 64 +agent1.sources.tcp-source.receiveBufferSize = 524288 +agent1.sources.tcp-source.sendBufferSize = 524288 +agent1.sources.tcp-source.custom-cp = true +agent1.sources.tcp-source.selector.type = org.apache.inlong.dataproxy.channel.FailoverChannelSelector +agent1.sources.tcp-source.selector.master = ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9 +agent1.sources.tcp-source.selector.transfer = ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9 +agent1.sources.tcp-source.selector.fileMetric = ch-back +agent1.sources.tcp-source.selector.slaMetric = ch-msg10 +agent1.sources.tcp-source.metric-recovery-path=FLUME_HOME/metric +agent1.sources.tcp-source.metric-agent-port=8003 +agent1.sources.tcp-source.metric-cache-size=10000 +agent1.sources.tcp-source.set=10 +agent1.sources.tcp-source.old-metric-on=true +agent1.sources.tcp-source.new-metric-on=true +agent1.sources.tcp-source.metric_topic_prefix=manager_tmertic + +agent1.channels.ch-back.type = memory +agent1.channels.ch-back.capacity = 10000000 +agent1.channels.ch-back.keep-alive = 0 +agent1.channels.ch-back.transactionCapacity = 200 + +agent1.channels.ch-msg1.type = memory +agent1.channels.ch-msg1.capacity = 200000 +agent1.channels.ch-msg1.keep-alive = 0 +agent1.channels.ch-msg1.transactionCapacity = 200 + +agent1.channels.ch-msg2.type = memory +agent1.channels.ch-msg2.capacity = 200000 +agent1.channels.ch-msg2.keep-alive = 0 +agent1.channels.ch-msg2.transactionCapacity = 200 + +agent1.channels.ch-msg3.type = memory +agent1.channels.ch-msg3.capacity = 200000 +agent1.channels.ch-msg3.keep-alive = 0 +agent1.channels.ch-msg3.transactionCapacity = 200 + +agent1.channels.ch-msg5.type = file +agent1.channels.ch-msg5.capacity = 100000000 +agent1.channels.ch-msg5.maxFileSize = 1073741824 +agent1.channels.ch-msg5.minimumRequiredSpace = 1073741824 +agent1.channels.ch-msg5.checkpointDir = FLUME_HOME/file/ch-msg5/check +agent1.channels.ch-msg5.dataDirs = FLUME_HOME/file/ch-msg5/data +agent1.channels.ch-msg5.fsyncPerTransaction = false +agent1.channels.ch-msg5.fsyncInterval = 5 + +agent1.channels.ch-msg6.type = file +agent1.channels.ch-msg6.capacity = 100000000 +agent1.channels.ch-msg6.maxFileSize = 1073741824 +agent1.channels.ch-msg6.minimumRequiredSpace = 1073741824 +agent1.channels.ch-msg6.checkpointDir = FLUME_HOME/file/ch-msg6/check +agent1.channels.ch-msg6.dataDirs = FLUME_HOME/file/ch-msg6/data +agent1.channels.ch-msg6.fsyncPerTransaction = false +agent1.channels.ch-msg6.fsyncInterval = 5 + +agent1.channels.ch-msg7.type = file +agent1.channels.ch-msg7.capacity = 100000000 +agent1.channels.ch-msg7.maxFileSize = 1073741824 +agent1.channels.ch-msg7.minimumRequiredSpace = 1073741824 +agent1.channels.ch-msg7.checkpointDir = FLUME_HOME/file/ch-msg7/check +agent1.channels.ch-msg7.dataDirs = FLUME_HOME/file/ch-msg7/data +agent1.channels.ch-msg7.fsyncPerTransaction = false +agent1.channels.ch-msg7.fsyncInterval = 5 + +agent1.channels.ch-msg8.type = file +agent1.channels.ch-msg8.capacity = 100000000 +agent1.channels.ch-msg8.maxFileSize = 1073741824 +agent1.channels.ch-msg8.minimumRequiredSpace = 1073741824 +agent1.channels.ch-msg8.checkpointDir = FLUME_HOME/file/ch-msg8/check +agent1.channels.ch-msg8.dataDirs = FLUME_HOME/file/ch-msg8/data +agent1.channels.ch-msg8.fsyncPerTransaction = false +agent1.channels.ch-msg8.fsyncInterval = 5 + +agent1.channels.ch-msg9.type = file +agent1.channels.ch-msg9.capacity = 100000000 +agent1.channels.ch-msg9.maxFileSize = 1073741824 +agent1.channels.ch-msg9.minimumRequiredSpace = 1073741824 +agent1.channels.ch-msg9.checkpointDir = FLUME_HOME/file/ch-msg9/check +agent1.channels.ch-msg9.dataDirs = FLUME_HOME/file/ch-msg9/data +agent1.channels.ch-msg9.fsyncPerTransaction = false +agent1.channels.ch-msg9.fsyncInterval = 5 + +agent1.channels.ch-msg10.type = file +agent1.channels.ch-msg10.capacity = 100000000 +agent1.channels.ch-msg10.maxFileSize = 1073741824 +agent1.channels.ch-msg10.minimumRequiredSpace = 1073741824 +agent1.channels.ch-msg10.checkpointDir = FLUME_HOME/file/ch-msg10/check +agent1.channels.ch-msg10.dataDirs = FLUME_HOME/file/ch-msg10/data +agent1.channels.ch-msg10.fsyncPerTransaction = false +agent1.channels.ch-msg10.fsyncInterval = 5 + +agent1.sinks.meta-sink-more1.channel = ch-msg1 +agent1.sinks.meta-sink-more1.type = org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-more1.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-more1.send_timeout = 30000 +agent1.sinks.meta-sink-more1.stat-interval-sec = 60 +agent1.sinks.meta-sink-more1.thread-num = 8 +agent1.sinks.meta-sink-more1.client-id-cache = true +agent1.sinks.meta-sink-more1.max-survived-time = 300000 +agent1.sinks.meta-sink-more1.max-survived-size = 3000000 +agent1.sinks.meta-sink-more1.new-check-pattern = true +agent1.sinks.meta-sink-more1.old-metric-on=true +agent1.sinks.meta-sink-more1.set=10 + +agent1.sinks.meta-sink-more2.channel = ch-more2 +agent1.sinks.meta-sink-more2.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-more2.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-more2.send_timeout = 30000 +agent1.sinks.meta-sink-more2.stat-interval-sec = 60 +agent1.sinks.meta-sink-more2.thread-num = 8 +agent1.sinks.meta-sink-more2.client-id-cache = true +agent1.sinks.meta-sink-more2.max-survived-time = 300000 +agent1.sinks.meta-sink-more2.max-survived-size = 3000000 +agent1.sinks.meta-sink-more2.new-check-pattern = true +agent1.sinks.meta-sink-more2.old-metric-on=true +agent1.sinks.meta-sink-more2.set=10 + +agent1.sinks.meta-sink-more3.channel = ch-more3 +agent1.sinks.meta-sink-more3.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-more3.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-more3.send_timeout = 30000 +agent1.sinks.meta-sink-more3.stat-interval-sec = 60 +agent1.sinks.meta-sink-more3.thread-num = 8 +agent1.sinks.meta-sink-more3.client-id-cache = true +agent1.sinks.meta-sink-more3.max-survived-time = 300000 +agent1.sinks.meta-sink-more3.max-survived-size = 3000000 +agent1.sinks.meta-sink-more3.new-check-pattern = true +agent1.sinks.meta-sink-more3.old-metric-on=true +agent1.sinks.meta-sink-more3.set=10 + +agent1.sinks.meta-sink-most1.channel = ch-most1 +agent1.sinks.meta-sink-most1.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-most1.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-most1.send_timeout = 30000 +agent1.sinks.meta-sink-most1.stat-interval-sec = 60 +agent1.sinks.meta-sink-most1.thread-num = 8 +agent1.sinks.meta-sink-most1.client-id-cache = true +agent1.sinks.meta-sink-most1.max-survived-time = 300000 +agent1.sinks.meta-sink-most1.max-survived-size = 3000000 +agent1.sinks.meta-sink-most1.new-check-pattern = true +agent1.sinks.meta-sink-most1.old-metric-on=true +agent1.sinks.meta-sink-most1.set=10 + +agent1.sinks.meta-sink-most2.channel = ch-most2 +agent1.sinks.meta-sink-most2.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-most2.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-most2.send_timeout = 30000 +agent1.sinks.meta-sink-most2.stat-interval-sec = 60 +agent1.sinks.meta-sink-most2.thread-num = 8 +agent1.sinks.meta-sink-most2.client-id-cache = true +agent1.sinks.meta-sink-most2.max-survived-time = 300000 +agent1.sinks.meta-sink-most2.max-survived-size = 3000000 +agent1.sinks.meta-sink-most2.new-check-pattern = true +agent1.sinks.meta-sink-most2.old-metric-on=true +agent1.sinks.meta-sink-most2.set=10 + +agent1.sinks.meta-sink-most3.channel = ch-most3 +agent1.sinks.meta-sink-most3.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-most3.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-most3.send_timeout = 30000 +agent1.sinks.meta-sink-most3.stat-interval-sec = 60 +agent1.sinks.meta-sink-most3.thread-num = 8 +agent1.sinks.meta-sink-most3.client-id-cache = true +agent1.sinks.meta-sink-most3.max-survived-time = 300000 +agent1.sinks.meta-sink-most3.max-survived-size = 3000000 +agent1.sinks.meta-sink-most3.new-check-pattern = true +agent1.sinks.meta-sink-most3.old-metric-on=true +agent1.sinks.meta-sink-most3.set=10 + + +agent1.sinks.meta-sink-msg1.channel = ch-msg1 +agent1.sinks.meta-sink-msg1.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-msg1.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-msg1.send_timeout = 30000 +agent1.sinks.meta-sink-msg1.stat-interval-sec = 60 +agent1.sinks.meta-sink-msg1.thread-num = 8 +agent1.sinks.meta-sink-msg1.client-id-cache = true +agent1.sinks.meta-sink-msg1.max-survived-time = 300000 +agent1.sinks.meta-sink-msg1.max-survived-size = 3000000 +agent1.sinks.meta-sink-msg1.new-check-pattern = true +agent1.sinks.meta-sink-msg1.old-metric-on=true +agent1.sinks.meta-sink-msg1.set=10#tonglevip + +agent1.sinks.meta-sink-msg2.channel = ch-msg2 +agent1.sinks.meta-sink-msg2.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-msg2.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-msg2.send_timeout = 30000 +agent1.sinks.meta-sink-msg2.stat-interval-sec = 60 +agent1.sinks.meta-sink-msg2.thread-num = 8 +agent1.sinks.meta-sink-msg2.client-id-cache = true +agent1.sinks.meta-sink-msg2.max-survived-time = 300000 +agent1.sinks.meta-sink-msg2.max-survived-size = 3000000 +agent1.sinks.meta-sink-msg2.new-check-pattern = true +agent1.sinks.meta-sink-msg2.old-metric-on=true +agent1.sinks.meta-sink-msg2.set=10#tonglevip + +agent1.sinks.meta-sink-msg3.channel = ch-msg3 +agent1.sinks.meta-sink-msg3.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-msg3.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-msg3.send_timeout = 30000 +agent1.sinks.meta-sink-msg3.stat-interval-sec = 60 +agent1.sinks.meta-sink-msg3.thread-num = 8 +agent1.sinks.meta-sink-msg3.client-id-cache = true +agent1.sinks.meta-sink-msg3.max-survived-time = 300000 +agent1.sinks.meta-sink-msg3.max-survived-size = 3000000 +agent1.sinks.meta-sink-msg3.new-check-pattern = true +agent1.sinks.meta-sink-msg3.old-metric-on=true +agent1.sinks.meta-sink-msg3.set=10#tonglevip + +agent1.sinks.meta-sink-msg8.channel = ch-msg1 +agent1.sinks.meta-sink-msg8.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-msg8.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-msg8.send_timeout = 30000 +agent1.sinks.meta-sink-msg8.stat-interval-sec = 60 +agent1.sinks.meta-sink-msg8.thread-num = 8 +agent1.sinks.meta-sink-msg8.client-id-cache = true +agent1.sinks.meta-sink-msg8.max-survived-time = 300000 +agent1.sinks.meta-sink-msg8.max-survived-size = 3000000 +agent1.sinks.meta-sink-msg8.new-check-pattern = true +agent1.sinks.meta-sink-msg8.old-metric-on=true +agent1.sinks.meta-sink-msg8.set=10 +agent1.sinks.meta-sink-msg8.disk-io-rate-per-sec=20000000 + +agent1.sinks.meta-sink-msg8.channel = ch-back +agent1.sinks.meta-sink-msg8.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-msg8.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-msg8.send_timeout = 30000 +agent1.sinks.meta-sink-msg8.stat-interval-sec = 60 +agent1.sinks.meta-sink-msg8.thread-num = 8 +agent1.sinks.meta-sink-msg8.client-id-cache = true +agent1.sinks.meta-sink-msg8.max-survived-time = 300000 +agent1.sinks.meta-sink-msg8.max-survived-size = 3000000 +agent1.sinks.meta-sink-msg8.new-check-pattern = true +agent1.sinks.meta-sink-msg8.old-metric-on=true +agent1.sinks.meta-sink-msg8.set=10 +agent1.sinks.meta-sink-msg8.disk-io-rate-per-sec=20000000 + +agent1.sources = tcp-source +agent1.channels = ch-back ch-msg1 ch-msg2 ch-msg3 ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9 ch-msg10 +agent1.sinks = meta-sink-msg1 meta-sink-msg2 meta-sink-msg3 meta-sink-msg5 meta-sink-msg6 meta-sink-msg7 meta-sink-msg8 meta-sink-transfer + diff --git a/inlong-dataproxy/conf/log4j.properties b/inlong-dataproxy/conf/log4j.properties new file mode 100644 index 00000000000..187e13166a0 --- /dev/null +++ b/inlong-dataproxy/conf/log4j.properties @@ -0,0 +1,202 @@ +# +# 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. +# + +log4j.rootLogger=DEBUG,debug,info,warn,error +log.dir = ./logs +log.file.name.prefix=flume.log +log.file.info.name.postfix= +log.file.debug.name.postfix=.debug +log.file.warn.name.postfix=.warn +log.file.error.name.postfix=.error +log.file.stat.name=flume_stat.log +log.file.flume.measure.name=flume_measure.log +log.file.agent.measure.name=agent_measure.log +log.file.agent.pcgmeasure.name=agent_pcg_measure.log +log.file.agent.hourmeasure.name=agent_hour_measure.log +log.file.agent.pcgcheckmeasure.name=agent_pcg_check_measure.log +log.file.monitors.name=flume_monitors.log +log.file.index.name=flume_index.log + + +log4j.logger.org.apache.flume.util.MonitorIndexExt=info,monitor +log4j.additivity.org.apache.flume.util.MonitorIndexExt = false +log4j.appender.monitor=org.apache.log4j.RollingFileAppender +log4j.appender.monitor.MaxFileSize=100MB +log4j.appender.monitor.MaxBackupIndex=10 +log4j.appender.monitor.BufferedIO=false +log4j.appender.monitor.BufferSize=8192 +log4j.appender.monitor.layout=org.apache.log4j.PatternLayout +log4j.appender.monitor.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %m%n +log4j.appender.monitor.Threshold = INFO +log4j.appender.monitor.Append=true +log4j.appender.monitor.File=${log.dir}/${log.file.monitors.name} + +log4j.logger.org.apache.flume.util.MonitorIndex=info,index +log4j.additivity.org.apache.flume.util.MonitorIndex = false +log4j.appender.index=org.apache.log4j.RollingFileAppender +log4j.appender.index.MaxFileSize=100MB +log4j.appender.index.MaxBackupIndex=10 +log4j.appender.index.BufferedIO=false +log4j.appender.index.BufferSize=8192 +log4j.appender.index.layout=org.apache.log4j.PatternLayout +log4j.appender.index.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %m%n +log4j.appender.index.Threshold = INFO +log4j.appender.index.Append=true +log4j.appender.index.File=${log.dir}/${log.file.index.name} + +log4j.logger.com.tencent.tdbank.monitor.StatRunner=info,stat +log4j.additivity.com.tencent.tdbank.monitor.StatRunner = false + +log4j.logger.com.tencent.tdbank.monitor.StatRunnerExt=info,flume-measure +log4j.additivity.com.tencent.tdbank.monitor.StatRunnerExt = false + +log4j.logger.org.apache.flume.sink.AgentMeasureLogger=info,agent-measure +log4j.additivity.org.apache.flume.sink.AgentMeasureLogger=false + +log4j.logger.org.apache.flume.util.PcgMetricsCollectorMin=info,agent-min-measure +log4j.additivity.org.apache.flume.util.PcgMetricsCollectorMin=false +log4j.logger.org.apache.flume.util.PcgMetricsCollectorHour=info,agent-hour-measure +log4j.additivity.org.apache.flume.util.PcgMetricsCollectorHour=false + +log4j.logger.org.apache.flume.util.PcgMetricsCheckCollector=info,agent-pcg-check-measure +log4j.additivity.org.apache.flume.util.PcgMetricsCheckCollector=false + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %p %c{2} (%F:%M:%L): %m%n + +log4j.appender.stat=org.apache.log4j.RollingFileAppender +log4j.appender.stat.MaxFileSize=100MB +log4j.appender.stat.MaxBackupIndex=10 +log4j.appender.stat.BufferedIO=false +log4j.appender.stat.BufferSize=8192 +log4j.appender.stat.layout=org.apache.log4j.PatternLayout +log4j.appender.stat.layout.ConversionPattern=%d{ISO8601} %p %c{2} (%F:%M:%L): %m%n +log4j.appender.stat.Threshold = INFO +log4j.appender.stat.Append=true +log4j.appender.stat.File=${log.dir}/${log.file.stat.name} + + +log4j.appender.flume-measure=org.apache.log4j.RollingFileAppender +log4j.appender.flume-measure.MaxFileSize=100MB +log4j.appender.flume-measure.MaxBackupIndex=10 +log4j.appender.flume-measure.BufferedIO=false +log4j.appender.flume-measure.BufferSize=8192 +log4j.appender.flume-measure.layout=org.apache.log4j.PatternLayout +log4j.appender.flume-measure.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %m%n +log4j.appender.flume-measure.Threshold = INFO +log4j.appender.flume-measure.Append=true +log4j.appender.flume-measure.File=${log.dir}/${log.file.flume.measure.name} + +log4j.appender.agent-measure=org.apache.log4j.RollingFileAppender +log4j.appender.agent-measure.MaxFileSize=100MB +log4j.appender.agent-measure.MaxBackupIndex=10 +log4j.appender.agent-measure.BufferedIO=false +log4j.appender.agent-measure.BufferSize=512 +log4j.appender.agent-measure.layout=org.apache.log4j.PatternLayout +log4j.appender.agent-measure.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %m%n +log4j.appender.agent-measure.Threshold = INFO +log4j.appender.agent-measure.Append=true +log4j.appender.agent-measure.File=${log.dir}/${log.file.agent.measure.name} + + +log4j.appender.agent-min-measure=org.apache.log4j.RollingFileAppender +log4j.appender.agent-min-measure.MaxFileSize=100MB +log4j.appender.agent-min-measure.MaxBackupIndex=10 +log4j.appender.agent-min-measure.BufferedIO=false +log4j.appender.agent-min-measure.BufferSize=512 +log4j.appender.agent-min-measure.layout=org.apache.log4j.PatternLayout +log4j.appender.agent-min-measure.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %m%n +log4j.appender.agent-min-measure.Threshold = INFO +log4j.appender.agent-min-measure.Append=true +log4j.appender.agent-min-measure.File=${log.dir}/${log.file.agent.pcgmeasure.name} + +log4j.appender.agent-hour-measure=org.apache.log4j.RollingFileAppender +log4j.appender.agent-hour-measure.MaxFileSize=100MB +log4j.appender.agent-hour-measure.MaxBackupIndex=10 +log4j.appender.agent-hour-measure.BufferedIO=false +log4j.appender.agent-hour-measure.BufferSize=512 +log4j.appender.agent-hour-measure.layout=org.apache.log4j.PatternLayout +log4j.appender.agent-hour-measure.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %m%n +log4j.appender.agent-hour-measure.Threshold = INFO +log4j.appender.agent-hour-measure.Append=true +log4j.appender.agent-hour-measure.File=${log.dir}/${log.file.agent.hourmeasure.name} + + + +log4j.appender.agent-pcg-check-measure=org.apache.log4j.RollingFileAppender +log4j.appender.agent-pcg-check-measure.MaxFileSize=100MB +log4j.appender.agent-pcg-check-measure.MaxBackupIndex=10 +log4j.appender.agent-pcg-check-measure.BufferedIO=false +log4j.appender.agent-pcg-check-measure.BufferSize=512 +log4j.appender.agent-pcg-check-measure.layout=org.apache.log4j.PatternLayout +log4j.appender.agent-pcg-check-measure.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %m%n +log4j.appender.agent-pcg-check-measure.Threshold = INFO +log4j.appender.agent-pcg-check-measure.Append=true +log4j.appender.agent-pcg-check-measure.File=${log.dir}/${log.file.agent.pcgcheckmeasure.name} + + +log4j.logger.info=info +log4j.appender.info=org.apache.log4j.RollingFileAppender +log4j.appender.info.MaxFileSize=100MB +log4j.appender.info.MaxBackupIndex=10 +log4j.appender.info.BufferedIO=false +log4j.appender.info.BufferSize=8192 +log4j.appender.info.layout=org.apache.log4j.PatternLayout +log4j.appender.info.layout.ConversionPattern=%d{ISO8601} %p %c{2} (%F:%M:%L): %m%n +log4j.appender.info.Threshold = INFO +log4j.appender.info.Append=true +log4j.appender.info.File=${log.dir}/${log.file.name.prefix}${log.file.info.name.postfix} + +log4j.logger.debug=debug +log4j.appender.debug=org.apache.log4j.RollingFileAppender +log4j.appender.debug.MaxFileSize=100MB +log4j.appender.debug.MaxBackupIndex=10 +log4j.appender.debug.BufferedIO=false +log4j.appender.debug.BufferSize=8192 +log4j.appender.debug.layout=org.apache.log4j.PatternLayout +log4j.appender.debug.layout.ConversionPattern=%d{ISO8601} %p %c{2} (%F:%M:%L): %m%n +log4j.appender.debug.Threshold = DEBUG +log4j.appender.debug.Append=true +log4j.appender.debug.File=${log.dir}/${log.file.name.prefix}${log.file.debug.name.postfix} + +log4j.logger.warn=warn +log4j.appender.warn=org.apache.log4j.RollingFileAppender +log4j.appender.warn.MaxFileSize=100MB +log4j.appender.warn.MaxBackupIndex=10 +log4j.appender.warn.BufferedIO=false +log4j.appender.warn.BufferSize=8192 +log4j.appender.warn.layout=org.apache.log4j.PatternLayout +log4j.appender.warn.layout.ConversionPattern=%d{ISO8601} %p %c{2} (%F:%M:%L): %m%n +log4j.appender.warn.Threshold = WARN +log4j.appender.warn.Append=true +log4j.appender.warn.File=${log.dir}/${log.file.name.prefix}${log.file.warn.name.postfix} + + +log4j.logger.error=error +log4j.appender.error=org.apache.log4j.RollingFileAppender +log4j.appender.error.MaxFileSize=100MB +log4j.appender.error.MaxBackupIndex=10 +log4j.appender.error.BufferedIO=false +log4j.appender.error.BufferSize=8192 +log4j.appender.error.layout=org.apache.log4j.PatternLayout +log4j.appender.error.layout.ConversionPattern=%d{ISO8601} %p %c{2} (%F:%M:%L): %m%n +log4j.appender.error.Threshold = ERROR +log4j.appender.error.Append=true +log4j.appender.error.File=${log.dir}/${log.file.name.prefix}${log.file.error.name.postfix} diff --git a/inlong-dataproxy/dataproxy-dist/pom.xml b/inlong-dataproxy/dataproxy-dist/pom.xml new file mode 100644 index 00000000000..04ac5f7b69f --- /dev/null +++ b/inlong-dataproxy/dataproxy-dist/pom.xml @@ -0,0 +1,71 @@ + + + + + + org.apache.inlong + inlong-dataproxy + 0.9.0-incubating-SNAPSHOT + + 4.0.0 + + flume-ng-dist + pom + Apache InLong - DataProxy Dist + + + 8 + 8 + + + + + ${project.groupId} + flume-ng-source-dataproxy + ${project.version} + + + + + + + org.apache.maven.plugins + maven-assembly-plugin + 2.5.3 + + + assemble + package + + single + + + apache-inlong-proxy-${project.version} + + src/main/assembly/bin.xml + + gnu + + + + + + + \ No newline at end of file diff --git a/inlong-dataproxy/dataproxy-dist/src/main/assembly/bin.xml b/inlong-dataproxy/dataproxy-dist/src/main/assembly/bin.xml new file mode 100644 index 00000000000..1aaf745b6ff --- /dev/null +++ b/inlong-dataproxy/dataproxy-dist/src/main/assembly/bin.xml @@ -0,0 +1,110 @@ + + + + + bin + + + dir + tar.gz + + + apache-inlong-proxy-${project.version}-bin + + + + lib + false + true + + org.apache.flume.flume-ng-clients:flume-ng-log4jappender:jar:jar-with-dependencies + + + + tools + false + true + + org.apache.flume.flume-ng-clients:flume-ng-log4jappender:jar:jar-with-dependencies + + + + + + + + ../ + + + flume-ng-configuration/** + flume-ng-sdk/** + flume-ng-core/** + flume-ng-node/** + flume-ng-dist/** + flume-ng-channels/** + flume-ng-sinks/** + flume-ng-sources/** + flume-ng-legacy-sources/** + flume-ng-clients/** + flume-ng-embedded-agent/** + flume-tools/** + **/target/** + **/.classpath + **/.project + **/.settings/** + lib/** + + + + DEVNOTES + README + LICENSE + NOTICE + CHANGELOG + RELEASE-NOTES + bin/** + conf/** + + + + + ../target/site + + /** + + docs + + + + conf + conf + + + + bin + bin + + + + + diff --git a/inlong-dataproxy/dataproxy-source/pom.xml b/inlong-dataproxy/dataproxy-source/pom.xml new file mode 100644 index 00000000000..6610705bcca --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/pom.xml @@ -0,0 +1,46 @@ + + + + + + org.apache.inlong + inlong-dataproxy + 0.9.0-incubating-SNAPSHOT + + 4.0.0 + Apache InLong - DataProxy Source + + + org.apache.inlong + tubemq-client + ${tubemq-client.version} + compile + + + + flume-ng-source-dataproxy + + + 8 + 8 + 0.9.0-incubating-SNAPSHOT + + + \ No newline at end of file diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/base/NamedThreadFactory.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/base/NamedThreadFactory.java new file mode 100644 index 00000000000..846c8a1a2a1 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/base/NamedThreadFactory.java @@ -0,0 +1,44 @@ +/* + * 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.inlong.dataproxy.base; + +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class NamedThreadFactory implements ThreadFactory { + + private static final Logger LOGGER = LoggerFactory.getLogger(NamedThreadFactory.class); + + private final AtomicInteger mThreadNum = new AtomicInteger(1); + + private final String threadType; + + public NamedThreadFactory(String threadType) { + this.threadType = threadType; + } + + @Override + public Thread newThread(Runnable r) { + Thread t = new Thread(r, threadType + "-running-thread-" + mThreadNum.getAndIncrement()); + LOGGER.debug("{} created", t.getName()); + return t; + } +} \ No newline at end of file diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/base/ProxyMessage.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/base/ProxyMessage.java new file mode 100644 index 00000000000..0a0b24c0020 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/base/ProxyMessage.java @@ -0,0 +1,82 @@ +/* + * 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.inlong.dataproxy.base; + +import java.util.Map; + +import org.apache.inlong.dataproxy.consts.AttributeConstants; + +public class ProxyMessage { + + private String bid; + private String topic; + private String tid; + + private Map attributeMap; + + private byte[] data; + + public ProxyMessage(String bid, String tid, Map attributeMap, byte[] data) { + this.bid = bid; + this.tid = tid; + this.attributeMap = attributeMap; + this.data = data; + } + + public String getBid() { + return bid; + } + + public void setBid(String bid) { + this.bid = bid; + this.attributeMap.put(AttributeConstants.BUSINESS_ID, bid); + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public String getTid() { + return tid; + } + + public void setTid(String tid) { + this.tid = tid; + this.attributeMap.put(AttributeConstants.INTERFACE_ID, tid); + } + + public Map getAttributeMap() { + return attributeMap; + } + + public void setAttributeMap(Map attributeMap) { + this.attributeMap = attributeMap; + } + + public byte[] getData() { + return data; + } + + public void setData(byte[] data) { + this.data = data; + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/channel/FailoverChannelSelector.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/channel/FailoverChannelSelector.java new file mode 100644 index 00000000000..d684cc123eb --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/channel/FailoverChannelSelector.java @@ -0,0 +1,143 @@ +/* + * 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.inlong.dataproxy.channel; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; +import org.apache.flume.Channel; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.FlumeException; +import org.apache.flume.channel.AbstractChannelSelector; +import org.apache.inlong.dataproxy.consts.ConfigConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FailoverChannelSelector extends AbstractChannelSelector { + + private static final Logger LOG = LoggerFactory.getLogger(FailoverChannelSelector.class); + + private static final String SELECTOR_PROPS = "selector."; + private static final String MASTER_CHANNEL = "master"; + private static final String TRANSFER_CHANNEL = "transfer"; + private static final String FILE_METRIC_CHANNEL = "fileMetric"; + private static final String SLA_METRIC_CHANNEL = "slaMetric"; + + private int masterIndex = 0; + private int slaveIndex = 0; + + private final List masterChannels = new ArrayList(); + private final List slaveChannels = new ArrayList(); + private final List transferChannels = new ArrayList(); + private final List agentFileMetricChannels = new ArrayList(); + private final List slaMetricChannels = new ArrayList(); + + + @Override + public List getRequiredChannels(Event event) { + List retChannels = new ArrayList(); + if (event.getHeaders().containsKey(ConfigConstants.TRANSFER_KEY)) { + retChannels.add(transferChannels.get(0)); + } else if (event.getHeaders().containsKey(ConfigConstants.FILE_CHECK_DATA)) { + retChannels.add(agentFileMetricChannels.get(0)); + } else if (event.getHeaders().containsKey(ConfigConstants.SLA_METRIC_DATA)) { + + + retChannels.add(slaMetricChannels.get(0)); + } else { + retChannels.add(masterChannels.get(masterIndex)); + masterIndex = (masterIndex + 1) % masterChannels.size(); + } + return retChannels; + } + + @Override + public List getOptionalChannels(Event event) { + List retChannels = new ArrayList(); + if (event.getHeaders().containsKey(ConfigConstants.TRANSFER_KEY)) { + retChannels.add(transferChannels.get(0)); + } else if (event.getHeaders().containsKey(ConfigConstants.FILE_CHECK_DATA)) { + retChannels.add(agentFileMetricChannels.get(0)); + } else if (event.getHeaders().containsKey(ConfigConstants.SLA_METRIC_DATA)) { + + retChannels.add(slaMetricChannels.get(1)); + } else { + retChannels.add(slaveChannels.get(slaveIndex)); + slaveIndex = (slaveIndex + 1) % slaveChannels.size(); + } + return retChannels; + } + + /** + * split channel name into name list. + * + * @param channelName - channel name + * @return - name list + */ + private List splitChannelName(String channelName) { + List fileMetricList = new ArrayList(); + if (StringUtils.isEmpty(channelName)) { + LOG.info("channel name is null!"); + } else { + fileMetricList = Arrays.asList(channelName.split("\\s+")); + } + return fileMetricList; + } + + @Override + public void configure(Context context) { +// LOG.info(context.toString()); + String masters = context.getString(MASTER_CHANNEL); + String transfer = context.getString(TRANSFER_CHANNEL); + String fileMertic = context.getString(FILE_METRIC_CHANNEL); + String slaMetric = context.getString(SLA_METRIC_CHANNEL); + if (StringUtils.isEmpty(masters)) { + throw new FlumeException("master channel is null!"); + } + List masterList = splitChannelName(masters); + List transferList = splitChannelName(transfer); + List fileMetricList = splitChannelName(fileMertic); + List slaMetricList = splitChannelName(slaMetric); + + + for (Map.Entry entry : getChannelNameMap().entrySet()) { + String channelName = entry.getKey(); + Channel channel = entry.getValue(); + if (masterList.contains(channelName)) { + this.masterChannels.add(channel); + } else if (transferList.contains(channelName)) { + this.transferChannels.add(channel); + } else if (fileMetricList.contains(channelName)) { + this.agentFileMetricChannels.add(channel); + } else if (slaMetricList.contains(channelName)) { + this.slaMetricChannels.add(channel); + } else { + this.slaveChannels.add(channel); + } + } + LOG.info("masters:" + this.masterChannels); + LOG.info("slaves:" + this.slaveChannels); + LOG.info("transfers:" + this.transferChannels); + LOG.info("agentFileMetrics:" + this.agentFileMetricChannels); + LOG.info("slaMetrics:" + this.slaMetricChannels); + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/AuthUtils.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/AuthUtils.java new file mode 100644 index 00000000000..aa0a1beaa58 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/AuthUtils.java @@ -0,0 +1,62 @@ +/* + * 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.inlong.dataproxy.config; + +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.security.SecureRandom; + +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.codec.binary.StringUtils; +import org.apache.commons.codec.digest.HmacUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class AuthUtils { + + private static final Logger LOG = LoggerFactory.getLogger(AuthUtils.class); + + public static String genAuthToken(final String userName, final String usrPassWord) { + long timestamp = System.currentTimeMillis(); + int nonce = + new SecureRandom(StringUtils.getBytesUtf8(String.valueOf(timestamp))) + .nextInt(Integer.MAX_VALUE); + String signature = getAuthSignature(userName, usrPassWord, timestamp, nonce); + return "manager" + " " + userName + " " + timestamp + " " + nonce + " " + signature; + } + + private static String getAuthSignature(final String usrName, + final String usrPassWord, + long timestamp, int randomValue) { + Base64 base64 = new Base64(); + StringBuilder sbuf = new StringBuilder(512); + byte[] baseStr = + base64.encode(HmacUtils.hmacSha1(usrPassWord, + sbuf.append(usrName).append(timestamp).append(randomValue).toString())); + sbuf.delete(0, sbuf.length()); + String signature = ""; + try { + signature = URLEncoder.encode(new String(baseStr, StandardCharsets.UTF_8), "UTF-8"); + } catch (UnsupportedEncodingException e) { + LOG.error("exception caught", e); + } + return signature; + } + +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/ConfigHolder.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/ConfigHolder.java new file mode 100644 index 00000000000..7936b5eba84 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/ConfigHolder.java @@ -0,0 +1,138 @@ +/* + * 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.inlong.dataproxy.config; + +import static org.apache.inlong.dataproxy.config.ConfigManager.CONFIG_HOLDER_LIST; + +import com.google.common.base.Splitter; + +import java.io.File; +import java.net.URL; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.inlong.dataproxy.config.holder.ConfigUpdateCallback; +import org.apache.inlong.dataproxy.consts.AttributeConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class ConfigHolder { + + public static final Splitter.MapSplitter MAP_SPLITTER = + Splitter.on(AttributeConstants.SEPARATOR) + .trimResults().withKeyValueSeparator(AttributeConstants.KEY_VALUE_SEPARATOR); + private static final Logger LOG = LoggerFactory.getLogger(ConfigHolder.class); + private final String fileName; + private final AtomicBoolean fileChanged = new AtomicBoolean(false); + // list of callbacks for this holder + private final List callbackList = new ArrayList(); + private long lastModifyTime; + private String filePath; + private File configFile; + + public ConfigHolder(String fileName) { + this.fileName = fileName; + setFilePath(fileName); + CONFIG_HOLDER_LIST.add(this); + if (configFile != null) { + this.lastModifyTime = configFile.lastModified(); + } + } + + /** + * add callback + * + * @param callback - callback + */ + public void addUpdateCallback(ConfigUpdateCallback callback) { + callbackList.add(callback); + } + + /** + * execute callbacks + */ + public void executeCallbacks() { + for (ConfigUpdateCallback callback : callbackList) { + callback.update(); + } + } + + /** + * load from file to holder + */ + public abstract void loadFromFileToHolder(); + + /** + * check updater + * + * @return - true if updated + */ + public boolean checkAndUpdateHolder() { + if (fileChanged.compareAndSet(true, false) + || (configFile != null && configFile.lastModified() != this.lastModifyTime)) { + if (configFile != null) { + this.lastModifyTime = configFile.lastModified(); + } + LOG.info("file {} has changed, reload from local file agent", getFileName()); + loadFromFileToHolder(); + return true; + } + return false; + } + + public String getFileName() { + return fileName; + } + + /** + * get file name + * + * @return file name with prefix + */ + public String getNextBackupFileName() { + SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHHmmss"); + String dateStr = format.format(new Date(System.currentTimeMillis())); + return getFilePath() + "." + dateStr; + } + + /** + * file name with base path. + * + * @return + */ + public String getFilePath() { + return filePath; + } + + private void setFilePath(String fileName) { + URL url = getClass().getClassLoader().getResource(fileName); + if (url != null) { + this.filePath = url.getPath(); + this.configFile = new File(this.filePath); + LOG.info("set file path lastTime: {}, currentTime: {}", + lastModifyTime, configFile.lastModified()); + } + } + + public AtomicBoolean getFileChanged() { + return fileChanged; + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/ConfigManager.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/ConfigManager.java new file mode 100644 index 00000000000..802c6f06182 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/ConfigManager.java @@ -0,0 +1,327 @@ +/* + * 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.inlong.dataproxy.config; + +import com.google.gson.Gson; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.lang.StringUtils; +import org.apache.http.HttpHeaders; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.util.EntityUtils; +import org.apache.inlong.dataproxy.config.RemoteConfigJson.DataItem; +import org.apache.inlong.dataproxy.config.holder.BidPropertiesHolder; +import org.apache.inlong.dataproxy.config.holder.FileConfigHolder; +import org.apache.inlong.dataproxy.config.holder.MxPropertiesHolder; +import org.apache.inlong.dataproxy.config.holder.PropertiesConfigHolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + */ +public class ConfigManager { + + public static final List CONFIG_HOLDER_LIST = new ArrayList(); + private static final Logger LOG = LoggerFactory.getLogger(ConfigManager.class); + private static volatile boolean isInit = false; + + private static ConfigManager instance = null; + + private final PropertiesConfigHolder commonConfig = + new PropertiesConfigHolder("common.properties"); + private final PropertiesConfigHolder topicConfig = + new PropertiesConfigHolder("topics.properties"); + private final MxPropertiesHolder mxConfig = new MxPropertiesHolder("mx.properties"); + private final BidPropertiesHolder bidConfig = + new BidPropertiesHolder("bid_mapping.properties"); + private final PropertiesConfigHolder dcConfig = + new PropertiesConfigHolder("dc_mapping.properties"); + private final PropertiesConfigHolder transferConfig = + new PropertiesConfigHolder("transfer.properties"); + private final PropertiesConfigHolder tubeSwitchConfig = + new PropertiesConfigHolder("tube_switch.properties"); + private final PropertiesConfigHolder weightHolder = + new PropertiesConfigHolder("weight.properties"); + private final FileConfigHolder blackListConfig = + new FileConfigHolder("blacklist.properties"); + + /** + * get instance for manager + * @return + */ + public static ConfigManager getInstance() { + + if (isInit && instance != null) { + return instance; + } + synchronized (ConfigManager.class) { + if (!isInit) { + instance = new ConfigManager(); + + for (ConfigHolder holder : CONFIG_HOLDER_LIST) { + + holder.loadFromFileToHolder(); + } + ReloadConfigWorker reloadProperties = new ReloadConfigWorker(instance); + reloadProperties.setDaemon(true); + reloadProperties.start(); + } + isInit = true; + } + return instance; + } + + + public Map getWeightProperties() { + return weightHolder.getHolder(); + } + + public Map getTopicProperties() { + return topicConfig.getHolder(); + } + + /** + * update old maps, reload local files if changed. + * + * @param result - map pending to be added + * @param holder - property holder + * @param addElseRemove - if add(true) else remove(false) + * @return true if changed else false. + */ + private boolean updatePropertiesHolder(Map result, + PropertiesConfigHolder holder, boolean addElseRemove) { + Map tmpHolder = holder.forkHolder(); + boolean changed = false; + for (Map.Entry entry : result.entrySet()) { + String oldValue = addElseRemove + ? tmpHolder.put(entry.getKey(), entry.getValue()) : tmpHolder.remove(entry.getKey()); + // if addElseRemove is false, that means removing item, changed is true. + if (oldValue == null || !oldValue.equals(entry.getValue()) || !addElseRemove) { + changed = true; + } + } + + if (changed) { + return holder.loadFromHolderToFile(tmpHolder); + } else { + return false; + } + } + + public boolean addTopicProperties(Map result) { + return updatePropertiesHolder(result, topicConfig, true); + } + + public boolean deleteTopicProperties(Map result) { + return updatePropertiesHolder(result, topicConfig, false); + } + + public Map getMxProperties() { + return mxConfig.getHolder(); + } + + public boolean addMxProperties(Map result) { + return updatePropertiesHolder(result, mxConfig, true); + } + + public boolean deleteMxProperties(Map result) { + return updatePropertiesHolder(result, mxConfig, false); + } + + public Map getDcMappingProperties() { + return dcConfig.getHolder(); + } + + public Map getTransferProperties() { + return transferConfig.getHolder(); + } + + public Map getTubeSwitchProperties() { + return tubeSwitchConfig.getHolder(); + } + + public Map> getMxPropertiesMaps() { + return mxConfig.getMxPropertiesMaps(); + } + + public Map getBidMappingProperties() { + return bidConfig.getBidMappingProperties(); + } + + public Map> getTidMappingProperties() { + return bidConfig.getTidMappingProperties(); + } + + public Map getBidEnableMappingProperties() { + return bidConfig.getBidEnableMappingProperties(); + } + + public Map getCommonProperties() { + return commonConfig.getHolder(); + } + + public PropertiesConfigHolder getTopicConfig() { + return topicConfig; + } + + /** + * load worker + */ + private static class ReloadConfigWorker extends Thread { + + private static final Logger LOG = LoggerFactory.getLogger(ReloadConfigWorker.class); + private final ConfigManager configManager; + private final CloseableHttpClient httpClient; + private final Gson gson = new Gson(); + private boolean isRunning = true; + + public ReloadConfigWorker(ConfigManager managerInstance) { + this.configManager = managerInstance; + this.httpClient = constructHttpClient(); + } + + private synchronized CloseableHttpClient constructHttpClient() { + long timeoutInMs = TimeUnit.MILLISECONDS.toMillis(50000); + RequestConfig requestConfig = RequestConfig.custom() + .setConnectTimeout((int) timeoutInMs) + .setSocketTimeout((int) timeoutInMs).build(); + HttpClientBuilder httpClientBuilder = HttpClientBuilder.create(); + httpClientBuilder.setDefaultRequestConfig(requestConfig); + return httpClientBuilder.build(); + } + + public int getRandom(int min, int max) { + return (int) (Math.random() * (max + 1 - min)) + min; + } + + private long getSleepTime() { + String sleepTimeInMsStr = + configManager.getCommonProperties().get("configCheckInterval"); + long sleepTimeInMs = 10000; + try { + if (sleepTimeInMsStr != null) { + sleepTimeInMs = Long.parseLong(sleepTimeInMsStr); + } + } catch (Exception ignored) { + LOG.info("ignored Exception ", ignored); + } + return sleepTimeInMs + getRandom(0, 5000); + } + + public void close() { + isRunning = false; + } + + private void checkLocalFile() { + + + for (ConfigHolder holder : CONFIG_HOLDER_LIST) { + + + boolean isChanged = holder.checkAndUpdateHolder(); + if (isChanged) { + holder.executeCallbacks(); + } + } + } + + private boolean checkWithManager(String host) { + HttpGet httpGet = null; + try { + String url = "http://" + host + "/openapi/inlong/manager/dataproxy/getConfig/" + + configManager.getCommonProperties().get("cluster_id"); + LOG.info("start to request {} to get config info", url); + httpGet = new HttpGet(url); + httpGet.addHeader(HttpHeaders.CONNECTION, "close"); + + // request with post + CloseableHttpResponse response = httpClient.execute(httpGet); + String returnStr = EntityUtils.toString(response.getEntity()); + // get bid <-> topic and m value. + + RemoteConfigJson configJson = gson.fromJson(returnStr, RemoteConfigJson.class); + Map bidToTopic = new HashMap(); + Map bidToMValue = new HashMap(); + + if (configJson.getErrCode() == 0) { + for (DataItem item : configJson.getData()) { + bidToMValue.put(item.getBid(), item.getM()); + bidToTopic.put(item.getBid(), item.getTopic()); + } + configManager.addMxProperties(bidToMValue); + configManager.addTopicProperties(bidToTopic); + } + } catch (Exception ex) { + LOG.error("exception caught", ex); + return false; + } finally { + if (httpGet != null) { + httpGet.releaseConnection(); + } + } + return true; + } + + private void checkRemoteConfig() { + + try { + String managerHosts = configManager.getCommonProperties().get("manager_hosts"); + String[] hostList = StringUtils.split(managerHosts, ","); + for (String host : hostList) { + + if (checkWithManager(host)) { + break; + } + } + } catch (Exception ex) { + LOG.error("exception caught", ex); + } + } + + @Override + public void run() { + long count = 0; + while (isRunning) { + + long sleepTimeInMs = getSleepTime(); + count += 1; + try { + checkLocalFile(); + if (count % 30 == 0) { + + checkRemoteConfig(); + count = 0; + } + TimeUnit.MILLISECONDS.sleep(sleepTimeInMs); + } catch (Exception ex) { + LOG.error("exception caught", ex); + } + } + } + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/RemoteConfigJson.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/RemoteConfigJson.java new file mode 100644 index 00000000000..44e932e1b79 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/RemoteConfigJson.java @@ -0,0 +1,54 @@ +/* + * 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.inlong.dataproxy.config; + +import java.util.List; + +public class RemoteConfigJson { + + private boolean result; + private List data; + private int errCode; + + public List getData() { + return data; + } + + public int getErrCode() { + return errCode; + } + + public static class DataItem { + + private String bid; + private String topic; + private String m; + + public String getBid() { + return bid; + } + + public String getTopic() { + return topic; + } + + public String getM() { + return m; + } + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/BidPropertiesHolder.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/BidPropertiesHolder.java new file mode 100644 index 00000000000..7330801278f --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/BidPropertiesHolder.java @@ -0,0 +1,88 @@ +/* + * 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.inlong.dataproxy.config.holder; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * bid to m value + */ +public class BidPropertiesHolder extends PropertiesConfigHolder { + + private static final Logger LOG = LoggerFactory.getLogger(BidPropertiesHolder.class); + private static final String BID_VALUE_SPLITTER = "#"; + + + private Map bidMappingProperties = + new HashMap(); + private Map> tidMappingProperties = + new HashMap>(); + private Map bidEnableMappingProperties = + new HashMap(); + + public BidPropertiesHolder(String fileName) { + super(fileName); + } + + @Override + public void loadFromFileToHolder() { + super.loadFromFileToHolder(); + try { + Map tmpBidMappingProperties = + new HashMap(); + Map> tmpTidMappingProperties = + new HashMap>(); + Map tmpBidEnableMappingProperties = new HashMap(); + for (Map.Entry entry : super.getHolder().entrySet()) { + String[] sArray = StringUtils.split(entry.getKey(), BID_VALUE_SPLITTER); + if (sArray.length != 3) { + LOG.warn("invalid bid key {}", entry.getKey()); + continue; + } + tmpBidMappingProperties.put(sArray[0].trim(), sArray[1].trim()); + tmpBidEnableMappingProperties.put(sArray[0].trim(), sArray[2].trim()); + if (StringUtils.isNotBlank(entry.getValue())) { + tmpTidMappingProperties.put(sArray[0].trim(), + MAP_SPLITTER.split(entry.getValue())); + } + } + bidMappingProperties = tmpBidMappingProperties; + tidMappingProperties = tmpTidMappingProperties; + bidEnableMappingProperties = tmpBidEnableMappingProperties; + } catch (Exception e) { + LOG.error("loadConfig error :", e); + } + } + + public Map getBidMappingProperties() { + return bidMappingProperties; + } + + public Map> getTidMappingProperties() { + return tidMappingProperties; + } + + public Map getBidEnableMappingProperties() { + return bidEnableMappingProperties; + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/ConfigUpdateCallback.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/ConfigUpdateCallback.java new file mode 100644 index 00000000000..723d366ebd9 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/ConfigUpdateCallback.java @@ -0,0 +1,29 @@ +/* + * 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.inlong.dataproxy.config.holder; + +/** + * update callback + */ +public interface ConfigUpdateCallback { + + /** + * update + */ + void update(); +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/FileConfigHolder.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/FileConfigHolder.java new file mode 100644 index 00000000000..4ea771a93a1 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/FileConfigHolder.java @@ -0,0 +1,94 @@ +/* + * 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.inlong.dataproxy.config.holder; + +import java.io.BufferedReader; +import java.io.FileReader; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.commons.io.IOUtils; +import org.apache.inlong.dataproxy.config.ConfigHolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * save to list + */ +public class FileConfigHolder extends ConfigHolder { + + private static final Logger LOG = LoggerFactory.getLogger(FileConfigHolder.class); + private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); + private List holder; + + public FileConfigHolder(String fileName) { + super(fileName); + holder = new ArrayList(); + } + + @Override + public void loadFromFileToHolder() { + readWriteLock.readLock().lock(); + try { + List tmpHolder = loadFile(); + LOG.info(getFileName() + " load content {}", tmpHolder); + holder = tmpHolder; + } finally { + readWriteLock.readLock().unlock(); + } + } + + /** + * deep copy holder + * + * @return + */ + public List forkHolder() { + List tmpHolder = new ArrayList(); + if (holder != null) { + tmpHolder.addAll(holder); + } + return tmpHolder; + } + + private List loadFile() { + ArrayList arrayList = new ArrayList(); + FileReader reader = null; + BufferedReader br = null; + try { + reader = new FileReader(getFilePath()); + br = new BufferedReader(reader); + String line; + while ((line = br.readLine()) != null) { + arrayList.add(line); + } + } catch (Exception e) { + LOG.error("fail to load file, file ={}, and e= {}", getFilePath(), e); + } finally { + IOUtils.closeQuietly(reader); + IOUtils.closeQuietly(br); + } + return arrayList; + } + + public List getHolder() { + return holder; + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/MxPropertiesHolder.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/MxPropertiesHolder.java new file mode 100644 index 00000000000..f0dbb988edb --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/MxPropertiesHolder.java @@ -0,0 +1,57 @@ +/* + * 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.inlong.dataproxy.config.holder; + +import java.util.HashMap; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * value is map + */ +public class MxPropertiesHolder extends PropertiesConfigHolder { + + private static final Logger LOG = LoggerFactory.getLogger(MxPropertiesHolder.class); + private final Map> mxPropertiesMaps = + new HashMap>(); + + public MxPropertiesHolder(String fileName) { + super(fileName); + } + + /** + * load m from file + */ + @Override + public void loadFromFileToHolder() { + super.loadFromFileToHolder(); + try { + for (Map.Entry entry : getHolder().entrySet()) { + mxPropertiesMaps.put(entry.getKey(), MAP_SPLITTER.split(entry.getValue())); + } + } catch (Exception e) { + LOG.error("loadConfig error :", e); + } + } + + public Map> getMxPropertiesMaps() { + return mxPropertiesMaps; + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/PropertiesConfigHolder.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/PropertiesConfigHolder.java new file mode 100644 index 00000000000..9b5cd0b5428 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/holder/PropertiesConfigHolder.java @@ -0,0 +1,154 @@ +/* + * 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.inlong.dataproxy.config.holder; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.UnsupportedEncodingException; +import java.net.URL; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.commons.io.FileUtils; +import org.apache.inlong.dataproxy.config.ConfigHolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * properties to map + */ +public class PropertiesConfigHolder extends ConfigHolder { + + private static final Logger LOG = LoggerFactory.getLogger(PropertiesConfigHolder.class); + private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); + private Map holder; + + public PropertiesConfigHolder(String fileName) { + super(fileName); + } + + @Override + public void loadFromFileToHolder() { + readWriteLock.readLock().lock(); + try { + Map tmpHolder = loadProperties(); + LOG.info(getFileName() + " load content {}", tmpHolder); + holder = tmpHolder; + } finally { + readWriteLock.readLock().unlock(); + } + } + + /** + * holder + * + * @return + */ + public Map forkHolder() { + Map tmpHolder = new HashMap(); + if (holder != null) { + for (Map.Entry entry : holder.entrySet()) { + tmpHolder.put(entry.getKey(), entry.getValue()); + } + } + return tmpHolder; + } + + private List getStringListFromHolder(Map tmpHolder) { + List result = new ArrayList(); + for (Map.Entry entry : tmpHolder.entrySet()) { + result.add(entry.getKey() + "=" + entry.getValue()); + } + return result; + } + + /** + * load from holder + * @param tmpHolder + * @return + */ + public boolean loadFromHolderToFile(Map tmpHolder) { + readWriteLock.writeLock().lock(); + boolean isSuccess = false; + try { + File sourceFile = new File(getFilePath()); + File targetFile = new File(getNextBackupFileName()); + File tmpNewFile = new File(getFileName() + ".tmp"); + + if (sourceFile.exists()) { + FileUtils.copyFile(sourceFile, targetFile); + } + + + List lines = getStringListFromHolder(tmpHolder); + FileUtils.writeLines(tmpNewFile, lines); + + + FileUtils.copyFile(tmpNewFile, sourceFile); + tmpNewFile.delete(); + isSuccess = true; + getFileChanged().set(true); + } catch (Exception ex) { + LOG.error("error in writing file", ex); + } finally { + readWriteLock.writeLock().unlock(); + } + return isSuccess; + } + + protected Map loadProperties() { + Map result = new HashMap(); + InputStream inStream = null; + try { + URL url = getClass().getClassLoader().getResource(getFileName()); + inStream = url != null ? url.openStream() : null; + + if (inStream == null) { + LOG.error("InputStream {} is null!", getFileName()); + } + Properties props = new Properties(); + props.load(inStream); + for (Map.Entry entry : props.entrySet()) { + result.put((String) entry.getKey(), (String) entry.getValue()); + } + } catch (UnsupportedEncodingException e) { + LOG.error("fail to load properties, file ={}, and e= {}", getFileName(), e); + } catch (Exception e) { + LOG.error("fail to load properties, file ={}, and e= {}", getFileName(), e); + } finally { + if (null != inStream) { + try { + inStream.close(); + } catch (IOException e) { + LOG.error("fail to loadTopics, inStream.close ,and e= {}", getFileName(), e); + } + } + } + return result; + } + + public Map getHolder() { + return holder; + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/remote/ConfigMessageServlet.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/remote/ConfigMessageServlet.java new file mode 100644 index 00000000000..168ea876fbf --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/remote/ConfigMessageServlet.java @@ -0,0 +1,129 @@ +/* + * 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.inlong.dataproxy.config.remote; + +import com.google.gson.Gson; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.PrintWriter; +import java.util.HashMap; +import java.util.Map; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +import org.apache.commons.io.IOUtils; +import org.apache.inlong.dataproxy.config.ConfigManager; +import org.apache.inlong.dataproxy.http.StatusCode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * http + */ +public class ConfigMessageServlet extends HttpServlet { + + private static final Logger LOG = LoggerFactory.getLogger(ConfigMessageServlet.class); + private static final ConfigManager configManager = ConfigManager.getInstance(); + + private final Gson gson = new Gson(); + + public ConfigMessageServlet() { + } + + @Override + protected void doGet( + HttpServletRequest req, HttpServletResponse resp) throws IOException { + doPost(req, resp); + } + + private boolean handleTopicConfig(RequestContent requestContent) { + Map bidToTopic = new HashMap(); + for (Map item : requestContent.getContent()) { + bidToTopic.put(item.get("bid"), item.get("topic")); + } + if ("add".equals(requestContent.getOperationType())) { + return configManager.addTopicProperties(bidToTopic); + } else if ("delete".equals(requestContent.getOperationType())) { + return configManager.deleteTopicProperties(bidToTopic); + } + return false; + } + + private boolean handleMxConfig(RequestContent requestContent) { + Map bidToMValue = new HashMap(); + for (Map item : requestContent.getContent()) { + bidToMValue.put(item.get("bid"), item.get("m")); + } + if ("add".equals(requestContent.getOperationType())) { + return configManager.addMxProperties(bidToMValue); + } else if ("delete".equals(requestContent.getOperationType())) { + return configManager.deleteMxProperties(bidToMValue); + } + return false; + } + + private void responseToJson(HttpServletResponse response, + ResponseResult result) throws IOException { + response.setContentType("application/json"); + response.setCharacterEncoding("UTF-8"); + String jsonStr = gson.toJson(result); + PrintWriter out = response.getWriter(); + out.print(jsonStr); + out.flush(); + } + + @Override + protected void doPost(HttpServletRequest req, HttpServletResponse resp) throws IOException { + ResponseResult result = new ResponseResult(StatusCode.SERVICE_ERR, ""); + BufferedReader reader = null; + try { + reader = req.getReader(); + boolean isSuccess = false; + RequestContent requestContent = gson.fromJson(IOUtils.toString(reader), + RequestContent.class); + if (requestContent.getRequestType() != null + && requestContent.getOperationType() != null) { + if ("topic".equals(requestContent.getRequestType())) { + isSuccess = handleTopicConfig(requestContent); + } else if ("mx".equals(requestContent.getRequestType())) { + isSuccess = handleMxConfig(requestContent); + } + } else { + result.setMessage("request format is not valid"); + } + + if (isSuccess) { + result.setCode(StatusCode.SUCCESS); + } else { + result.setMessage("cannot operate config update, please check it"); + } + + } catch (Exception ex) { + LOG.error("error while do post", ex); + result.setMessage(ex.getMessage()); + } finally { + if (reader != null) { + reader.close(); + } + } + responseToJson(resp, result); + } + +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/remote/RequestContent.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/remote/RequestContent.java new file mode 100644 index 00000000000..46e8af4f363 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/remote/RequestContent.java @@ -0,0 +1,55 @@ +/* + * 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.inlong.dataproxy.config.remote; + +import java.util.List; +import java.util.Map; + +/** + * json str to java + */ +public class RequestContent { + + private String requestType; + private String operationType; + private List> content; + + public String getRequestType() { + return requestType; + } + + public void setRequestType(String requestType) { + this.requestType = requestType; + } + + public String getOperationType() { + return operationType; + } + + public void setOperationType(String operationType) { + this.operationType = operationType; + } + + public List> getContent() { + return content; + } + + public void setContent(List> content) { + this.content = content; + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/remote/ResponseResult.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/remote/ResponseResult.java new file mode 100644 index 00000000000..e65e03a5634 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/config/remote/ResponseResult.java @@ -0,0 +1,48 @@ +/* + * 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.inlong.dataproxy.config.remote; + +/** + * json str to client + */ +public class ResponseResult { + + private int code; + private String message; + + public ResponseResult(int code, String message) { + this.code = code; + this.message = message; + } + + public int getCode() { + return code; + } + + public void setCode(int code) { + this.code = code; + } + + public String getMessage() { + return message; + } + + public void setMessage(String message) { + this.message = message; + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/consts/AttributeConstants.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/consts/AttributeConstants.java new file mode 100644 index 00000000000..bb63a7b4816 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/consts/AttributeConstants.java @@ -0,0 +1,83 @@ +/* + * 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.inlong.dataproxy.consts; + +public interface AttributeConstants { + + String SEPARATOR = "&"; + String KEY_VALUE_SEPARATOR = "="; + + String REQUEST_TYPE = "requestType"; + + String OPERATION_TYPE = "operationType"; + + String OPERATION_CONTENT = "content"; + + /** + * business id unique string id for each business or product + */ + String BUSINESS_ID = "bid"; + + /** + * interface id unique string id for each interface of business An interface stand for a kind of + * data + */ + String INTERFACE_ID = "tid"; + + /** + * iname is like a tid but used in file protocol(m=xxx) + */ + String INAME = "iname"; + + /** + * data time + */ + String DATA_TIME = "dt"; + + String TIME_STAMP = "t"; + + /* compress type */ + String COMPRESS_TYPE = "cp"; + + /* count value for how many records a message body contains */ + String MESSAGE_COUNT = "cnt"; + + /* message type */ + String MESSAGE_TYPE = "mt"; + + /* sort type */ + String METHOD = "m"; + + /* global unique id for a message*/ + String SEQUENCE_ID = "sid"; + + String UNIQ_ID = "uniq"; + + /* from where */ + String FROM = "f"; + + String RCV_TIME = "rt"; + + String NODE_IP = "NodeIP"; + + String NUM2NAME = "num2name"; + + String BID_NUM = "bidnum"; + + String TID_NUM = "tidnum"; +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/consts/ConfigConstants.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/consts/ConfigConstants.java new file mode 100644 index 00000000000..b9063864d93 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/consts/ConfigConstants.java @@ -0,0 +1,116 @@ +/* + * 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.inlong.dataproxy.consts; + +public class ConfigConstants { + + public static final String CONFIG_PORT = "port"; + + public static final String CONFIG_HOST = "host"; + + public static final String MSG_FACTORY_NAME = "msg-factory-name"; + + public static final String SERVICE_PROCESSOR_NAME = "service-decoder-name"; + + public static final String MESSAGE_HANDLER_NAME = "message-handler-name"; + + public static final String MAX_MSG_LENGTH = "max-msg-length"; + + public static final String MSG_COMPRESSED = "msg-compressed"; + + public static final String TOPIC = "topic"; + + public static final String ATTR = "attr"; + + public static final String FILTER_EMPTY_MSG = "filter-empty-msg"; + + public static final String TCP_NO_DELAY = "tcpNoDelay"; + + public static final String KEEP_ALIVE = "keepAlive"; + + public static final String HIGH_WATER_MARK = "highWaterMark"; + + public static final String RECEIVE_BUFFER_SIZE = "receiveBufferSize"; + public static final String ENABLE_EXCEPTION_RETURN = "enableExceptionReturn"; + + public static final String SEND_BUFFER_SIZE = "sendBufferSize"; + + public static final String TRAFFIC_CLASS = "trafficClass"; + + public static final String MAX_THREADS = "max-threads"; + + public static final String STAT_INTERVAL_SEC = "stat-interval-sec"; + + public static final String HEART_INTERVAL_SEC = "heart-interval-sec"; + + public static final String PACKAGE_TIMEOUT_SEC = "package-timeout-sec"; + + public static final String HEART_SERVERS = "heart-servers"; + + public static final String CUSTOM_CHANNEL_PROCESSOR = "custom-cp"; + + public static final String UDP_PROTOCOL = "udp"; + public static final String TCP_PROTOCOL = "tcp"; + + public static final String TOPIC_KEY = "topic"; + public static final String REMOTE_IP_KEY = "srcIp"; + public static final String REMOTE_IDC_KEY = "idc"; + public static final String MSG_COUNTER_KEY = "msgcnt"; + public static final String PKG_COUNTER_KEY = "pkgcnt"; + public static final String PKG_TIME_KEY = "msg.pkg.time"; + public static final String TRANSFER_KEY = "transfer"; + public static final String DEST_IP_KEY = "dstIp"; + public static final String INTERFACE_KEY = "interface"; + public static final String SINK_MIN_METRIC_KEY = "sink-min-metric-topic"; + public static final String SINK_HOUR_METRIC_KEY = "sink-hour-metric-topic"; + public static final String SINK_TEN_METRIC_KEY = "sink-ten-metric-topic"; + public static final String SINK_QUA_METRIC_KEY = "sink-qua-metric-topic"; + public static final String L5_MIN_METRIC_KEY = "l5-min-metric-topic"; + public static final String L5_MIN_FAIL_METRIC_KEY = "l5-min-fail-metric-key"; + public static final String L5_HOUR_METRIC_KEY = "l5-hour-metric-topic"; + public static final String L5_ID_KEY = "l5id"; + public static final String SET_KEY = "set"; + public static final String CLUSTER_ID_KEY = "clusterId"; + + public static final String DECODER_BODY = "body"; + public static final String DECODER_TOPICKEY = "topic_key"; + public static final String DECODER_ATTRS = "attrs"; + public static final String MSG_TYPE = "msg_type"; + public static final String COMPRESS_TYPE = "compress_type"; + public static final String EXTRA_ATTR = "extra_attr"; + public static final String COMMON_ATTR_MAP = "common_attr_map"; + public static final String MSG_LIST = "msg_list"; + public static final String VERSION_TYPE = "version"; + public static final String FILE_CHECK_DATA = "file-check-data"; + public static final String MINUTE_CHECK_DATA = "minute-check-data"; + public static final String SLA_METRIC_DATA = "sla-metric-data"; + public static final String SLA_METRIC_BID = "manager_sla_metric"; + + public static final String FILE_BODY = "file-body"; + public static final int MSG_MAX_LENGTH_BYTES = 20 * 1024 * 1024; + + public static final String SEQUENCE_ID = "sequencial_id"; + + public static final String TOTAL_LEN = "totalLen"; + + public static final String LINK_MAX_ALLOWED_DELAYED_MSG_COUNT = "link_max_allowed_delayed_msg_count"; + public static final String SESSION_WARN_DELAYED_MSG_COUNT = "session_warn_delayed_msg_count"; + public static final String SESSION_MAX_ALLOWED_DELAYED_MSG_COUNT = "session_max_allowed_delayed_msg_count"; + public static final String NETTY_WRITE_BUFFER_HIGH_WATER_MARK = "netty_write_buffer_high_water_mark"; + public static final String RECOVER_THREAD_COUNT = "recover_thread_count"; +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/consts/StatConstants.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/consts/StatConstants.java new file mode 100644 index 00000000000..b967e43f55e --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/consts/StatConstants.java @@ -0,0 +1,40 @@ +/* + * 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.inlong.dataproxy.consts; + +public class StatConstants { + + public static final java.lang.String METASINK_SUCCESS = "metasink.success"; + public static final java.lang.String METASINK_DROPPED = "metasink.dropped"; + public static final java.lang.String METASINK_RETRY = "metasink.retry"; + public static final java.lang.String METASINK_OTHEREXP = "metasink.otherexp"; + public static final java.lang.String METASINK_NOTOPIC = "metasink.notopic"; + public static final java.lang.String METASINK_NOSLAVE = "metasink.noslave"; + public static final java.lang.String METASINK_MSG_NOTOPIC = "metasink.msgnotopic"; + public static final java.lang.String METASINK_PROCESS_SPEED = "metasink.process.speed"; + public static final java.lang.String EVENT_SUCCESS = "socketmsg.success"; + public static final java.lang.String EVENT_DROPPED = "socketmsg.dropped"; + public static final java.lang.String EVENT_EMPTY = "socketmsg.empty"; + public static final java.lang.String EVENT_OTHEREXP = "socketmsg.otherexp"; + public static final java.lang.String EVENT_INVALID = "socketmsg.invalid"; + public static final java.lang.String AGENT_MESSAGES_SENT_SUCCESS = "agent.messages.success"; + public static final java.lang.String AGENT_PACKAGES_SENT_SUCCESS = "agent.packages.success"; + public static final java.lang.String MSG_COUNTER_KEY = "msgcnt"; + public static final java.lang.String MSG_PKG_TIME_KEY = "msg.pkg.time"; + public static final java.lang.String AGENT_MESSAGES_COUNT_PREFIX = "agent.messages.count."; +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/exception/ErrorCode.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/exception/ErrorCode.java new file mode 100644 index 00000000000..a4ccb979e88 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/exception/ErrorCode.java @@ -0,0 +1,47 @@ +/* + * 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.inlong.dataproxy.exception; + +public enum ErrorCode { + /** + * attr error + */ + ATTR_ERROR(1), + DT_ERROR(2), + COMPRESS_ERROR(3), + OTHER_ERROR(4), + TOO_LONG_ERROR(5); + + private final int value; + + ErrorCode(int value) { + this.value = value; + } + + public int getValue() { + return value; + } + + public byte[] intToByteArray() { + return new byte[]{ + (byte) (value >>> 24), + (byte) (value >>> 16), + (byte) (value >>> 8), + (byte) value}; + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/exception/MessageIDException.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/exception/MessageIDException.java new file mode 100644 index 00000000000..301b0597e9e --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/exception/MessageIDException.java @@ -0,0 +1,39 @@ +/* + * 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.inlong.dataproxy.exception; + +public class MessageIDException extends Exception { + + private final long messageId; + private final ErrorCode errorCode; + + public MessageIDException(long messageId, ErrorCode errorCode, Throwable throwable) { + super(throwable); + this.errorCode = errorCode; + this.messageId = messageId; + } + + public long getMessageId() { + return messageId; + } + + public ErrorCode getErrorCode() { + return errorCode; + } +} + diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/StatusCode.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/StatusCode.java new file mode 100644 index 00000000000..5a9743d3a00 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/http/StatusCode.java @@ -0,0 +1,30 @@ +/* + * 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.inlong.dataproxy.http; + +public class StatusCode { + + + public static final int SUCCESS = 1; + + public static final int ILLEGAL_ARGUMENT = -100; + + public static final int EXCEED_LEN = -101; + + public static final int SERVICE_ERR = -105; +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/EventStat.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/EventStat.java new file mode 100644 index 00000000000..0c67153c0ad --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/EventStat.java @@ -0,0 +1,64 @@ +/* + * 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.inlong.dataproxy.sink; + +import org.apache.flume.Event; + +public class EventStat { + private Event event; + private int myRetryCnt; + + public EventStat(Event event) { + this.event = event; + this.myRetryCnt = 0; + } + + public EventStat(Event event, int retryCnt) { + this.event = event; + this.myRetryCnt = retryCnt; + } + + public Event getEvent() { + return event; + } + + public void setEvent(Event event) { + this.event = event; + } + + public int getRetryCnt() { + return myRetryCnt; + } + + public void setRetryCnt(int retryCnt) { + this.myRetryCnt = retryCnt; + } + + public void incRetryCnt() { + this.myRetryCnt++; + } + + public boolean shouldDrop() { + return false; + } + + public void reset() { + this.event = null; + this.myRetryCnt = 0; + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/IdCacheCleaner.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/IdCacheCleaner.java new file mode 100644 index 00000000000..b726be44106 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/IdCacheCleaner.java @@ -0,0 +1,79 @@ +/* + * 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.inlong.dataproxy.sink; + +import static org.apache.inlong.dataproxy.sink.MetaSink.agentIdMap; +import static org.apache.inlong.dataproxy.sink.MetaSink.idCleanerStarted; + +import java.util.Map; + +import org.apache.flume.Sink; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class IdCacheCleaner extends Thread { + + private static final Logger logger = LoggerFactory.getLogger(IdCacheCleaner.class); + private final long maxSurvivedTime; + private final int maxSurvivedSize; + + public IdCacheCleaner(Sink sink, long maxSurvivedTime, int maxSurvivedSize) { + this.sink = sink; + this.maxSurvivedSize = maxSurvivedSize; + this.maxSurvivedTime = maxSurvivedTime; + } + + private Sink sink; + + @Override + public synchronized void start() { + if (idCleanerStarted) { + return; + } + idCleanerStarted = true; + super.start(); + } + + @Override + public void run() { + while (idCleanerStarted) { + long currentTime = System.currentTimeMillis(); + long survivedTime = maxSurvivedTime; + if (agentIdMap.size() > maxSurvivedSize) { + survivedTime = maxSurvivedTime / 2; + } + logger.info("{} map size is:{};set survive time:{}", sink.getName(), + agentIdMap.size(), survivedTime); + int cleanCount = 0; + for (Map.Entry entry : agentIdMap.entrySet()) { + long idTime = entry.getValue(); + if (currentTime - idTime > survivedTime) { + agentIdMap.remove(entry.getKey()); + cleanCount++; + } + } + logger.info("{} clear {} client ids", sink.getName(), cleanCount); + try { + Thread.sleep(30 * 1000); + } catch (InterruptedException e) { + logger.info("cleaner thread has exit! cache size :", agentIdMap.size()); + return; + } + } + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/MetaSink.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/MetaSink.java new file mode 100644 index 00000000000..a4fb9f5eeb8 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/sink/MetaSink.java @@ -0,0 +1,740 @@ +/* + * 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.inlong.dataproxy.sink; + +import com.google.common.base.Preconditions; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.flume.Channel; +import org.apache.flume.Context; +import org.apache.flume.Event; +import org.apache.flume.EventDeliveryException; +import org.apache.flume.FlumeException; +import org.apache.flume.Transaction; +import org.apache.flume.conf.Configurable; +import org.apache.flume.sink.AbstractSink; +import org.apache.flume.source.shaded.guava.RateLimiter; +import org.apache.inlong.dataproxy.config.ConfigManager; +import org.apache.inlong.dataproxy.config.holder.ConfigUpdateCallback; +import org.apache.inlong.dataproxy.consts.AttributeConstants; +import org.apache.inlong.dataproxy.consts.ConfigConstants; +import org.apache.inlong.dataproxy.utils.NetworkUtils; +import org.apache.inlong.tubemq.client.config.TubeClientConfig; +import org.apache.inlong.tubemq.client.exception.TubeClientException; +import org.apache.inlong.tubemq.client.factory.TubeMultiSessionFactory; +import org.apache.inlong.tubemq.client.producer.MessageProducer; +import org.apache.inlong.tubemq.client.producer.MessageSentCallback; +import org.apache.inlong.tubemq.client.producer.MessageSentResult; +import org.apache.inlong.tubemq.corebase.Message; +import org.apache.inlong.tubemq.corebase.TErrCodeConstants; +import org.apache.inlong.tubemq.corerpc.exception.OverflowException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class MetaSink extends AbstractSink implements Configurable { + + private static final Logger logger = LoggerFactory.getLogger(MetaSink.class); + private static int MAX_TOPICS_EACH_PRODUCER_HOLD = 200; + private static final String TUBE_REQUEST_TIMEOUT = "tube-request-timeout"; + + private static int BAD_EVENT_QUEUE_SIZE = 10000; + + private static final String SINK_THREAD_NUM = "thread-num"; + private static int EVENT_QUEUE_SIZE = 1000; + private volatile boolean canTake = false; + private volatile boolean canSend = false; + private static int BATCH_SIZE = 10000; + private static final int defaultRetryCnt = -1; + private static final int defaultLogEveryNEvents = 100000; + private static final int defaultSendTimeout = 20000; // in millsec + private static final int defaultStatIntervalSec = 60; + private static final int sendNewMetricRetryCount = 3; + + private static String MASTER_HOST_PORT_LIST = "master-host-port-list"; + private static String TOPIC = "topic"; + private static String SEND_TIMEOUT = "send_timeout"; // in millsec + private static String LOG_EVERY_N_EVENTS = "log-every-n-events"; + private static String RETRY_CNT = "retry-currentSuccSendedCnt"; + private static String STAT_INTERVAL_SEC = "stat-interval-sec"; // in sec + private static String MAX_TOPICS_EACH_PRODUCER_HOLD_NAME = "max-topic-each-producer-hold"; + + private static final String LOG_TOPIC = "proxy-log-topic"; + private static final String TID = "proxy-log-tid"; + private static final String BID = "proxy-log-bid"; + private static final String SEND_REMOTE = "send-remote"; + private static final String topicsFilePath = "topics.properties"; + private static final String slaTopicFilePath = "slaTopics.properties"; + private static final String SLA_METRIC_SINK = "sla-metric-sink"; + + private static String MAX_SURVIVED_TIME = "max-survived-time"; + private static String MAX_SURVIVED_SIZE = "max-survived-size"; + private static String CLIENT_ID_CACHE = "client-id-cache"; + + private int maxSurvivedTime = 3 * 1000 * 30; + private int maxSurvivedSize = 100000; + + private String proxyLogTopic = "teg_manager"; + private String proxyLogBid = "b_teg_manager"; + private String proxyLogTid = "proxy_measure_log"; + private boolean sendRemote = false; + private ConfigManager configManager; + private Map topicProperties; + + public MessageProducer producer; + public Map producerMap; + + private LinkedBlockingQueue resendQueue; + private LinkedBlockingQueue eventQueue; + + private long diskIORatePerSec; + private RateLimiter diskRateLimiter; + + public AtomicInteger currentPublishTopicNum = new AtomicInteger(0); + public TubeMultiSessionFactory sessionFactory; + private String masterHostAndPortList; + private Integer logEveryNEvents; + private Integer sendTimeout; + private static int retryCnt = defaultRetryCnt; + private int requestTimeout = 60; + private int threadNum; + private Thread[] sinkThreadPool; + + private String metaTopicFilePath = topicsFilePath; + private long linkMaxAllowedDelayedMsgCount; + private long sessionWarnDelayedMsgCount; + private long sessionMaxAllowedDelayedMsgCount; + private long nettyWriteBufferHighWaterMark; + private int recoverthreadcount; + + private static final LoadingCache agentIdCache = CacheBuilder + .newBuilder().concurrencyLevel(4 * 8).initialCapacity(5000000).expireAfterAccess(30, TimeUnit.SECONDS) + .build(new CacheLoader() { + + @Override + public Long load(String key) { + return System.currentTimeMillis(); + } + }); + + private IdCacheCleaner idCacheCleaner; + protected static boolean idCleanerStarted = false; + protected static final ConcurrentHashMap agentIdMap = + new ConcurrentHashMap(); + private static ConcurrentHashMap illegalTopicMap = + new ConcurrentHashMap(); + + private boolean clientIdCache = false; + private boolean isNewCache = true; + + private boolean overflow = false; + + /** + * diff publish + * + * @param originalSet + * @param endSet + */ + public void diffSetPublish(Set originalSet, Set endSet) { + + boolean changed = false; + for (String s : endSet) { + if (!originalSet.contains(s)) { + changed = true; + try { + producer = getProducer(s); + } catch (Exception e) { + logger.error("Get producer failed!", e); + } + } + } + + if (changed) { + logger.info("topics.properties has changed, trigger diff publish for {}", getName()); + topicProperties = configManager.getTopicProperties(); + } + } + + private MessageProducer getProducer(String topic) throws TubeClientException { + if (producerMap.containsKey(topic)) { + return producerMap.get(topic); + } else { + synchronized (this) { + if (!producerMap.containsKey(topic)) { + if (producer == null || currentPublishTopicNum.get() >= MAX_TOPICS_EACH_PRODUCER_HOLD) { + producer = sessionFactory.createProducer(); + currentPublishTopicNum.set(0); + } + // publish topic + producer.publish(topic); + producerMap.put(topic, producer); + currentPublishTopicNum.incrementAndGet(); + } + } + return producerMap.get(topic); + } + } + + private TubeClientConfig initTubeConfig() throws Exception { + final TubeClientConfig tubeClientConfig = new TubeClientConfig(NetworkUtils.getLocalIp(), + this.masterHostAndPortList); + tubeClientConfig.setLinkMaxAllowedDelayedMsgCount(linkMaxAllowedDelayedMsgCount); + tubeClientConfig.setSessionWarnDelayedMsgCount(sessionWarnDelayedMsgCount); + tubeClientConfig.setSessionMaxAllowedDelayedMsgCount(sessionMaxAllowedDelayedMsgCount); + tubeClientConfig.setNettyWriteBufferHighWaterMark(nettyWriteBufferHighWaterMark); + tubeClientConfig.setHeartbeatPeriodMs(15000L); + tubeClientConfig.setRpcTimeoutMs(20000L); + + + return tubeClientConfig; + } + + /** + * If this function is called successively without calling {@see #destroyConnection()}, only the + * first call has any effect. + * + * @throws FlumeException if an RPC client connection could not be opened + */ + private void createConnection() throws FlumeException { +// synchronized (tubeSessionLock) { + // if already connected, just skip + if (sessionFactory != null) { + return; + } + + try { + TubeClientConfig conf = initTubeConfig(); + //sessionFactory = new TubeMutilMessageSessionFactory(conf); + sessionFactory = new TubeMultiSessionFactory(conf); + } catch (TubeClientException e) { + logger.error("create connnection error in metasink, " + + "maybe tube master set error, please re-check. ex1 {}", e.getMessage()); + throw new FlumeException("connect to Tube error1, " + + "maybe zkstr/zkroot set error, please re-check"); + } catch (Throwable e) { + logger.error("create connnection error in metasink, " + + "maybe tube master set error/shutdown in progress, please re-check. ex2 {}", + e.getMessage()); + throw new FlumeException("connect to meta error2, " + + "maybe tube master set error/shutdown in progress, please re-check"); + } + + if (producerMap == null) { + producerMap = new HashMap(); + } + logger.debug("building tube producer"); +// } + } + + private void destroyConnection() { + for (Map.Entry entry : producerMap.entrySet()) { + MessageProducer producer = entry.getValue(); + try { + producer.shutdown(); + } catch (TubeClientException e) { + logger.error("destroy producer error in metasink, MetaClientException {}", e.getMessage()); + } catch (Throwable e) { + logger.error("destroy producer error in metasink, ex {}", e.getMessage()); + } + } + producerMap.clear(); + + if (sessionFactory != null) { + try { + sessionFactory.shutdown(); + } catch (TubeClientException e) { + logger.error("destroy sessionFactory error in metasink, MetaClientException {}", + e.getMessage()); + } catch (Exception e) { + logger.error("destroy sessionFactory error in metasink, ex {}", e.getMessage()); + } + } + sessionFactory = null; + logger.debug("closed meta producer"); + } + + + private void initTopicSet(Set topicSet) throws Exception { + List sortedList = new ArrayList(topicSet); + Collections.sort(sortedList); + int cycle = sortedList.size() / MAX_TOPICS_EACH_PRODUCER_HOLD; + int remainder = sortedList.size() % MAX_TOPICS_EACH_PRODUCER_HOLD; + long startTime = System.currentTimeMillis(); + for (int i = 0; i <= cycle; i++) { + Set subset = new HashSet(); + int startIndex = i * MAX_TOPICS_EACH_PRODUCER_HOLD; + int endIndex = startIndex + MAX_TOPICS_EACH_PRODUCER_HOLD - 1; + if (i == cycle) { + if (remainder == 0) { + continue; + } else { + endIndex = startIndex + remainder - 1; + } + } + for (int index = startIndex; index <= endIndex; index++) { + subset.add(sortedList.get(index)); + } + producer = sessionFactory.createProducer(); + try { + Set succTopicSet = producer.publish(subset); + if (succTopicSet != null) { + for (String succTopic : succTopicSet) { + producerMap.put(succTopic, producer); + } + currentPublishTopicNum.set(succTopicSet.size()); + logger.info(getName() + " success Subset : " + succTopicSet); + } + } catch (Exception e) { + logger.info(getName() + " meta sink initTopicSet fail.", e); + } + } + logger.info(getName() + " initTopicSet cost: " + (System.currentTimeMillis() - startTime) + "ms"); + logger.info(getName() + " producer is ready for topics : " + producerMap.keySet()); + } + + + @Override + public void start() { + try { + createConnection(); + } catch (FlumeException e) { + logger.error("Unable to create tube client" + ". Exception follows.", e); + + /* Try to prevent leaking resources. */ + destroyConnection(); + + /* FIXME: Mark ourselves as failed. */ + stop(); + return; + } + + // start the cleaner thread + if (clientIdCache && !isNewCache) { + idCacheCleaner = new IdCacheCleaner(this, maxSurvivedTime, maxSurvivedSize); + idCacheCleaner.start(); + } + + super.start(); + this.canSend = true; + this.canTake = true; + + try { + initTopicSet(new HashSet(topicProperties.values())); + } catch (Exception e) { + logger.info("meta sink start publish topic fail.", e); + } + + for (int i = 0; i < sinkThreadPool.length; i++) { + sinkThreadPool[i] = new Thread(new SinkTask(), getName() + "_tube_sink_sender-" + i); + sinkThreadPool[i].start(); + } + + } + + + + + class SinkTask implements Runnable { + private void sendMessage(Event event, String topic, AtomicBoolean flag, EventStat es) + throws TubeClientException, InterruptedException { + String clientId = event.getHeaders().get(ConfigConstants.SEQUENCE_ID); + if (!isNewCache) { + Long lastTime = 0L; + if (clientIdCache && clientId != null) { + lastTime = agentIdMap.put(clientId, System.currentTimeMillis()); + } + if (clientIdCache && clientId != null && lastTime != null && lastTime > 0) { + logger.info("{} agent package {} existed,just discard.", getName(), clientId); + } else { + Message message = new Message(topic, event.getBody()); + message.setAttrKeyVal("dataproxyip", NetworkUtils.getLocalIp()); + String tid = ""; + if (event.getHeaders().containsKey(AttributeConstants.INTERFACE_ID)) { + tid = event.getHeaders().get(AttributeConstants.INTERFACE_ID); + } else if (event.getHeaders().containsKey(AttributeConstants.INAME)) { + tid = event.getHeaders().get(AttributeConstants.INAME); + } + message.putSystemHeader(tid, event.getHeaders().get(ConfigConstants.PKG_TIME_KEY)); + + producer.sendMessage(message, new MyCallback(es)); + flag.set(true); + + } + } else { + boolean hasKey = false; + if (clientIdCache && clientId != null) { + hasKey = agentIdCache.asMap().containsKey(clientId); + } + + if (clientIdCache && clientId != null && hasKey) { + agentIdCache.put(clientId, System.currentTimeMillis()); + logger.info("{} agent package {} existed,just discard.", getName(), clientId); + } else { + if (clientId != null) { + agentIdCache.put(clientId, System.currentTimeMillis()); + } + + Message message = new Message(topic, event.getBody()); + message.setAttrKeyVal("dataproxyip", NetworkUtils.getLocalIp()); + String tid = ""; + if (event.getHeaders().containsKey(AttributeConstants.INTERFACE_ID)) { + tid = event.getHeaders().get(AttributeConstants.INTERFACE_ID); + } else if (event.getHeaders().containsKey(AttributeConstants.INAME)) { + tid = event.getHeaders().get(AttributeConstants.INAME); + } + message.putSystemHeader(tid, event.getHeaders().get(ConfigConstants.PKG_TIME_KEY)); + + producer.sendMessage(message, new MyCallback(es)); + flag.set(true); + } + } + illegalTopicMap.remove(topic); + } + + private void handleException(Throwable t, String topic, boolean decrementFlag, EventStat es) { + if (t instanceof TubeClientException) { + String message = t.getMessage(); + if (message != null && (message.contains("No available queue for topic") + || message.contains("The brokers of topic are all forbidden"))) { + illegalTopicMap.put(topic, System.currentTimeMillis() + 60 * 1000); + logger.info("IllegalTopicMap.put " + topic); + return; + } else { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + //ignore.. + } + } + } + logger.error("Sink task fail to send the message, decrementFlag=" + decrementFlag + ",sink.name=" + + Thread.currentThread().getName() + + ",event.headers=" + es.getEvent().getHeaders(), t); + } + + @Override + public void run() { + logger.info("Sink task {} started.", Thread.currentThread().getName()); + while (canSend) { + boolean decrementFlag = false; + boolean resendBadEvent = false; + Event event = null; + EventStat es = null; + String topic = null; + try { + if (MetaSink.this.overflow) { + MetaSink.this.overflow = false; + Thread.sleep(10); + } + if (!resendQueue.isEmpty()) { + es = resendQueue.poll(); + if (es != null) { + event = es.getEvent(); + // logger.warn("Resend event: {}", event.toString()); + if (event.getHeaders().containsKey(TOPIC)) { + topic = event.getHeaders().get(TOPIC); + } + resendBadEvent = true; + } + } else { + event = eventQueue.take(); + es = new EventStat(event); +// sendCnt.incrementAndGet(); + if (event.getHeaders().containsKey(TOPIC)) { + topic = event.getHeaders().get(TOPIC); + } + } + + if (event == null) { + // ignore event is null, when multiple-thread SinkTask running + // this null value comes from resendQueue + continue; + } + + if (topic == null || topic.equals("")) { + logger.warn("no topic specified in event header, just skip this event"); + continue; + } + + Long expireTime = illegalTopicMap.get(topic); + if (expireTime != null) { + long currentTime = System.currentTimeMillis(); + if (expireTime > currentTime) { + + // TODO: need to be improved. +// reChannelEvent(es, topic); + continue; + } else { + + illegalTopicMap.remove(topic); + } + } + MessageProducer producer = null; + try { + producer = getProducer(topic); + } catch (Exception e) { + logger.error("Get producer failed!", e); + } + + if (producer == null) { + illegalTopicMap.put(topic, System.currentTimeMillis() + 30 * 1000); + continue; + } + + AtomicBoolean flagAtomic = new AtomicBoolean(decrementFlag); + sendMessage(event, topic, flagAtomic, es); + decrementFlag = flagAtomic.get(); + + } catch (InterruptedException e) { + logger.info("Thread {} has been interrupted!", Thread.currentThread().getName()); + return; + } catch (Throwable t) { + handleException(t, topic, decrementFlag, es); + resendEvent(es, decrementFlag); + } + } + } + } + + + public class MyCallback implements MessageSentCallback { + private EventStat myEventStat; + + public MyCallback(EventStat eventStat) { + this.myEventStat = eventStat; + } + + @Override + public void onMessageSent(final MessageSentResult result) { + if (result.isSuccess()) { + // TODO: add stats + } else { + if (result.getErrCode() == TErrCodeConstants.FORBIDDEN) { + logger.warn("Send message failed, error message: {}, resendQueue size: {}, event:{}", + result.getErrMsg(), resendQueue.size(), + myEventStat.getEvent().hashCode()); + + return; + } + if (result.getErrCode() != TErrCodeConstants.SERVER_RECEIVE_OVERFLOW) { + logger.warn("Send message failed, error message: {}, resendQueue size: {}, event:{}", + result.getErrMsg(), resendQueue.size(), + myEventStat.getEvent().hashCode()); + } + resendEvent(myEventStat, true); + } + } + + @Override + public void onException(final Throwable e) { + Throwable t = e; + while (t.getCause() != null) { + t = t.getCause(); + } + if (t instanceof OverflowException) { + MetaSink.this.overflow = true; + } + resendEvent(myEventStat, true); + } + } + + + /** + * resend event + * + * @param es + * @param isDecrement + */ + private void resendEvent(EventStat es, boolean isDecrement) { + try { + if (es == null || es.getEvent() == null) { + return; + } + + if (clientIdCache) { + String clientId = es.getEvent().getHeaders().get(ConfigConstants.SEQUENCE_ID); + if (!isNewCache) { + if (clientId != null && agentIdMap.containsKey(clientId)) { + agentIdMap.remove(clientId); + } + } else { + if (clientId != null && agentIdCache.asMap().containsKey(clientId)) { + agentIdCache.invalidate(clientId); + } + } + } + } catch (Throwable throwable) { + logger.error(getName() + " Discard msg because put events to both of queue and " + + "fileChannel fail,current resendQueue.size = " + + resendQueue.size(), throwable); + } + } + + + @Override + public Status process() throws EventDeliveryException { + if (!this.canTake) { + return Status.BACKOFF; + } + Status status = Status.READY; + Channel channel = getChannel(); + Transaction tx = channel.getTransaction(); + tx.begin(); + try { + Event event = channel.take(); + if (event != null) { + if (diskRateLimiter != null) { + diskRateLimiter.acquire(event.getBody().length); + } + if (!eventQueue.offer(event, 3 * 1000, TimeUnit.MILLISECONDS)) { + logger.info("[{}] Channel --> Queue(has no enough space,current code point) " + + "--> Tube,Check if Tube server or network is ok.(if this situation last long time " + + "it will cause memoryChannel full and fileChannel write.)", getName()); + tx.rollback(); + } else { + tx.commit(); + } + } else { + + // logger.info("[{}]No data to process in the channel.",getName()); + status = Status.BACKOFF; + tx.commit(); + } + } catch (Throwable t) { + logger.error("Process event failed!" + this.getName(), t); + try { + tx.rollback(); + } catch (Throwable e) { + logger.error("metasink transaction rollback exception", e); + + } + } finally { + tx.close(); + } + return status; + } + + @Override + public void configure(Context context) { + logger.info(context.toString()); +// logger.info("sinktest:"+getName()+getChannel());//sinktest:meta-sink-msg2null + + configManager = ConfigManager.getInstance(); + topicProperties = configManager.getTopicProperties(); + configManager.getTopicConfig().addUpdateCallback(new ConfigUpdateCallback() { + @Override + public void update() { + + diffSetPublish(new HashSet(topicProperties.values()), + new HashSet(configManager.getTopicProperties().values())); + } + }); + + masterHostAndPortList = context.getString(MASTER_HOST_PORT_LIST); + Preconditions.checkState(masterHostAndPortList != null, "No master and port list specified"); + + producerMap = new HashMap(); + + logEveryNEvents = context.getInteger(LOG_EVERY_N_EVENTS, defaultLogEveryNEvents); + logger.debug(this.getName() + " " + LOG_EVERY_N_EVENTS + " " + logEveryNEvents); + Preconditions.checkArgument(logEveryNEvents > 0, "logEveryNEvents must be > 0"); + + sendTimeout = context.getInteger(SEND_TIMEOUT, defaultSendTimeout); + logger.debug(this.getName() + " " + SEND_TIMEOUT + " " + sendTimeout); + Preconditions.checkArgument(sendTimeout > 0, "sendTimeout must be > 0"); + + MAX_TOPICS_EACH_PRODUCER_HOLD = context.getInteger(MAX_TOPICS_EACH_PRODUCER_HOLD_NAME, 200); + retryCnt = context.getInteger(RETRY_CNT, defaultRetryCnt); + logger.debug(this.getName() + " " + RETRY_CNT + " " + retryCnt); + + boolean isSlaMetricSink = context.getBoolean(SLA_METRIC_SINK, false); + if (isSlaMetricSink) { + this.metaTopicFilePath = slaTopicFilePath; + } + + clientIdCache = context.getBoolean(CLIENT_ID_CACHE, clientIdCache); + if (clientIdCache) { + int survivedTime = context.getInteger(MAX_SURVIVED_TIME, maxSurvivedTime); + if (survivedTime > 0) { + maxSurvivedTime = survivedTime; + } else { + logger.warn("invalid {}:{}", MAX_SURVIVED_TIME, survivedTime); + } + + int survivedSize = context.getInteger(MAX_SURVIVED_SIZE, maxSurvivedSize); + if (survivedSize > 0) { + maxSurvivedSize = survivedSize; + } else { + logger.warn("invalid {}:{}", MAX_SURVIVED_SIZE, survivedSize); + } + } + + String requestTimeout = context.getString(TUBE_REQUEST_TIMEOUT); + if (requestTimeout != null) { + this.requestTimeout = Integer.parseInt(requestTimeout); + } + + String sendRemoteStr = context.getString(SEND_REMOTE); + if (sendRemoteStr != null) { + sendRemote = Boolean.parseBoolean(sendRemoteStr); + } + if (sendRemote) { + proxyLogTopic = context.getString(LOG_TOPIC, proxyLogTopic); + proxyLogBid = context.getString(BID, proxyLogTid); + proxyLogTid = context.getString(TID, proxyLogTid); + } + + resendQueue = new LinkedBlockingQueue<>(BAD_EVENT_QUEUE_SIZE); + + String sinkThreadNum = context.getString(SINK_THREAD_NUM, "4"); + threadNum = Integer.parseInt(sinkThreadNum); + Preconditions.checkArgument(threadNum > 0, "threadNum must be > 0"); + sinkThreadPool = new Thread[threadNum]; + eventQueue = new LinkedBlockingQueue(EVENT_QUEUE_SIZE); + + diskIORatePerSec = context.getLong("disk-io-rate-per-sec", 0L); + if (diskIORatePerSec != 0) { + diskRateLimiter = RateLimiter.create(diskIORatePerSec); + } + + linkMaxAllowedDelayedMsgCount = context.getLong(ConfigConstants.LINK_MAX_ALLOWED_DELAYED_MSG_COUNT, + 80000L); + sessionWarnDelayedMsgCount = context.getLong(ConfigConstants.SESSION_WARN_DELAYED_MSG_COUNT, + 2000000L); + sessionMaxAllowedDelayedMsgCount = context.getLong(ConfigConstants.SESSION_MAX_ALLOWED_DELAYED_MSG_COUNT, + 4000000L); + nettyWriteBufferHighWaterMark = context.getLong(ConfigConstants.NETTY_WRITE_BUFFER_HIGH_WATER_MARK, + 15 * 1024 * 1024L); + recoverthreadcount = context.getInteger(ConfigConstants.RECOVER_THREAD_COUNT, + Runtime.getRuntime().availableProcessors() + 1); + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/DefaultServiceDecoder.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/DefaultServiceDecoder.java new file mode 100644 index 00000000000..e0bfbc94476 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/DefaultServiceDecoder.java @@ -0,0 +1,519 @@ +/* + * 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.inlong.dataproxy.source; + +import com.google.common.base.Splitter; + +import java.io.IOException; +import java.net.SocketAddress; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; +import org.apache.inlong.dataproxy.base.ProxyMessage; +import org.apache.inlong.dataproxy.consts.AttributeConstants; +import org.apache.inlong.dataproxy.consts.ConfigConstants; +import org.apache.inlong.dataproxy.exception.ErrorCode; +import org.apache.inlong.dataproxy.exception.MessageIDException; +import org.jboss.netty.buffer.ChannelBuffer; +import org.jboss.netty.channel.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.xerial.snappy.Snappy; + +public class DefaultServiceDecoder implements ServiceDecoder { + + private static final int BIN_MSG_TOTALLEN_OFFSET = 0; + private static final int BIN_MSG_TOTALLEN_SIZE = 4; + private static final int BIN_MSG_MSGTYPE_OFFSET = 4; + private static final int BIN_MSG_EXTEND_OFFSET = 9; + private static final int BIN_MSG_EXTEND_SIZE = 2; + private static final int BIN_MSG_SET_SNAPPY = (1 << 5); + private static final int BIN_MSG_BODYLEN_SIZE = 4; + private static final int BIN_MSG_BODYLEN_OFFSET = 21; + private static final int BIN_MSG_BODY_OFFSET = BIN_MSG_BODYLEN_SIZE + BIN_MSG_BODYLEN_OFFSET; + private static final int BIN_MSG_ATTRLEN_SIZE = 2; + + private static final int BIN_MSG_FORMAT_SIZE = 29; + private static final int BIN_MSG_MAGIC_SIZE = 2; + private static final int BIN_MSG_MAGIC = 0xEE01; + + private static final int BIN_HB_TOTALLEN_SIZE = 4; + private static final int BIN_HB_BODYLEN_OFFSET = 10; + private static final int BIN_HB_BODYLEN_SIZE = 4; + private static final int BIN_HB_BODY_OFFSET = BIN_HB_BODYLEN_SIZE + BIN_HB_BODYLEN_OFFSET; + private static final int BIN_HB_ATTRLEN_SIZE = 2; + private static final int BIN_HB_FORMAT_SIZE = 17; + + + private static final Logger LOG = LoggerFactory + .getLogger(DefaultServiceDecoder.class); + + private static final Splitter.MapSplitter mapSplitter = Splitter + .on(AttributeConstants.SEPARATOR).trimResults() + .withKeyValueSeparator(AttributeConstants.KEY_VALUE_SEPARATOR); + + + /** + * extract bin heart beat data, message type is 8 + * + * @param resultMap + * @param cb + * @param channel + * @param totalDataLen + * @return + * @throws + */ + private Map extractNewBinHB(Map resultMap, + ChannelBuffer cb, Channel channel, + int totalDataLen) throws Exception { + int msgHeadPos = cb.readerIndex() - 5; + + // check validation + int bodyLen = cb.getInt(msgHeadPos + BIN_HB_BODYLEN_OFFSET); + int attrLen = cb.getShort(msgHeadPos + BIN_HB_BODY_OFFSET + bodyLen); + int msgMagic = cb.getUnsignedShort(msgHeadPos + BIN_HB_BODY_OFFSET + + bodyLen + BIN_HB_ATTRLEN_SIZE + attrLen); + + if ((totalDataLen + BIN_HB_TOTALLEN_SIZE < (bodyLen + + attrLen + BIN_HB_FORMAT_SIZE)) || (msgMagic != BIN_MSG_MAGIC)) { + + LOG.error("err msg, bodyLen + attrLen > totalDataLen, " + + "and bodyLen={},attrLen={},totalDataLen={},magic={};Connection info:{}", + bodyLen, attrLen, totalDataLen, Integer.toHexString(msgMagic), channel.toString()); + + return resultMap; + } + cb.skipBytes(9 + bodyLen + BIN_HB_ATTRLEN_SIZE); + // extract common attr + String strAttr = null; + if (attrLen != 0) { + byte[] attrData = new byte[attrLen]; + cb.readBytes(attrData, 0, attrLen); + strAttr = new String(attrData, StandardCharsets.UTF_8); + resultMap.put(ConfigConstants.DECODER_ATTRS, strAttr); + } + byte version = cb.getByte(msgHeadPos + 9); + resultMap.put(ConfigConstants.VERSION_TYPE, version); + + return resultMap; + } + + private void handleDateTime(Map commonAttrMap, Channel channel, + long uniq, long dataTime, int msgCount) { + commonAttrMap.put(AttributeConstants.UNIQ_ID, String.valueOf(uniq)); + String time = ""; + if (commonAttrMap.containsKey(ConfigConstants.PKG_TIME_KEY)) { + time = commonAttrMap + .get(ConfigConstants.PKG_TIME_KEY); + } else { + time = String.valueOf(dataTime); + } + StringBuilder sidBuilder = new StringBuilder(); + sidBuilder.append(channel.getRemoteAddress().toString()).append("#").append(time) + .append("#").append(uniq); + commonAttrMap.put(AttributeConstants.SEQUENCE_ID, new String(sidBuilder)); + + // datetime from sdk + commonAttrMap.put(AttributeConstants.DATA_TIME, String.valueOf(dataTime)); + commonAttrMap + .put(AttributeConstants.RCV_TIME, String.valueOf(System.currentTimeMillis())); + commonAttrMap.put(AttributeConstants.MESSAGE_COUNT, + String.valueOf(msgCount != 0 ? msgCount : 1)); + } + + private boolean handleExtMap(Map commonAttrMap, ChannelBuffer cb, + Map resultMap, int extendField, int msgHeadPos) { + boolean index = false; + if ((extendField & 0x8) == 0x8) { + index = true; + // if message is file metric + int dataLen = cb.getInt(msgHeadPos + BIN_MSG_BODYLEN_OFFSET + 4); + byte[] data = new byte[dataLen]; + cb.getBytes(msgHeadPos + BIN_MSG_BODY_OFFSET + 4, data, 0, + dataLen); + resultMap.put(ConfigConstants.FILE_BODY, data); + commonAttrMap.put(ConfigConstants.FILE_CHECK_DATA, "true"); + } else if ((extendField & 0x10) == 0x10) { + index = true; + // if message is verification metric message + int dataLen = cb.getInt(msgHeadPos + BIN_MSG_BODYLEN_OFFSET + 4); + byte[] data = new byte[dataLen]; + // remove body len + cb.getBytes(msgHeadPos + BIN_MSG_BODY_OFFSET + 4, data, 0, + dataLen); + resultMap.put(ConfigConstants.FILE_BODY, data); + commonAttrMap.put(ConfigConstants.MINUTE_CHECK_DATA, "true"); + } + return index; + } + + private ByteBuffer handleTrace(Channel channel, ChannelBuffer cb, int extendField, + int msgHeadPos, int totalDataLen, int attrLen, String strAttr, int bodyLen) { + // whether enable trace + boolean enableTrace = (((extendField & 0x2) >> 1) == 0x1); + ByteBuffer dataBuf; + if (!enableTrace) { + dataBuf = ByteBuffer.allocate(totalDataLen + BIN_MSG_TOTALLEN_SIZE); + cb.getBytes(msgHeadPos, dataBuf.array(), 0, + totalDataLen + BIN_MSG_TOTALLEN_SIZE); + } else { + String traceInfo; + String strNode2Ip = null; + + SocketAddress loacalSockAddr = channel.getLocalAddress(); + if (null != loacalSockAddr) { + strNode2Ip = loacalSockAddr.toString(); + try { + strNode2Ip = strNode2Ip.substring(1, strNode2Ip.indexOf(':')); + } catch (Exception ee) { + LOG.warn("fail to get the local IP, and strIP={},localSocketAddress={}", + strNode2Ip, loacalSockAddr); + } + } + + traceInfo = "node2ip=" + strNode2Ip + "&rtime2=" + System.currentTimeMillis(); + + int newTotalLen = 0; + + if (attrLen != 0) { + newTotalLen = totalDataLen + traceInfo.length() + "&".length(); + strAttr = strAttr + "&" + traceInfo; + } else { + newTotalLen = totalDataLen + traceInfo.length(); + strAttr = traceInfo; + } + + dataBuf = ByteBuffer.allocate(newTotalLen + BIN_MSG_TOTALLEN_SIZE); + cb.getBytes(msgHeadPos, dataBuf.array(), 0, + bodyLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_ATTRLEN_SIZE + - BIN_MSG_MAGIC_SIZE)); + dataBuf.putShort( + bodyLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_ATTRLEN_SIZE - BIN_MSG_MAGIC_SIZE), + (short) strAttr.length()); + + System.arraycopy(strAttr.getBytes(StandardCharsets.UTF_8), 0, dataBuf.array(), + bodyLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_MAGIC_SIZE), + strAttr.length()); + + dataBuf.putInt(0, newTotalLen); + dataBuf.putShort(newTotalLen + BIN_MSG_TOTALLEN_SIZE - BIN_MSG_MAGIC_SIZE, + (short) 0xee01); + } + return dataBuf; + } + + /** + * extract bin data, message type is 7 + * + * @param resultMap + * @param cb + * @param channel + * @param totalDataLen + * @param msgType + * @return + * @throws Exception + */ + private Map extractNewBinData(Map resultMap, + ChannelBuffer cb, Channel channel, + int totalDataLen, MsgType msgType) throws Exception { + int msgHeadPos = cb.readerIndex() - 5; + + int bodyLen = cb.getInt(msgHeadPos + BIN_MSG_BODYLEN_OFFSET); + int attrLen = cb.getShort(msgHeadPos + BIN_MSG_BODY_OFFSET + bodyLen); + int msgMagic = cb.getUnsignedShort(msgHeadPos + BIN_MSG_BODY_OFFSET + + bodyLen + BIN_MSG_ATTRLEN_SIZE + attrLen); + + if (bodyLen == 0) { + throw new Exception(new Throwable("err msg, bodyLen is empty" + + ";Connection info:" + channel.toString())); + } + + if ((totalDataLen + BIN_MSG_TOTALLEN_SIZE < (bodyLen + attrLen + BIN_MSG_FORMAT_SIZE)) + || (msgMagic != BIN_MSG_MAGIC)) { + throw new Exception(new Throwable( + "err msg, bodyLen + attrLen > totalDataLen,or msgMagic is valid! and bodyLen=" + + bodyLen + ",totalDataLen=" + totalDataLen + ",attrLen=" + attrLen + + ";magic=" + Integer.toHexString(msgMagic) + + ";Connection info:" + channel.toString())); + } + + int bidNum = cb.readUnsignedShort(); + int tidNum = cb.readUnsignedShort(); + final int extendField = cb.readUnsignedShort(); + long dataTime = cb.readUnsignedInt(); + int msgCount = cb.readUnsignedShort(); + long uniq = cb.readUnsignedInt(); + + + dataTime = dataTime * 1000; + Map commonAttrMap = new HashMap(); + cb.skipBytes(BIN_MSG_BODYLEN_SIZE + bodyLen + BIN_MSG_ATTRLEN_SIZE); + resultMap.put(ConfigConstants.COMMON_ATTR_MAP, commonAttrMap); + + resultMap.put(ConfigConstants.EXTRA_ATTR, ((extendField & 0x1) == 0x1) ? "true" : "false"); + + // read body data + byte[] bodyData = new byte[bodyLen]; + cb.getBytes(msgHeadPos + BIN_MSG_BODY_OFFSET, bodyData, 0, bodyLen); + resultMap.put(ConfigConstants.DECODER_BODY, bodyData); + + // read attr and write to map. + String strAttr = null; + if (attrLen != 0) { + byte[] attrData = new byte[attrLen]; + cb.readBytes(attrData, 0, attrLen); + strAttr = new String(attrData, StandardCharsets.UTF_8); + resultMap.put(ConfigConstants.DECODER_ATTRS, strAttr); + + try { + commonAttrMap.putAll(mapSplitter.split(strAttr)); + } catch (Exception e) { + cb.clear(); + throw new MessageIDException(uniq, + ErrorCode.ATTR_ERROR, + new Throwable("[Parse Error]new six segment protocol ,attr is " + + strAttr + " , channel info:" + channel.toString())); + } + } + + try { + handleDateTime(commonAttrMap, channel, uniq, dataTime, msgCount); + final boolean index = handleExtMap(commonAttrMap, cb, resultMap, extendField, msgHeadPos); + ByteBuffer dataBuf = handleTrace(channel, cb, extendField, msgHeadPos, + totalDataLen, attrLen, strAttr, bodyLen); + + String bid = null; + String tid = null; + + if (commonAttrMap.containsKey(AttributeConstants.BUSINESS_ID)) { + bid = commonAttrMap.get(AttributeConstants.BUSINESS_ID); + } + if (commonAttrMap.containsKey(AttributeConstants.INTERFACE_ID)) { + tid = commonAttrMap.get(AttributeConstants.INTERFACE_ID); + } + + if ((bid != null) && (tid != null)) { + commonAttrMap.put(AttributeConstants.NUM2NAME, "FALSE"); + dataBuf.putShort(BIN_MSG_EXTEND_OFFSET, (short) (extendField | 0x4)); + } else { + boolean hasNumBid = (((extendField & 0x4) >> 2) == 0x0); + if (hasNumBid && (0 != bidNum) && (0 != tidNum)) { + commonAttrMap.put(AttributeConstants.NUM2NAME, "TRUE"); + commonAttrMap.put(AttributeConstants.BID_NUM, String.valueOf(bidNum)); + commonAttrMap.put(AttributeConstants.TID_NUM, String.valueOf(tidNum)); + } + } + + if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType) && !index) { + List msgList = new ArrayList<>(1); + msgList.add(new ProxyMessage(bid, tid, commonAttrMap, dataBuf.array())); + resultMap.put(ConfigConstants.MSG_LIST, msgList); + } else if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType)) { + List msgList = new ArrayList<>(1); + msgList.add(new ProxyMessage(bid, tid, commonAttrMap, + (byte[]) resultMap.get(ConfigConstants.FILE_BODY))); + resultMap.put(ConfigConstants.MSG_LIST, msgList); + } + } catch (Exception ex) { + cb.clear(); + throw new MessageIDException(uniq, ErrorCode.OTHER_ERROR, ex.getCause()); + } + + return resultMap; + } + + + /** + * extract bin data, message type less than 7 + * + * @param resultMap + * @param cb + * @param channel + * @param totalDataLen + * @param msgType + * @return + * @throws Exception + */ + private Map extractDefaultData(Map resultMap, + ChannelBuffer cb, Channel channel, + int totalDataLen, MsgType msgType) throws Exception { + int bodyLen = cb.readInt(); + if (bodyLen == 0) { + throw new Exception(new Throwable("err msg, bodyLen is empty" + ";" + + "Connection info:" + channel.toString())); + } + // if body len is bigger than totalDataLen - 5(bodyLen bytes + message type bytes), + // that means an invalid message, reject it. + if (bodyLen > totalDataLen - 5) { + throw new Exception(new Throwable("err msg, firstLen > totalDataLen, and bodyLen=" + + bodyLen + ",totalDataLen=" + totalDataLen + + ";Connection info:" + channel.toString())); + } + + // extract body bytes + byte[] bodyData = new byte[bodyLen]; + cb.readBytes(bodyData, 0, bodyLen); + resultMap.put(ConfigConstants.DECODER_BODY, bodyData); + + int attrLen = cb.readInt(); + // 9 means bodyLen bytes(4) + message type bytes(1) + attrLen bytes(4) + if (totalDataLen != 9 + attrLen + bodyLen) { + throw new Exception(new Throwable( + "err msg, totalDataLen != 9 + bodyLen + attrLen,and bodyLen=" + bodyLen + + ",totalDataLen=" + totalDataLen + ",attrDataLen=" + attrLen + + ";Connection info:" + channel.toString())); + } + + // extract attr bytes + byte[] attrData = new byte[attrLen]; + cb.readBytes(attrData, 0, attrLen); + String strAttr = new String(attrData, StandardCharsets.UTF_8); + resultMap.put(ConfigConstants.DECODER_ATTRS, strAttr); + + // convert attr bytes to map + Map commonAttrMap = null; + try { + commonAttrMap = new HashMap(mapSplitter.split(strAttr)); + } catch (Exception e) { + throw new Exception(new Throwable("Parse commonAttrMap error.commonAttrString is: " + + strAttr + " ,channel is :" + channel.toString())); + } + resultMap.put(ConfigConstants.COMMON_ATTR_MAP, commonAttrMap); + + // decompress body data if compress type exists. + String compressType = commonAttrMap.get(AttributeConstants.COMPRESS_TYPE); + resultMap.put(ConfigConstants.COMPRESS_TYPE, compressType); + if (StringUtils.isNotBlank(compressType)) { + byte[] unCompressedData = processUnCompress(bodyData, compressType); + if (unCompressedData == null || unCompressedData.length == 0) { + throw new Exception(new Throwable("Uncompress data error!compress type:" + + compressType + ";data:" + new String(bodyData, StandardCharsets.UTF_8) + + ";attr:" + strAttr + ";channel:" + channel.toString())); + } + bodyData = unCompressedData; + } + + // fill up attr map with some keys. + commonAttrMap.put(AttributeConstants.RCV_TIME, String.valueOf(System.currentTimeMillis())); + String bid = commonAttrMap.get(AttributeConstants.BUSINESS_ID); + String tid = commonAttrMap.get(AttributeConstants.INTERFACE_ID); + + // add message count attr + String cntStr = commonAttrMap.get(AttributeConstants.MESSAGE_COUNT); + int msgCnt = cntStr != null ? Integer.parseInt(cntStr) : 1; + commonAttrMap.put(AttributeConstants.MESSAGE_COUNT, String.valueOf(msgCnt)); + + // extract data from bodyData and if message type is 5, convert data into list. + List msgList = null; + ByteBuffer bodyBuffer = ByteBuffer.wrap(bodyData); + if (MsgType.MSG_MULTI_BODY.equals(msgType)) { + msgList = new ArrayList<>(msgCnt); + while (bodyBuffer.remaining() > 0) { + int singleMsgLen = bodyBuffer.getInt(); + if (singleMsgLen <= 0 || singleMsgLen > bodyBuffer.remaining()) { + throw new Exception(new Throwable("[Malformed Data]Invalid data len!channel is " + + channel.toString())); + } + byte[] record = new byte[singleMsgLen]; + bodyBuffer.get(record); + + ProxyMessage message = new ProxyMessage(bid, tid, commonAttrMap, record); + msgList.add(message); + } + } else { + msgList = new ArrayList<>(1); + msgList.add(new ProxyMessage(bid, tid, commonAttrMap, bodyData)); + } + resultMap.put(ConfigConstants.MSG_LIST, msgList); + + return resultMap; + } + + + private byte[] processUnCompress(byte[] input, String compressType) { + byte[] result = null; + try { + int uncompressedLen = Snappy.uncompressedLength(input, 0, input.length); + result = new byte[uncompressedLen]; + Snappy.uncompress(input, 0, input.length, result, 0); + } catch (IOException e) { + LOG.error("Uncompress data error!", e); + return null; + } + return result; + } + + + /** + * BEFORE AFTER + * +--------+--------+--------+----------------+--------+----------------+------------------------+ + * | Length | Msgtype| Length | Actual Content1| Length | Actual Content2|----- >| Actual + * Content1| Time postfix | | 0x000C | 0x02 | 0x000C | "HELLO, WORLD" | 0x000C + * | "view video? " | | "HELLO, WORLD" | ",recvtimestamp:12345674321" | + * +--------+--------+--------+----------------+--------+----------------+------------------------+ + */ + @Override + public Map extractData(ChannelBuffer cb, Channel channel) throws Exception { + Map resultMap = new HashMap(); + if (null == cb) { + LOG.error("cb == null"); + return resultMap; + } + int totalLen = cb.readableBytes(); + if (ConfigConstants.MSG_MAX_LENGTH_BYTES < totalLen) { + throw new Exception(new Throwable("err msg, ConfigConstants.MSG_MAX_LENGTH_BYTES " + + "< totalLen, and totalLen=" + totalLen)); + } + // save index, reset it if buffer is not satisfied. + cb.markReaderIndex(); + int totalDataLen = cb.readInt(); + if (totalDataLen + HEAD_LENGTH <= totalLen) { + int msgTypeInt = cb.readByte(); + int compressType = ((msgTypeInt & 0xE0) >> 5); + MsgType msgType = MsgType.valueOf(msgTypeInt); + resultMap.put(ConfigConstants.MSG_TYPE, msgType); + + // if it's heart beat or unknown message, just return without handling it. + if (MsgType.MSG_HEARTBEAT.equals(msgType) + || MsgType.MSG_UNKNOWN.equals(msgType)) { + return resultMap; + } + // if it's bin heart beat. + if (MsgType.MSG_BIN_HEARTBEAT.equals(msgType)) { + return extractNewBinHB(resultMap, cb, channel, totalDataLen); + } + + if (msgType.getValue() >= MsgType.MSG_BIN_MULTI_BODY.getValue()) { + resultMap.put(ConfigConstants.COMPRESS_TYPE, (compressType != 0) ? "snappy" : ""); + return extractNewBinData(resultMap, cb, channel, totalDataLen, msgType); + } else { + return extractDefaultData(resultMap, cb, channel, totalDataLen, msgType); + } + + } else { + // reset index. + cb.resetReaderIndex(); + return null; + } + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/MsgType.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/MsgType.java new file mode 100644 index 00000000000..55479736a42 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/MsgType.java @@ -0,0 +1,54 @@ +/* + * 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.inlong.dataproxy.source; + +public enum MsgType { + /** + * heartbeat + */ + MSG_HEARTBEAT(1), + MSG_COMMON_SERVICE(2), + MSG_ACK_SERVICE(3), + MSG_ORIGINAL_RETURN(4), + MSG_MULTI_BODY(5), + MSG_MULTI_BODY_ATTR(6), + MSG_BIN_MULTI_BODY(7), + MSG_BIN_HEARTBEAT(8), + MSG_UNKNOWN(-1); + + private final int value; + + MsgType(int value) { + this.value = value; + } + + public static MsgType valueOf(int type) { + int inputType = (type & 0x1F); + + for (MsgType msgType : MsgType.values()) { + if (msgType.getValue() == inputType) { + return msgType; + } + } + return MSG_UNKNOWN; + } + + public int getValue() { + return value; + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageFactory.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageFactory.java new file mode 100644 index 00000000000..da1d6278adb --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageFactory.java @@ -0,0 +1,148 @@ +/* + * 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.inlong.dataproxy.source; + +import java.lang.reflect.Constructor; +import java.util.concurrent.TimeUnit; + +import org.apache.flume.channel.ChannelProcessor; +import org.apache.inlong.dataproxy.consts.ConfigConstants; +import org.jboss.netty.channel.ChannelPipeline; +import org.jboss.netty.channel.ChannelPipelineFactory; +import org.jboss.netty.channel.Channels; +import org.jboss.netty.channel.SimpleChannelHandler; +import org.jboss.netty.channel.group.ChannelGroup; +import org.jboss.netty.handler.codec.frame.LengthFieldBasedFrameDecoder; +import org.jboss.netty.handler.execution.ExecutionHandler; +import org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor; +import org.jboss.netty.handler.timeout.ReadTimeoutHandler; +import org.jboss.netty.util.HashedWheelTimer; +import org.jboss.netty.util.Timer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ServerMessageFactory implements ChannelPipelineFactory { + + private static final Logger LOG = LoggerFactory.getLogger(ServerMessageFactory.class); + private static final int DEFAULT_READ_IDLE_TIME = 70 * 60 * 1000; + private ChannelProcessor processor; + private ChannelGroup allChannels; + private ExecutionHandler executionHandler; + private String protocolType; + private ServiceDecoder serviceProcessor; + private String messageHandlerName; + private int maxConnections = Integer.MAX_VALUE; + private int maxMsgLength; + private boolean isCompressed; + private String name; + private String topic; + private String attr; + private boolean filterEmptyMsg; + private Timer timer = new HashedWheelTimer(); + + /** + * get server factory + * + * @param processor + * @param allChannels + * @param protocol + * @param serProcessor + * @param messageHandlerName + * @param maxMsgLength + * @param topic + * @param attr + * @param filterEmptyMsg + * @param maxCons + * @param isCompressed + * @param name + */ + public ServerMessageFactory(ChannelProcessor processor, + ChannelGroup allChannels, String protocol, ServiceDecoder serProcessor, + String messageHandlerName, Integer maxMsgLength, + String topic, String attr, Boolean filterEmptyMsg, Integer maxCons, + Boolean isCompressed, String name) { + this.processor = processor; + this.allChannels = allChannels; + this.topic = topic; + this.attr = attr; + this.filterEmptyMsg = filterEmptyMsg; + int cores = Runtime.getRuntime().availableProcessors(); + this.protocolType = protocol; + this.serviceProcessor = serProcessor; + this.messageHandlerName = messageHandlerName; + this.name = name; + this.maxConnections = maxCons; + this.maxMsgLength = maxMsgLength; + this.isCompressed = isCompressed; + + if (protocolType.equalsIgnoreCase(ConfigConstants.UDP_PROTOCOL)) { + this.executionHandler = new ExecutionHandler( + new OrderedMemoryAwareThreadPoolExecutor(cores * 2, + 1024 * 1024, 1024 * 1024)); + } + } + + @Override + public ChannelPipeline getPipeline() throws Exception { + ChannelPipeline cp = Channels.pipeline(); + return addMessageHandlersTo(cp); + } + + /** + * get message handlers + * @param cp + * @return + */ + public ChannelPipeline addMessageHandlersTo(ChannelPipeline cp) { + + if (this.protocolType + .equalsIgnoreCase(ConfigConstants.TCP_PROTOCOL)) { + cp.addLast("messageDecoder", new LengthFieldBasedFrameDecoder( + this.maxMsgLength, 0, 4, 0, 0, true)); + cp.addLast("readTimeoutHandler", new ReadTimeoutHandler(timer, + DEFAULT_READ_IDLE_TIME, TimeUnit.MILLISECONDS)); + } + + if (processor != null) { + try { + Class clazz = (Class) Class + .forName(messageHandlerName); + + Constructor ctor = clazz.getConstructor( + ChannelProcessor.class, ServiceDecoder.class, ChannelGroup.class, + String.class, String.class, Boolean.class, Integer.class, + Integer.class, Boolean.class, String.class); + + SimpleChannelHandler messageHandler = (SimpleChannelHandler) ctor + .newInstance(processor, serviceProcessor, allChannels, topic, attr, + filterEmptyMsg, maxMsgLength, maxConnections, isCompressed, protocolType + ); + + cp.addLast("messageHandler", messageHandler); + } catch (Exception e) { + LOG.info("SimpleChannelHandler.newInstance has error:" + name, e); + } + } + + if (this.protocolType.equalsIgnoreCase(ConfigConstants.UDP_PROTOCOL)) { + cp.addLast("execution", executionHandler); + } + + return cp; + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageHandler.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageHandler.java new file mode 100644 index 00000000000..dce4a3cb240 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServerMessageHandler.java @@ -0,0 +1,645 @@ +/* + * 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.inlong.dataproxy.source; + +import static org.apache.inlong.dataproxy.consts.AttributeConstants.SEPARATOR; +import static org.apache.inlong.dataproxy.consts.ConfigConstants.SLA_METRIC_BID; +import static org.apache.inlong.dataproxy.consts.ConfigConstants.SLA_METRIC_DATA; +import static org.apache.inlong.dataproxy.source.SimpleTcpSource.blacklist; + +import com.google.common.base.Joiner; +import com.google.common.base.Splitter; + +import java.io.IOException; +import java.net.SocketAddress; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import java.util.concurrent.atomic.AtomicReference; +import org.apache.commons.lang.StringUtils; +import org.apache.flume.ChannelException; +import org.apache.flume.Event; +import org.apache.flume.channel.ChannelProcessor; +import org.apache.flume.event.EventBuilder; +import org.apache.inlong.commons.msg.TDMsg1; +import org.apache.inlong.dataproxy.base.ProxyMessage; +import org.apache.inlong.dataproxy.config.ConfigManager; +import org.apache.inlong.dataproxy.consts.AttributeConstants; +import org.apache.inlong.dataproxy.consts.ConfigConstants; +import org.apache.inlong.dataproxy.exception.ErrorCode; +import org.apache.inlong.dataproxy.exception.MessageIDException; +import org.jboss.netty.buffer.ChannelBuffer; +import org.jboss.netty.buffer.ChannelBuffers; +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelHandlerContext; +import org.jboss.netty.channel.ChannelStateEvent; +import org.jboss.netty.channel.ExceptionEvent; +import org.jboss.netty.channel.MessageEvent; +import org.jboss.netty.channel.SimpleChannelHandler; +import org.jboss.netty.channel.group.ChannelGroup; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Server message handler + * + */ +public class ServerMessageHandler extends SimpleChannelHandler { + + private static final Logger logger = LoggerFactory.getLogger(ServerMessageHandler.class); + + private static final String DEFAULT_REMOTE_IP_VALUE = "0.0.0.0"; + private static final String DEFAULT_REMOTE_IDC_VALUE = "0"; + private static final ConfigManager configManager = ConfigManager.getInstance(); + private static final Joiner.MapJoiner mapJoiner = Joiner.on(AttributeConstants.SEPARATOR) + .withKeyValueSeparator(AttributeConstants.KEY_VALUE_SEPARATOR); + private static final Splitter.MapSplitter mapSplitter = Splitter + .on(AttributeConstants.SEPARATOR) + .trimResults().withKeyValueSeparator(AttributeConstants.KEY_VALUE_SEPARATOR); + + private static final ThreadLocal dateFormator = + new ThreadLocal() { + @Override + protected SimpleDateFormat initialValue() { + return new SimpleDateFormat("yyyyMMddHHmm"); + } + }; + private static final ThreadLocal dateFormator4Transfer = + new ThreadLocal() { + @Override + protected SimpleDateFormat initialValue() { + return new SimpleDateFormat("yyyyMMddHHmmss"); + } + }; + + private final ChannelGroup allChannels; + private int maxConnections = Integer.MAX_VALUE; + private boolean filterEmptyMsg = false; + private final boolean isCompressed; + private final ChannelProcessor processor; + private final ServiceDecoder serviceProcessor; + private final String defaultTopic; + private String defaultMXAttr = "m=3"; + private final ChannelBuffer heartbeatBuffer; + private final String protocolType; + + + public ServerMessageHandler(ChannelProcessor processor, ServiceDecoder serProcessor, + ChannelGroup allChannels, + String topic, String attr, Boolean filterEmptyMsg, Integer maxMsgLength, + Integer maxCons, + Boolean isCompressed, String protocolType) { + + this.processor = processor; + this.serviceProcessor = serProcessor; + this.allChannels = allChannels; + this.defaultTopic = topic; + if (null != attr) { + this.defaultMXAttr = attr; + } + + this.filterEmptyMsg = filterEmptyMsg; + this.isCompressed = isCompressed; + this.heartbeatBuffer = ChannelBuffers.wrappedBuffer(new byte[]{0, 0, 0, 1, 1}); + this.maxConnections = maxCons; + this.protocolType = protocolType; + } + + private String getRemoteIp(Channel channel) { + String strRemoteIp = DEFAULT_REMOTE_IP_VALUE; + SocketAddress remoteSocketAddress = channel.getRemoteAddress(); + if (null != remoteSocketAddress) { + strRemoteIp = remoteSocketAddress.toString(); + try { + strRemoteIp = strRemoteIp.substring(1, strRemoteIp.indexOf(':')); + } catch (Exception ee) { + logger.warn("fail to get the remote IP, and strIP={},remoteSocketAddress={}", + strRemoteIp, + remoteSocketAddress); + } + } + return strRemoteIp; + } + + private byte[] newBinMsg(byte[] orgBinMsg, String extraAttr) { + final int BIN_MSG_TOTALLEN_OFFSET = 0; + final int BIN_MSG_TOTALLEN_SIZE = 4; + final int BIN_MSG_BODYLEN_SIZE = 4; + final int BIN_MSG_EXTEND_OFFSET = 9; + final int BIN_MSG_BODYLEN_OFFSET = 21; + final int BIN_MSG_BODY_OFFSET = BIN_MSG_BODYLEN_SIZE + BIN_MSG_BODYLEN_OFFSET; + final int BIN_MSG_ATTRLEN_SIZE = 2; + final int BIN_MSG_FORMAT_SIZE = 29; + final int BIN_MSG_MAGIC_SIZE = 2; + final int BIN_MSG_MAGIC = 0xEE01; + + ByteBuffer orgBuf = ByteBuffer.wrap(orgBinMsg); + int totalLen = orgBuf.getInt(BIN_MSG_TOTALLEN_OFFSET); + int dataLen = orgBuf.getInt(BIN_MSG_BODYLEN_OFFSET); + int attrLen = orgBuf.getShort(BIN_MSG_BODY_OFFSET + dataLen); + + int newTotalLen = 0; + String strAttr; + if (attrLen != 0) { + newTotalLen = totalLen + extraAttr.length() + "&".length(); + strAttr = "&" + extraAttr; + } else { + newTotalLen = totalLen + extraAttr.length(); + strAttr = extraAttr; + } + + ByteBuffer dataBuf = ByteBuffer.allocate(newTotalLen + BIN_MSG_TOTALLEN_SIZE); + dataBuf + .put(orgBuf.array(), 0, dataLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_MAGIC_SIZE) + attrLen); + dataBuf + .putShort(dataLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_ATTRLEN_SIZE - BIN_MSG_MAGIC_SIZE), + (short) (strAttr.length() + attrLen)); + + + System.arraycopy(strAttr.getBytes(StandardCharsets.UTF_8), 0, dataBuf.array(), + dataLen + (BIN_MSG_FORMAT_SIZE - BIN_MSG_MAGIC_SIZE) + attrLen, + strAttr.length()); + int extendField = orgBuf.getShort(BIN_MSG_EXTEND_OFFSET); + dataBuf.putShort(BIN_MSG_EXTEND_OFFSET, (short) (extendField | 0x4)); + dataBuf.putInt(0, newTotalLen); + dataBuf.putShort(newTotalLen + BIN_MSG_TOTALLEN_SIZE - BIN_MSG_MAGIC_SIZE, + (short) BIN_MSG_MAGIC); + return dataBuf.array(); + } + + public boolean checkBlackIp(Channel channel) { + String strRemoteIp = getRemoteIp(channel); + if (strRemoteIp != null && blacklist != null && blacklist.contains(strRemoteIp)) { + logger.error(strRemoteIp + " is in blacklist, so refuse it !"); + channel.disconnect(); + channel.close(); + allChannels.remove(channel); + return true; + } else { + return false; + } + } + + + @Override + public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent e) throws Exception { + if (allChannels.size() - 1 >= maxConnections) { + logger.warn("refuse to connect , and connections=" + (allChannels.size() - 1) + + ", maxConnections=" + + maxConnections + ",channel is " + e.getChannel()); + e.getChannel().disconnect(); + e.getChannel().close(); + } + if (!checkBlackIp(e.getChannel())) { + logger.info("connections={},maxConnections={}", allChannels.size() - 1, maxConnections); + allChannels.add(e.getChannel()); + super.channelOpen(ctx, e); + } + } + + private void checkBidInfo(ProxyMessage message, Map commonAttrMap, + Map attrMap, AtomicReference topicInfo) { + String bid = message.getBid(); + String tid; + if (null != bid) { + String from = commonAttrMap.get(AttributeConstants.FROM); + if ("dc".equals(from)) { + String dcInterfaceId = message.getTid(); + if (StringUtils.isNotEmpty(dcInterfaceId) + && configManager.getDcMappingProperties() + .containsKey(dcInterfaceId.trim())) { + bid = configManager.getDcMappingProperties() + .get(dcInterfaceId.trim()).trim(); + message.setBid(bid); + } + } + + + String value = configManager.getTopicProperties().get(bid); + if (StringUtils.isNotEmpty(value)) { + topicInfo.set(value.trim()); + } + + + Map mxValue = configManager.getMxPropertiesMaps().get(bid); + if (mxValue != null && mxValue.size() != 0) { + message.getAttributeMap().putAll(mxValue); + } else { + message.getAttributeMap().putAll(mapSplitter.split(this.defaultMXAttr)); + } + } else { + String num2name = commonAttrMap.get(AttributeConstants.NUM2NAME); + String bidNum = commonAttrMap.get(AttributeConstants.BID_NUM); + String tidNum = commonAttrMap.get(AttributeConstants.TID_NUM); + + if (configManager.getBidMappingProperties() != null + && configManager.getTidMappingProperties() != null) { + bid = configManager.getBidMappingProperties().get(bidNum); + tid = (configManager.getTidMappingProperties().get(bidNum) == null) + ? null : configManager.getTidMappingProperties().get(bidNum).get(tidNum); + if (bid != null && tid != null) { + String enableTrans = + (configManager.getBidEnableMappingProperties() == null) + ? null : configManager.getBidEnableMappingProperties().get(bidNum); + if (("TRUE".equalsIgnoreCase(enableTrans) && "TRUE" + .equalsIgnoreCase(num2name))) { + String extraAttr = "bid=" + bid + "&" + "tid=" + tid; + message.setData(newBinMsg(message.getData(), extraAttr)); + } + + attrMap.put(AttributeConstants.BUSINESS_ID, bid); + attrMap.put(AttributeConstants.INTERFACE_ID, tid); + message.setBid(bid); + message.setTid(tid); + + + String value = configManager.getTopicProperties().get(bid); + if (StringUtils.isNotEmpty(value)) { + topicInfo.set(value.trim()); + } + } + } + } + } + + private void updateMsgList(List msgList, Map commonAttrMap, + Map>> messageMap, + String strRemoteIP, MsgType msgType) { + for (ProxyMessage message : msgList) { + Map attrMap = message.getAttributeMap(); + + String topic = this.defaultTopic; + + AtomicReference topicInfo = new AtomicReference<>(topic); + checkBidInfo(message, commonAttrMap, attrMap, topicInfo); + topic = topicInfo.get(); + +// if(bid==null)bid="b_test";//default bid + + message.setTopic(topic); + commonAttrMap.put(AttributeConstants.NODE_IP, strRemoteIP); + + String bid = message.getBid(); + String tid = message.getTid(); + + // whether sla + if (SLA_METRIC_BID.equals(bid)) { + commonAttrMap.put(SLA_METRIC_DATA, "true"); + message.setTopic(SLA_METRIC_DATA); + } + + if (bid != null && tid != null) { + String tubeSwtichKey = bid + SEPARATOR + tid; + if (configManager.getTubeSwitchProperties().get(tubeSwtichKey) != null + && "false".equals(configManager.getTubeSwitchProperties() + .get(tubeSwtichKey).trim())) { + continue; + } + } + + if (!"pb".equals(attrMap.get(AttributeConstants.MESSAGE_TYPE)) + && !MsgType.MSG_MULTI_BODY.equals(msgType) + && !MsgType.MSG_MULTI_BODY_ATTR.equals(msgType)) { + byte[] data = message.getData(); + if (data[data.length - 1] == '\n') { + int tripDataLen = data.length - 1; + if (data[data.length - 2] == '\r') { + tripDataLen = data.length - 2; + } + byte[] tripData = new byte[tripDataLen]; + System.arraycopy(data, 0, tripData, 0, tripDataLen); + message.setData(tripData); + } + } + + if (tid == null) { + tid = ""; + } + HashMap> tidMsgMap = messageMap + .computeIfAbsent(topic, k -> new HashMap<>()); + List tidMsgList = tidMsgMap + .computeIfAbsent(tid, k -> new ArrayList<>()); + tidMsgList.add(message); + } + } + + private void formatMessagesAndSend(Map commonAttrMap, + Map>> messageMap, + String strRemoteIP, MsgType msgType) throws MessageIDException { + + int tdMsgVer = 1; + if (MsgType.MSG_MULTI_BODY_ATTR.equals(msgType)) { + tdMsgVer = 3; + } else if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType)) { + tdMsgVer = 4; + } + + for (Map.Entry>> topicEntry : messageMap.entrySet()) { + for (Map.Entry> tidEntry : topicEntry.getValue().entrySet()) { + + TDMsg1 tdMsg = TDMsg1.newTDMsg(this.isCompressed, tdMsgVer); + Map headers = new HashMap(); + for (ProxyMessage message : tidEntry.getValue()) { + if (MsgType.MSG_MULTI_BODY_ATTR.equals(msgType) || MsgType.MSG_MULTI_BODY.equals(msgType)) { + message.getAttributeMap().put(AttributeConstants.MESSAGE_COUNT, String.valueOf(1)); + tdMsg.addMsg(mapJoiner.join(message.getAttributeMap()), message.getData()); + } else if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType)) { + tdMsg.addMsg(message.getData()); + } else { + tdMsg.addMsg(mapJoiner.join(message.getAttributeMap()), message.getData()); + } + } + + + long pkgTimeInMillis = tdMsg.getCreatetime(); + String pkgTimeStr = dateFormator.get().format(pkgTimeInMillis); + + if (tdMsgVer == 4) { + if (commonAttrMap.containsKey(ConfigConstants.PKG_TIME_KEY)) { + pkgTimeStr = commonAttrMap.get(ConfigConstants.PKG_TIME_KEY); + } else { + pkgTimeStr = dateFormator.get().format(System.currentTimeMillis()); + } + } + + + if (commonAttrMap.get(AttributeConstants.DATA_TIME) != null) { + headers.put(AttributeConstants.DATA_TIME, commonAttrMap.get(AttributeConstants.DATA_TIME)); + } else { + headers.put(AttributeConstants.DATA_TIME, String.valueOf(System.currentTimeMillis())); + } + + headers.put(ConfigConstants.TOPIC_KEY, topicEntry.getKey()); + headers.put(AttributeConstants.INTERFACE_ID, tidEntry.getKey()); + headers.put(ConfigConstants.REMOTE_IP_KEY, strRemoteIP); + headers.put(ConfigConstants.REMOTE_IDC_KEY, DEFAULT_REMOTE_IDC_VALUE); + // every message share the same msg cnt? what if msgType = 5 + String proxyMetricMsgCnt = commonAttrMap.get(AttributeConstants.MESSAGE_COUNT); + headers.put(ConfigConstants.MSG_COUNTER_KEY, proxyMetricMsgCnt); + + + byte[] data = tdMsg.buildArray(); + headers.put(ConfigConstants.TOTAL_LEN, String.valueOf(data.length)); + + String sequenceId = commonAttrMap.get(AttributeConstants.SEQUENCE_ID); + if (StringUtils.isNotEmpty(sequenceId)) { + + + StringBuilder sidBuilder = new StringBuilder(); + sidBuilder.append(topicEntry.getKey()).append(SEPARATOR).append(tidEntry.getKey()) + .append(SEPARATOR).append(sequenceId); + headers.put(ConfigConstants.SEQUENCE_ID, sidBuilder.toString()); + } + + headers.put(ConfigConstants.PKG_TIME_KEY, pkgTimeStr); + Event event = EventBuilder.withBody(data, headers); + + long dtten = 0; + try { + dtten = Long.parseLong(headers.get(AttributeConstants.DATA_TIME)); + } catch (Exception e1) { + long uniqVal = Long.parseLong(commonAttrMap.get(AttributeConstants.UNIQ_ID)); + throw new MessageIDException(uniqVal, + ErrorCode.DT_ERROR, + new Throwable("attribute dt=" + headers.get(AttributeConstants.DATA_TIME + + " has error, detail is: topic=" + topicEntry.getKey() + "&tid=" + + tidEntry.getKey() + "&NodeIP=" + strRemoteIP), e1)); + } + + dtten = dtten / 1000 / 60 / 10; + dtten = dtten * 1000 * 60 * 10; + try { + processor.processEvent(event); + } catch (Throwable ex) { + logger.error("Error writting to channel,data will discard.", ex); + + throw new ChannelException("ProcessEvent error can't write event to channel."); + } + } + } + } + + private void responsePackage(Map commonAttrMap, + Map resultMap, + Channel remoteChannel, + SocketAddress remoteSocketAddress, + MsgType msgType) throws Exception { + if (!commonAttrMap.containsKey("isAck") || "true".equals(commonAttrMap.get("isAck"))) { + if (MsgType.MSG_ACK_SERVICE.equals(msgType) || MsgType.MSG_ORIGINAL_RETURN + .equals(msgType) + || MsgType.MSG_MULTI_BODY.equals(msgType) || MsgType.MSG_MULTI_BODY_ATTR + .equals(msgType)) { + byte[] backAttr = mapJoiner.join(commonAttrMap).getBytes(StandardCharsets.UTF_8); + byte[] backBody = null; + + if (backAttr != null && !new String(backAttr, StandardCharsets.UTF_8).isEmpty()) { + if (MsgType.MSG_ORIGINAL_RETURN.equals(msgType)) { + + backBody = (byte[]) resultMap.get(ConfigConstants.DECODER_BODY); + } else { + + backBody = new byte[]{50}; + } + int backTotalLen = 1 + 4 + backBody.length + 4 + backAttr.length; + ChannelBuffer buffer = ChannelBuffers.buffer(4 + backTotalLen); + buffer.writeInt(backTotalLen); + buffer.writeByte(msgType.getValue()); + buffer.writeInt(backBody.length); + buffer.writeBytes(backBody); + buffer.writeInt(backAttr.length); + buffer.writeBytes(backAttr); + if (remoteChannel.isWritable()) { + remoteChannel.write(buffer, remoteSocketAddress); + } else { + String backAttrStr = new String(backAttr, StandardCharsets.UTF_8); + logger.warn( + "the send buffer1 is full, so disconnect it!please check remote client" + + "; Connection info:" + + remoteChannel + ";attr is " + backAttrStr); + throw new Exception(new Throwable( + "the send buffer1 is full, so disconnect it!please check remote client" + + + "; Connection info:" + remoteChannel + ";attr is " + + backAttrStr)); + } + } + } else if (MsgType.MSG_BIN_MULTI_BODY.equals(msgType)) { + String backattrs = null; + if (resultMap.containsKey(ConfigConstants.DECODER_ATTRS)) { + backattrs = (String) resultMap.get(ConfigConstants.DECODER_ATTRS); + } + + int binTotalLen = 1 + 4 + 2 + 2; + if (null != backattrs) { + binTotalLen += backattrs.length(); + } + + ChannelBuffer binBuffer = ChannelBuffers.buffer(4 + binTotalLen); + binBuffer.writeInt(binTotalLen); + binBuffer.writeByte(msgType.getValue()); + + long uniqVal = Long.parseLong(commonAttrMap.get(AttributeConstants.UNIQ_ID)); + byte[] uniq = new byte[4]; + uniq[0] = (byte) ((uniqVal >> 24) & 0xFF); + uniq[1] = (byte) ((uniqVal >> 16) & 0xFF); + uniq[2] = (byte) ((uniqVal >> 8) & 0xFF); + uniq[3] = (byte) (uniqVal & 0xFF); + binBuffer.writeBytes(uniq); + + if (null != backattrs) { + binBuffer.writeShort(backattrs.length()); + binBuffer.writeBytes(backattrs.getBytes(StandardCharsets.UTF_8)); + } else { + binBuffer.writeShort(0x0); + } + + binBuffer.writeShort(0xee01); + if (remoteChannel.isWritable()) { + remoteChannel.write(binBuffer, remoteSocketAddress); + } else { + logger.warn( + "the send buffer2 is full, so disconnect it!please check remote client" + + "; Connection info:" + remoteChannel + ";attr is " + + backattrs); + throw new Exception(new Throwable( + "the send buffer2 is full,so disconnect it!please check remote client, Connection info:" + + remoteChannel + ";attr is " + backattrs)); + } + } + } + } + + @Override + public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception { + logger.info("message received"); + if (e == null) { + logger.error("get null messageevent, just skip"); + return; + } + ChannelBuffer cb = ((ChannelBuffer) e.getMessage()); + String strRemoteIP = getRemoteIp(e.getChannel()); + SocketAddress remoteSocketAddress = e.getRemoteAddress(); + int len = cb.readableBytes(); + if (len == 0 && this.filterEmptyMsg) { + logger.warn("skip empty msg."); + cb.clear(); + return; + } + + Channel remoteChannel = e.getChannel(); + Map resultMap = null; + try { + resultMap = serviceProcessor.extractData(cb, remoteChannel); + } catch (MessageIDException ex) { + throw new IOException(ex.getCause()); + } + + if (resultMap == null) { + logger.info("result is null"); + return; + } + + MsgType msgType = (MsgType) resultMap.get(ConfigConstants.MSG_TYPE); + if (MsgType.MSG_HEARTBEAT.equals(msgType)) { + remoteChannel.write(heartbeatBuffer, remoteSocketAddress); + return; + } + + if (MsgType.MSG_BIN_HEARTBEAT.equals(msgType)) { +// ChannelBuffer binBuffer = getBinHeart(resultMap,msgType); +// remoteChannel.write(binBuffer, remoteSocketAddress); + return; + } + + Map commonAttrMap = + (Map) resultMap.get(ConfigConstants.COMMON_ATTR_MAP); + if (commonAttrMap == null) { + commonAttrMap = new HashMap(); + } + + List msgList = (List) resultMap.get(ConfigConstants.MSG_LIST); + if (msgList != null + && !commonAttrMap.containsKey(ConfigConstants.FILE_CHECK_DATA) + && !commonAttrMap.containsKey(ConfigConstants.MINUTE_CHECK_DATA)) { + Map>> messageMap = + new HashMap>>( + msgList.size()); + + updateMsgList(msgList, commonAttrMap, messageMap, strRemoteIP, msgType); + + formatMessagesAndSend(commonAttrMap, messageMap, strRemoteIP, msgType); + + } else if (msgList != null && commonAttrMap.containsKey(ConfigConstants.FILE_CHECK_DATA)) { +// logger.info("i am in FILE_CHECK_DATA "); + Map headers = new HashMap(); + headers.put("msgtype", "filestatus"); + headers.put(ConfigConstants.FILE_CHECK_DATA, + "true"); + for (ProxyMessage message : msgList) { + byte[] body = message.getData(); +// logger.info("data:"+new String(body)); + Event event = EventBuilder.withBody(body, headers); + try { + processor.processEvent(event); + } catch (Throwable ex) { + logger.error("Error writing to controller,data will discard.", ex); + + throw new ChannelException( + "Process Controller Event error can't write event to channel."); + } + } + } else if (msgList != null && commonAttrMap + .containsKey(ConfigConstants.MINUTE_CHECK_DATA)) { + logger.info("i am in MINUTE_CHECK_DATA"); + Map headers = new HashMap(); + headers.put("msgtype", "measure"); + headers.put(ConfigConstants.FILE_CHECK_DATA, + "true"); + for (ProxyMessage message : msgList) { + byte[] body = message.getData(); +// logger.info("data:"+new String(body)); + Event event = EventBuilder.withBody(body, headers); + try { + processor.processEvent(event); + } catch (Throwable ex) { + logger.error("Error writing to controller,data will discard.", ex); + + + throw new ChannelException( + "Process Controller Event error can't write event to channel."); + } + } + } + responsePackage(commonAttrMap, resultMap, remoteChannel, remoteSocketAddress, msgType); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) throws Exception { + logger.error("exception caught", e.getCause()); + } + + @Override + public void channelClosed(ChannelHandlerContext ctx, ChannelStateEvent e) throws Exception { + logger.error("channel closed {}", ctx.getChannel()); + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServiceDecoder.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServiceDecoder.java new file mode 100644 index 00000000000..eb0226fac9b --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/ServiceDecoder.java @@ -0,0 +1,42 @@ +/* + * 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.inlong.dataproxy.source; + +import java.util.Map; + +import org.jboss.netty.buffer.ChannelBuffer; +import org.jboss.netty.channel.Channel; + +/** + * decoder interface definition + * + */ +public interface ServiceDecoder { + + int HEAD_LENGTH = 4; + + /** + * extract data from buffer and convert it into map. + * + * @param cb + * @param channel + * @return + * @throws + */ + Map extractData(ChannelBuffer cb, Channel channel) throws Exception; +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleTcpSource.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleTcpSource.java new file mode 100644 index 00000000000..d1643d8c414 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/source/SimpleTcpSource.java @@ -0,0 +1,374 @@ +/* + * 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.inlong.dataproxy.source; + +import com.google.common.base.Preconditions; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.io.UnsupportedEncodingException; +import java.lang.reflect.Constructor; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.Iterator; +import java.util.concurrent.Executors; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.flume.Context; +import org.apache.flume.EventDrivenSource; +import org.apache.flume.FlumeException; +import org.apache.flume.channel.ChannelProcessor; +import org.apache.flume.conf.Configurable; +import org.apache.flume.conf.Configurables; +import org.apache.flume.source.AbstractSource; +import org.apache.inlong.dataproxy.base.NamedThreadFactory; +import org.apache.inlong.dataproxy.consts.ConfigConstants; +import org.jboss.netty.bootstrap.ServerBootstrap; +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelFactory; +import org.jboss.netty.channel.ChannelPipelineFactory; +import org.jboss.netty.channel.group.ChannelGroup; +import org.jboss.netty.channel.group.DefaultChannelGroup; +import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; +import org.jboss.netty.util.ThreadNameDeterminer; +import org.jboss.netty.util.ThreadRenamingRunnable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Simple tcp source + * + */ +public class SimpleTcpSource extends AbstractSource implements Configurable, EventDrivenSource { + + private static final Logger logger = LoggerFactory.getLogger(SimpleTcpSource.class); + public static ArrayList blacklist = new ArrayList(); + private static final String blacklistFilePath = "blacklist.properties"; + private static long propsLastModified; + private static final String CONNECTIONS = "connections"; + + + protected int maxConnections = Integer.MAX_VALUE; + private ServerBootstrap serverBootstrap = null; + protected ChannelGroup allChannels; + protected int port; + protected String host = null; + protected String msgFactoryName; + protected String serviceDecoderName; + protected String messageHandlerName; + protected int maxMsgLength; + protected boolean isCompressed; + private CheckBlackListThread checkBlackListThread; + private int maxThreads = 32; + + private boolean tcpNoDelay = true; + private boolean keepAlive = true; + private int receiveBufferSize; + private int highWaterMark; + private int sendBufferSize; + private int trafficClass; + + protected String topic; + protected String attr; + protected boolean filterEmptyMsg; + + private Channel nettyChannel = null; + + public SimpleTcpSource() { + super(); + allChannels = new DefaultChannelGroup(); + } + + /** + * check black list + * @param blacklist + * @param allChannels + */ + public static void checkBlackList(ArrayList blacklist, ChannelGroup allChannels) { + if (blacklist != null) { + Iterator it = allChannels.iterator(); + while (it.hasNext()) { + Channel channel = it.next(); + String strRemoteIP = null; + SocketAddress remoteSocketAddress = channel.getRemoteAddress(); + if (null != remoteSocketAddress) { + strRemoteIP = remoteSocketAddress.toString(); + try { + strRemoteIP = strRemoteIP.substring(1, strRemoteIP.indexOf(':')); + } catch (Exception ee) { + logger.warn("fail to get the remote IP, and strIP={},remoteSocketAddress={}", strRemoteIP, + remoteSocketAddress); + } + } + if (strRemoteIP != null && blacklist.contains(strRemoteIP)) { + logger.error(strRemoteIP + " is in blacklist, so disconnect it !"); + channel.disconnect(); + channel.close(); + allChannels.remove(channel); + } + } + } + } + + private ArrayList load(String fileName) { + ArrayList arrayList = new ArrayList(); + if (fileName == null) { + logger.error("fail to loadProperties, filename is null"); + return arrayList; + } + FileReader reader = null; + BufferedReader br = null; + try { + reader = new FileReader("conf/" + fileName); + br = new BufferedReader(reader); + String line = null; + while ((line = br.readLine()) != null) { + arrayList.add(line); + } + } catch (UnsupportedEncodingException e) { + logger.error("fail to loadPropery, file ={}, and e= {}", fileName, e); + } catch (Exception e) { + logger.error("fail to loadProperty, file ={}, and e= {}", fileName, e); + } finally { + IOUtils.closeQuietly(reader); + IOUtils.closeQuietly(br); + } + return arrayList; + } + + + private class CheckBlackListThread extends Thread { + private boolean shutdown = false; + + public void shutdouwn() { + shutdown = true; + } + + @Override + public void run() { + logger.info("CheckBlackListThread thread {} start.", Thread.currentThread().getName()); + while (!shutdown) { + try { + File blacklistFile = new File("conf/" + blacklistFilePath); + if (blacklistFile.lastModified() > propsLastModified) { + blacklist = load(blacklistFilePath); + propsLastModified = blacklistFile.lastModified(); + SimpleDateFormat formator = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + logger.info("blacklist.properties:{}\n{}", + formator.format(new Date(blacklistFile.lastModified())), blacklist); + } + Thread.sleep(5 * 1000); + checkBlackList(blacklist, allChannels); + } catch (InterruptedException e) { + logger.info("ConfigReloader thread exit!"); + return; + } catch (Throwable t) { + logger.error("ConfigReloader thread error!", t); + } + } + } + } + + @Override + public synchronized void start() { + logger.info("start " + this.getName()); + checkBlackListThread = new CheckBlackListThread(); + checkBlackListThread.start(); + super.start(); + + ThreadRenamingRunnable.setThreadNameDeterminer(ThreadNameDeterminer.CURRENT); + ChannelFactory factory = + new NioServerSocketChannelFactory( + Executors.newCachedThreadPool( + new NamedThreadFactory("tcpSource-nettyBoss-threadGroup")), 1, + Executors.newCachedThreadPool( + new NamedThreadFactory("tcpSource-nettyWorker-threadGroup")), maxThreads); + logger.info("Set max workers : {} ;", maxThreads); + ChannelPipelineFactory fac = null; + + serverBootstrap = new ServerBootstrap(factory); + serverBootstrap.setOption("child.tcpNoDelay", tcpNoDelay); + serverBootstrap.setOption("child.keepAlive", keepAlive); + serverBootstrap.setOption("child.receiveBufferSize", receiveBufferSize); + serverBootstrap.setOption("child.sendBufferSize", sendBufferSize); + serverBootstrap.setOption("child.trafficClass", trafficClass); + serverBootstrap.setOption("child.writeBufferHighWaterMark", highWaterMark); + logger.info("load msgFactory=" + msgFactoryName + + " and serviceDecoderName=" + serviceDecoderName); + try { + + ServiceDecoder serviceDecoder = + (ServiceDecoder) Class.forName(serviceDecoderName).newInstance(); + + Class clazz = + (Class) Class.forName(msgFactoryName); + + Constructor ctor = + clazz.getConstructor(ChannelProcessor.class, ChannelGroup.class, + String.class, ServiceDecoder.class, String.class, + Integer.class, String.class, String.class, Boolean.class, + Integer.class, Boolean.class, String.class); + + logger.info("Using channel processor:{}", getChannelProcessor().getClass().getName()); + fac = (ChannelPipelineFactory) ctor.newInstance(getChannelProcessor(), allChannels, + "tcp", serviceDecoder, messageHandlerName, + maxMsgLength, topic, attr, filterEmptyMsg, maxConnections, isCompressed, this.getName()); + + + } catch (Exception e) { + logger.error("Simple Tcp Source start error, fail to construct ChannelPipelineFactory with name {}, ex {}", + msgFactoryName, e); + stop(); + throw new FlumeException(e.getMessage()); + } + + serverBootstrap.setPipelineFactory(fac); + + try { + if (host == null) { + nettyChannel = serverBootstrap.bind(new InetSocketAddress(port)); + } else { + nettyChannel = serverBootstrap.bind(new InetSocketAddress(host, port)); + } + } catch (Exception e) { + logger.error("Simple TCP Source error bind host {} port {},program will exit!", host, port); + System.exit(-1); + } + + allChannels.add(nettyChannel); + + logger.info("Simple TCP Source started at host {}, port {}", host, port); + + } + + @Override + public synchronized void stop() { + logger.info("[STOP SOURCE]{} stopping...", super.getName()); + checkBlackListThread.shutdouwn(); + if (allChannels != null && !allChannels.isEmpty()) { + try { + allChannels.unbind().awaitUninterruptibly(); + allChannels.close().awaitUninterruptibly(); + } catch (Exception e) { + logger.warn("Simple TCP Source netty server stop ex", e); + } finally { + allChannels.clear(); + // allChannels = null; + } + } + + if (serverBootstrap != null) { + try { + + serverBootstrap.releaseExternalResources(); + } catch (Exception e) { + logger.warn("Simple TCP Source serverBootstrap stop ex ", e); + } finally { + serverBootstrap = null; + } + } + + super.stop(); + logger.info("[STOP SOURCE]{} stopped", super.getName()); + } + + @Override + public void configure(Context context) { + logger.info("context is {}", context); + port = context.getInteger(ConfigConstants.CONFIG_PORT); + host = context.getString(ConfigConstants.CONFIG_HOST, "0.0.0.0"); + + tcpNoDelay = context.getBoolean(ConfigConstants.TCP_NO_DELAY, true); + + keepAlive = context.getBoolean(ConfigConstants.KEEP_ALIVE, true); + highWaterMark = context.getInteger(ConfigConstants.HIGH_WATER_MARK, 64 * 1024); + receiveBufferSize = context.getInteger(ConfigConstants.RECEIVE_BUFFER_SIZE, 1024 * 64); + if (receiveBufferSize > 16 * 1024 * 1024) { + receiveBufferSize = 16 * 1024 * 1024; + } + Preconditions.checkArgument(receiveBufferSize > 0, "receiveBufferSize must be > 0"); + + sendBufferSize = context.getInteger(ConfigConstants.SEND_BUFFER_SIZE, 1024 * 64); + if (sendBufferSize > 16 * 1024 * 1024) { + sendBufferSize = 16 * 1024 * 1024; + } + Preconditions.checkArgument(sendBufferSize > 0, "sendBufferSize must be > 0"); + + trafficClass = context.getInteger(ConfigConstants.TRAFFIC_CLASS, 0); + Preconditions.checkArgument((trafficClass == 0 || trafficClass == 96), + "trafficClass must be == 0 or == 96"); + + try { + maxThreads = context.getInteger(ConfigConstants.MAX_THREADS, 32); + } catch (NumberFormatException e) { + logger.warn("Simple TCP Source max-threads property must specify an integer value. {}", + context.getString(ConfigConstants.MAX_THREADS)); + } + + try { + maxConnections = context.getInteger(CONNECTIONS, 5000); + } catch (NumberFormatException e) { + logger.warn("BaseSource\'s \"connections\" property must specify an integer value.", + context.getString(CONNECTIONS)); + } + + topic = context.getString(ConfigConstants.TOPIC); + attr = context.getString(ConfigConstants.ATTR); + Configurables.ensureRequiredNonNull(context, ConfigConstants.TOPIC, ConfigConstants.ATTR); + + topic = topic.trim(); + Preconditions.checkArgument(!topic.isEmpty(), "topic is empty"); + attr = attr.trim(); + Preconditions.checkArgument(!attr.isEmpty(), "attr is empty"); + + filterEmptyMsg = context.getBoolean(ConfigConstants.FILTER_EMPTY_MSG, false); + + + msgFactoryName = + context.getString(ConfigConstants.MSG_FACTORY_NAME, + "org.apache.inlong.dataproxy.source.ServerMessageFactory"); + msgFactoryName = msgFactoryName.trim(); + Preconditions.checkArgument(StringUtils.isNotBlank(msgFactoryName), + "msgFactoryName is empty"); + + serviceDecoderName = + context.getString(ConfigConstants.SERVICE_PROCESSOR_NAME, + "org.apache.inlong.dataproxy.source.DefaultServiceDecoder"); + serviceDecoderName = serviceDecoderName.trim(); + Preconditions.checkArgument(StringUtils.isNotBlank(serviceDecoderName), + "serviceProcessorName is empty"); + + messageHandlerName = + context.getString(ConfigConstants.MESSAGE_HANDLER_NAME, + "org.apache.flume.source.ServerMessageHandler"); + messageHandlerName = messageHandlerName.trim(); + Preconditions.checkArgument(StringUtils.isNotBlank(messageHandlerName), + "messageHandlerName is empty"); + + maxMsgLength = context.getInteger(ConfigConstants.MAX_MSG_LENGTH, 1024 * 64); + Preconditions.checkArgument( + (maxMsgLength >= 4 && maxMsgLength <= ConfigConstants.MSG_MAX_LENGTH_BYTES), + "maxMsgLength must be >= 4 and <= " + ConfigConstants.MSG_MAX_LENGTH_BYTES); + isCompressed = context.getBoolean(ConfigConstants.MSG_COMPRESSED, true); + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/utils/NetworkUtils.java b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/utils/NetworkUtils.java new file mode 100644 index 00000000000..ebb3baec8eb --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/java/org/apache/inlong/dataproxy/utils/NetworkUtils.java @@ -0,0 +1,75 @@ +/* + * 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.inlong.dataproxy.utils; + +import java.net.InetAddress; +import java.net.NetworkInterface; +import java.net.SocketException; +import java.util.Enumeration; + +public class NetworkUtils { + public static String INNER_NETWORK_INTERFACE = "eth1"; + + private static String localIp = null; + + static { + localIp = getLocalIp(); + } + + /** + * get local ip + * @return + */ + public static String getLocalIp() { + if (localIp == null) { + String ip = null; + Enumeration allInterface; + try { + allInterface = NetworkInterface.getNetworkInterfaces(); + for (; allInterface.hasMoreElements(); ) { + NetworkInterface oneInterface = allInterface.nextElement(); + String interfaceName = oneInterface.getName(); + if (oneInterface.isLoopback() + || !oneInterface.isUp() + || !interfaceName + .equalsIgnoreCase(INNER_NETWORK_INTERFACE)) { + continue; + } + + Enumeration allAddress = oneInterface + .getInetAddresses(); + for (; allAddress.hasMoreElements(); ) { + InetAddress oneAddress = allAddress.nextElement(); + ip = oneAddress.getHostAddress(); + if (ip == null || ip.isEmpty() || ip.equals("127.0.0.1")) { + continue; + } + return ip; + } + } + } catch (SocketException e1) { + // ignore it + } + + ip = "0.0.0.0"; + localIp = ip; + } + + return localIp; + } +} diff --git a/inlong-dataproxy/dataproxy-source/src/main/resources/flume.conf b/inlong-dataproxy/dataproxy-source/src/main/resources/flume.conf new file mode 100644 index 00000000000..d0aa15e46d7 --- /dev/null +++ b/inlong-dataproxy/dataproxy-source/src/main/resources/flume.conf @@ -0,0 +1,272 @@ +# +# 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. +# + +#6-segment protocol TCP source +agent1.sources.tcp-source.channels = ch-msg1 ch-msg2 ch-msg3 ch-more1 ch-more2 ch-more3 ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9 ch-msg10 ch-back +agent1.sources.tcp-source.type = org.apache.inlong.dataproxy.source.SimpleTcpSource +agent1.sources.tcp-source.msg-factory-name = org.apache.inlong.dataproxy.source.ServerMessageFactory +agent1.sources.tcp-source.host = 0.0.0.0 +agent1.sources.tcp-source.port = 46801 +agent1.sources.tcp-source.highWaterMark=2621440 +agent1.sources.tcp-source.enableExceptionReturn=true +agent1.sources.tcp-source.max-msg-length = 524288 +agent1.sources.tcp-source.topic = test_token +agent1.sources.tcp-source.attr = m=9 +agent1.sources.tcp-source.connections = 5000 +agent1.sources.tcp-source.max-threads = 64 +agent1.sources.tcp-source.receiveBufferSize = 524288 +agent1.sources.tcp-source.sendBufferSize = 524288 +agent1.sources.tcp-source.custom-cp = true +agent1.sources.tcp-source.selector.type = org.apache.inlong.dataproxy.channel.FailoverChannelSelector +agent1.sources.tcp-source.selector.master = ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9 +agent1.sources.tcp-source.selector.transfer = ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9 +agent1.sources.tcp-source.selector.fileMetric = ch-back +agent1.sources.tcp-source.selector.slaMetric = ch-msg10 +agent1.sources.tcp-source.metric-recovery-path=FLUME_HOME/metric +agent1.sources.tcp-source.metric-agent-port=8003 +agent1.sources.tcp-source.metric-cache-size=10000 +agent1.sources.tcp-source.set=10 +agent1.sources.tcp-source.old-metric-on=true +agent1.sources.tcp-source.new-metric-on=true +agent1.sources.tcp-source.metric_topic_prefix=manager_tmertic + +agent1.channels.ch-back.type = memory +agent1.channels.ch-back.capacity = 10000000 +agent1.channels.ch-back.keep-alive = 0 +agent1.channels.ch-back.transactionCapacity = 200 + +agent1.channels.ch-msg1.type = memory +agent1.channels.ch-msg1.capacity = 200000 +agent1.channels.ch-msg1.keep-alive = 0 +agent1.channels.ch-msg1.transactionCapacity = 200 + +agent1.channels.ch-msg2.type = memory +agent1.channels.ch-msg2.capacity = 200000 +agent1.channels.ch-msg2.keep-alive = 0 +agent1.channels.ch-msg2.transactionCapacity = 200 + +agent1.channels.ch-msg3.type = memory +agent1.channels.ch-msg3.capacity = 200000 +agent1.channels.ch-msg3.keep-alive = 0 +agent1.channels.ch-msg3.transactionCapacity = 200 + +agent1.channels.ch-msg5.type = file +agent1.channels.ch-msg5.capacity = 100000000 +agent1.channels.ch-msg5.maxFileSize = 1073741824 +agent1.channels.ch-msg5.minimumRequiredSpace = 1073741824 +agent1.channels.ch-msg5.checkpointDir = FLUME_HOME/file/ch-msg5/check +agent1.channels.ch-msg5.dataDirs = FLUME_HOME/file/ch-msg5/data +agent1.channels.ch-msg5.fsyncPerTransaction = false +agent1.channels.ch-msg5.fsyncInterval = 5 + +agent1.channels.ch-msg6.type = file +agent1.channels.ch-msg6.capacity = 100000000 +agent1.channels.ch-msg6.maxFileSize = 1073741824 +agent1.channels.ch-msg6.minimumRequiredSpace = 1073741824 +agent1.channels.ch-msg6.checkpointDir = FLUME_HOME/file/ch-msg6/check +agent1.channels.ch-msg6.dataDirs = FLUME_HOME/file/ch-msg6/data +agent1.channels.ch-msg6.fsyncPerTransaction = false +agent1.channels.ch-msg6.fsyncInterval = 5 + +agent1.channels.ch-msg7.type = file +agent1.channels.ch-msg7.capacity = 100000000 +agent1.channels.ch-msg7.maxFileSize = 1073741824 +agent1.channels.ch-msg7.minimumRequiredSpace = 1073741824 +agent1.channels.ch-msg7.checkpointDir = FLUME_HOME/file/ch-msg7/check +agent1.channels.ch-msg7.dataDirs = FLUME_HOME/file/ch-msg7/data +agent1.channels.ch-msg7.fsyncPerTransaction = false +agent1.channels.ch-msg7.fsyncInterval = 5 + +agent1.channels.ch-msg8.type = file +agent1.channels.ch-msg8.capacity = 100000000 +agent1.channels.ch-msg8.maxFileSize = 1073741824 +agent1.channels.ch-msg8.minimumRequiredSpace = 1073741824 +agent1.channels.ch-msg8.checkpointDir = FLUME_HOME/file/ch-msg8/check +agent1.channels.ch-msg8.dataDirs = FLUME_HOME/file/ch-msg8/data +agent1.channels.ch-msg8.fsyncPerTransaction = false +agent1.channels.ch-msg8.fsyncInterval = 5 + +agent1.channels.ch-msg9.type = file +agent1.channels.ch-msg9.capacity = 100000000 +agent1.channels.ch-msg9.maxFileSize = 1073741824 +agent1.channels.ch-msg9.minimumRequiredSpace = 1073741824 +agent1.channels.ch-msg9.checkpointDir = FLUME_HOME/file/ch-msg9/check +agent1.channels.ch-msg9.dataDirs = FLUME_HOME/file/ch-msg9/data +agent1.channels.ch-msg9.fsyncPerTransaction = false +agent1.channels.ch-msg9.fsyncInterval = 5 + +agent1.channels.ch-msg10.type = file +agent1.channels.ch-msg10.capacity = 100000000 +agent1.channels.ch-msg10.maxFileSize = 1073741824 +agent1.channels.ch-msg10.minimumRequiredSpace = 1073741824 +agent1.channels.ch-msg10.checkpointDir = FLUME_HOME/file/ch-msg10/check +agent1.channels.ch-msg10.dataDirs = FLUME_HOME/file/ch-msg10/data +agent1.channels.ch-msg10.fsyncPerTransaction = false +agent1.channels.ch-msg10.fsyncInterval = 5 + +agent1.sinks.meta-sink-more1.channel = ch-msg1 +agent1.sinks.meta-sink-more1.type = org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-more1.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-more1.send_timeout = 30000 +agent1.sinks.meta-sink-more1.stat-interval-sec = 60 +agent1.sinks.meta-sink-more1.thread-num = 8 +agent1.sinks.meta-sink-more1.client-id-cache = true +agent1.sinks.meta-sink-more1.max-survived-time = 300000 +agent1.sinks.meta-sink-more1.max-survived-size = 3000000 +agent1.sinks.meta-sink-more1.new-check-pattern = true +agent1.sinks.meta-sink-more1.old-metric-on=true +agent1.sinks.meta-sink-more1.set=10 + +agent1.sinks.meta-sink-more2.channel = ch-more2 +agent1.sinks.meta-sink-more2.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-more2.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-more2.send_timeout = 30000 +agent1.sinks.meta-sink-more2.stat-interval-sec = 60 +agent1.sinks.meta-sink-more2.thread-num = 8 +agent1.sinks.meta-sink-more2.client-id-cache = true +agent1.sinks.meta-sink-more2.max-survived-time = 300000 +agent1.sinks.meta-sink-more2.max-survived-size = 3000000 +agent1.sinks.meta-sink-more2.new-check-pattern = true +agent1.sinks.meta-sink-more2.old-metric-on=true +agent1.sinks.meta-sink-more2.set=10 + +agent1.sinks.meta-sink-more3.channel = ch-more3 +agent1.sinks.meta-sink-more3.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-more3.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-more3.send_timeout = 30000 +agent1.sinks.meta-sink-more3.stat-interval-sec = 60 +agent1.sinks.meta-sink-more3.thread-num = 8 +agent1.sinks.meta-sink-more3.client-id-cache = true +agent1.sinks.meta-sink-more3.max-survived-time = 300000 +agent1.sinks.meta-sink-more3.max-survived-size = 3000000 +agent1.sinks.meta-sink-more3.new-check-pattern = true +agent1.sinks.meta-sink-more3.old-metric-on=true +agent1.sinks.meta-sink-more3.set=10 + +agent1.sinks.meta-sink-most1.channel = ch-most1 +agent1.sinks.meta-sink-most1.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-most1.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-most1.send_timeout = 30000 +agent1.sinks.meta-sink-most1.stat-interval-sec = 60 +agent1.sinks.meta-sink-most1.thread-num = 8 +agent1.sinks.meta-sink-most1.client-id-cache = true +agent1.sinks.meta-sink-most1.max-survived-time = 300000 +agent1.sinks.meta-sink-most1.max-survived-size = 3000000 +agent1.sinks.meta-sink-most1.new-check-pattern = true +agent1.sinks.meta-sink-most1.old-metric-on=true +agent1.sinks.meta-sink-most1.set=10 + +agent1.sinks.meta-sink-most2.channel = ch-most2 +agent1.sinks.meta-sink-most2.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-most2.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-most2.send_timeout = 30000 +agent1.sinks.meta-sink-most2.stat-interval-sec = 60 +agent1.sinks.meta-sink-most2.thread-num = 8 +agent1.sinks.meta-sink-most2.client-id-cache = true +agent1.sinks.meta-sink-most2.max-survived-time = 300000 +agent1.sinks.meta-sink-most2.max-survived-size = 3000000 +agent1.sinks.meta-sink-most2.new-check-pattern = true +agent1.sinks.meta-sink-most2.old-metric-on=true +agent1.sinks.meta-sink-most2.set=10 + +agent1.sinks.meta-sink-most3.channel = ch-most3 +agent1.sinks.meta-sink-most3.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-most3.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-most3.send_timeout = 30000 +agent1.sinks.meta-sink-most3.stat-interval-sec = 60 +agent1.sinks.meta-sink-most3.thread-num = 8 +agent1.sinks.meta-sink-most3.client-id-cache = true +agent1.sinks.meta-sink-most3.max-survived-time = 300000 +agent1.sinks.meta-sink-most3.max-survived-size = 3000000 +agent1.sinks.meta-sink-most3.new-check-pattern = true +agent1.sinks.meta-sink-most3.old-metric-on=true +agent1.sinks.meta-sink-most3.set=10 + + +agent1.sinks.meta-sink-msg1.channel = ch-msg1 +agent1.sinks.meta-sink-msg1.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-msg1.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-msg1.send_timeout = 30000 +agent1.sinks.meta-sink-msg1.stat-interval-sec = 60 +agent1.sinks.meta-sink-msg1.thread-num = 8 +agent1.sinks.meta-sink-msg1.client-id-cache = true +agent1.sinks.meta-sink-msg1.max-survived-time = 300000 +agent1.sinks.meta-sink-msg1.max-survived-size = 3000000 +agent1.sinks.meta-sink-msg1.new-check-pattern = true +agent1.sinks.meta-sink-msg1.old-metric-on=true +agent1.sinks.meta-sink-msg1.set=10#tonglevip + +agent1.sinks.meta-sink-msg2.channel = ch-msg2 +agent1.sinks.meta-sink-msg2.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-msg2.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-msg2.send_timeout = 30000 +agent1.sinks.meta-sink-msg2.stat-interval-sec = 60 +agent1.sinks.meta-sink-msg2.thread-num = 8 +agent1.sinks.meta-sink-msg2.client-id-cache = true +agent1.sinks.meta-sink-msg2.max-survived-time = 300000 +agent1.sinks.meta-sink-msg2.max-survived-size = 3000000 +agent1.sinks.meta-sink-msg2.new-check-pattern = true +agent1.sinks.meta-sink-msg2.old-metric-on=true +agent1.sinks.meta-sink-msg2.set=10#tonglevip + +agent1.sinks.meta-sink-msg3.channel = ch-msg3 +agent1.sinks.meta-sink-msg3.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-msg3.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-msg3.send_timeout = 30000 +agent1.sinks.meta-sink-msg3.stat-interval-sec = 60 +agent1.sinks.meta-sink-msg3.thread-num = 8 +agent1.sinks.meta-sink-msg3.client-id-cache = true +agent1.sinks.meta-sink-msg3.max-survived-time = 300000 +agent1.sinks.meta-sink-msg3.max-survived-size = 3000000 +agent1.sinks.meta-sink-msg3.new-check-pattern = true +agent1.sinks.meta-sink-msg3.old-metric-on=true +agent1.sinks.meta-sink-msg3.set=10#tonglevip + +agent1.sinks.meta-sink-msg8.channel = ch-msg1 +agent1.sinks.meta-sink-msg8.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-msg8.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-msg8.send_timeout = 30000 +agent1.sinks.meta-sink-msg8.stat-interval-sec = 60 +agent1.sinks.meta-sink-msg8.thread-num = 8 +agent1.sinks.meta-sink-msg8.client-id-cache = true +agent1.sinks.meta-sink-msg8.max-survived-time = 300000 +agent1.sinks.meta-sink-msg8.max-survived-size = 3000000 +agent1.sinks.meta-sink-msg8.new-check-pattern = true +agent1.sinks.meta-sink-msg8.old-metric-on=true +agent1.sinks.meta-sink-msg8.set=10 +agent1.sinks.meta-sink-msg8.disk-io-rate-per-sec=20000000 + +agent1.sinks.meta-sink-msg8.channel = ch-back +agent1.sinks.meta-sink-msg8.type =org.apache.inlong.dataproxy.sink.MetaSink +agent1.sinks.meta-sink-msg8.master-host-port-list = TUBE_LIST +agent1.sinks.meta-sink-msg8.send_timeout = 30000 +agent1.sinks.meta-sink-msg8.stat-interval-sec = 60 +agent1.sinks.meta-sink-msg8.thread-num = 8 +agent1.sinks.meta-sink-msg8.client-id-cache = true +agent1.sinks.meta-sink-msg8.max-survived-time = 300000 +agent1.sinks.meta-sink-msg8.max-survived-size = 3000000 +agent1.sinks.meta-sink-msg8.new-check-pattern = true +agent1.sinks.meta-sink-msg8.old-metric-on=true +agent1.sinks.meta-sink-msg8.set=10 +agent1.sinks.meta-sink-msg8.disk-io-rate-per-sec=20000000 + +agent1.sources = tcp-source +agent1.channels = ch-back ch-msg1 ch-msg2 ch-msg3 ch-msg5 ch-msg6 ch-msg7 ch-msg8 ch-msg9 ch-msg10 +agent1.sinks = meta-sink-msg1 meta-sink-msg2 meta-sink-msg3 meta-sink-msg5 meta-sink-msg6 meta-sink-msg7 meta-sink-msg8 meta-sink-transfer + diff --git a/inlong-dataproxy/pom.xml b/inlong-dataproxy/pom.xml new file mode 100644 index 00000000000..6aeaf62d23c --- /dev/null +++ b/inlong-dataproxy/pom.xml @@ -0,0 +1,96 @@ + + + + + 4.0.0 + + org.apache.inlong + inlong + 0.9.0-incubating-SNAPSHOT + + pom + inlong-dataproxy + Apache InLong - DataProxy + + + dataproxy-source + dataproxy-dist + + + + 8 + 8 + 1.9.0 + 3.8.0.Final + 1.15 + 2.5-20110124 + 3.8.6 + 1.3.4 + + + + + org.apache.flume + flume-ng-core + ${flume.version} + + + org.apache.flume + flume-ng-node + ${flume.version} + + + org.apache.flume + flume-ng-sdk + ${flume.version} + + + org.apache.flume + flume-ng-configuration + ${flume.version} + + + io.netty + netty + ${netty.version} + + + commons-codec + commons-codec + ${codec.version} + + + org.mortbay.jetty + servlet-api + ${servlet.version} + + + org.apache.tubemq + tubemq-client + 0.8.0-incubating + + + org.apache.inlong + inlong-common + ${project.version} + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index 8e04b63a522..4b958402189 100644 --- a/pom.xml +++ b/pom.xml @@ -86,6 +86,7 @@ inlong-common inlong-tubemq + inlong-dataproxy inlong-manager inlong-sort