/**
* Licensed to Cloudera, Inc. under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Cloudera, Inc. 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 com.cloudera.flume.agent;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.cloudera.flume.conf.Context;
import com.cloudera.flume.conf.FlumeBuilder;
import com.cloudera.flume.conf.FlumeConfiguration;
import com.cloudera.flume.conf.FlumeSpecException;
import com.cloudera.flume.conf.SinkFactory.SinkBuilder;
import com.cloudera.flume.core.Event;
import com.cloudera.flume.core.EventSink;
import com.cloudera.flume.reporter.ReportEvent;
import com.cloudera.flume.reporter.Reportable;
import com.google.common.base.Preconditions;
/**
* This is the default agent sink in a agent/collector architecture.
*
* This is purposely setup as a layer of indirection between the actual
* implementation details to may user configuration simpler. It has a default
* options that come from flume-*.xml configuration file.
*
* For the easiest configuration path, all agents can use this as the default
* sink. This only gives high level options to the user by hiding implementation
* and mechanism details.
*
* The actual implementation may change in the future (for example, thrift may
* be replaced with avro, or we may have a ring of collectors to send to instead
* of a single collector..) but user configurations would not need to change.
*
* TODO (jon) replace with substitution instead of this sink.
*/
public class AgentSink extends EventSink.Base {
static final Logger LOG = LoggerFactory.getLogger(AgentSink.class);
public enum ReliabilityMode {
ENDTOEND, // this does writeahead along with end-to-end acks
// TODO (jon) maybe do acks but no writeahead
DISK_FAILOVER, // this is equivalent to scribe's failover mechanism
BEST_EFFORT, // this is equivalent to syslog's best effort mechanism.
};
public static final String BATCH_COUNT = "batchCount";
public static final String BATCH_MILLIS = "batchMillis";
final EventSink sink;
public AgentSink(Context ctx, String dsthost, int port, ReliabilityMode mode)
throws FlumeSpecException {
Preconditions.checkNotNull(dsthost);
// batching is overhead so if no settings specified, don't do it.
String batchGzDeco = "";
String batchN = ctx.getValue(BATCH_COUNT);
int n = 1;
if (batchN != null) {
n = Integer.parseInt(batchN);
}
String batchLatency = ctx.getValue(BATCH_MILLIS);
int ms = 0; // never time out due to time.
if (batchLatency != null) {
ms = Integer.parseInt(batchLatency);
}
if (n > 1) {
batchGzDeco += " batch(" + n + "," + ms + ") ";
}
if (ctx.getValue("compression") != null) {
// currently ignore all values, just use gzip
batchGzDeco += " gzip ";
}
switch (mode) {
case ENDTOEND: {
String snk = String.format("ackedWriteAhead " + batchGzDeco
+ " stubbornAppend insistentOpen rpcSink(\"%s\", %d)", dsthost, port);
sink = FlumeBuilder.buildSink(ctx, snk);
break;
}
case DISK_FAILOVER: {
// Move these options to the builder.
FlumeConfiguration conf = FlumeConfiguration.get();
long maxSingleBo = conf.getFailoverMaxSingleBackoff();
long initialBo = conf.getFailoverInitialBackoff();
long maxCumulativeBo = conf.getFailoverMaxCumulativeBackoff();
String rpc = String.format("rpcSink(\"%s\", %d)", dsthost, port);
String snk = String.format(batchGzDeco
+ " < %s ? diskFailover insistentAppend "
+ " stubbornAppend insistentOpen(%d,%d,%d) %s >", rpc, maxSingleBo,
initialBo, maxCumulativeBo, rpc);
sink = FlumeBuilder.buildSink(ctx, snk);
break;
}
case BEST_EFFORT: {
String snk = String.format("< " + batchGzDeco
+ " insistentOpen stubbornAppend rpcSink(\"%s\", %d) ? null>",
dsthost, port);
sink = FlumeBuilder.buildSink(ctx, snk);
break;
}
default:
throw new FlumeSpecException("unexpected agent mode: " + mode + "!");
}
}
@Override
public void append(Event e) throws IOException, InterruptedException {
sink.append(e);
super.append(e);
}
@Override
public void close() throws IOException, InterruptedException {
sink.close();
}
@Override
public void open() throws IOException, InterruptedException {
sink.open();
}
/**
* This builder creates a end to end acking, writeahead logging sink. This has
* the recovery mechanism of the WAL and a retry mechanism that resends events
* if user specified amount of time has passed without receiving an ack.
*
* The arguments for this are optional. the first is the name of the collector
* machine, and the second is a port
*
* e2eWal[("machine" [, port ] )]
*/
public static SinkBuilder e2eBuilder() {
return new SinkBuilder() {
@Override
public EventSink build(Context context, String... argv) {
Preconditions.checkArgument(argv.length <= 2,
"usage: agentE2ESink(collectorhost[, port]{, " + BATCH_COUNT
+ "=1}{, " + BATCH_MILLIS + "=0}{,compression=false})");
FlumeConfiguration conf = FlumeConfiguration.get();
String collector = conf.getCollectorHost();
int port = conf.getCollectorPort();
if (argv.length >= 1) {
collector = argv[0];
}
if (argv.length >= 2) {
port = Integer.parseInt(argv[1]);
}
try {
return new AgentSink(context, collector, port,
ReliabilityMode.ENDTOEND);
} catch (FlumeSpecException e) {
LOG.error("AgentSink spec error " + e, e);
throw new IllegalArgumentException(e);
}
}
};
}
/**
* This builder creates a disk failover logging sink. If a detectable error
* occurs, it falls back to writing to local disk and then resending data from
* disk.
*
* The arguments for this are optional. The first is the name of the collector
* machine, and the second is a port
*
* dfo[("machine" [, port ] )]
*
* TODO(jon) Need to reimplement/check this to make sure it still works in
* light of the changes to disk log management mechanisms.
*/
public static SinkBuilder dfoBuilder() {
return new SinkBuilder() {
@Override
public EventSink build(Context context, String... argv) {
Preconditions.checkArgument(argv.length <= 2,
"usage: agentDFOSink(collectorhost[, port]{, " + BATCH_COUNT
+ "=1}{, " + BATCH_MILLIS + "=0}{,compression=false})");
FlumeConfiguration conf = FlumeConfiguration.get();
String collector = conf.getCollectorHost();
int port = conf.getCollectorPort();
if (argv.length >= 1) {
collector = argv[0];
}
if (argv.length >= 2) {
port = Integer.parseInt(argv[1]);
}
try {
return new AgentSink(context, collector, port,
ReliabilityMode.DISK_FAILOVER);
} catch (FlumeSpecException e) {
LOG.error("AgentSink spec error " + e, e);
throw new IllegalArgumentException(e);
}
}
};
}
/**
* This builder creates a best effort logging sink. If a detectable error
* occurs, it moves on without trying to recover or retry.
*
* The arguments for this are optional. The first is the name of the collector
* machine, and the second is a port
*
* be[("machine" [, port ] )]
*/
public static SinkBuilder beBuilder() {
return new SinkBuilder() {
@Override
public EventSink build(Context context, String... argv) {
Preconditions.checkArgument(argv.length <= 2,
"usage: agentBESink(collectorhost[, port]{, " + BATCH_COUNT
+ "=1}{, " + BATCH_MILLIS + "=0}{,compression=false})");
FlumeConfiguration conf = FlumeConfiguration.get();
String collector = conf.getCollectorHost();
int port = conf.getCollectorPort();
if (argv.length >= 1) {
collector = argv[0];
}
if (argv.length >= 2) {
port = Integer.parseInt(argv[1]);
}
try {
return new AgentSink(context, collector, port,
ReliabilityMode.BEST_EFFORT);
} catch (FlumeSpecException e) {
LOG.error("AgentSink spec error " + e, e);
throw new IllegalArgumentException(e);
}
}
};
}
@Override
public String getName() {
return "Agent";
}
@Override
public ReportEvent getMetrics() {
ReportEvent rpt = new ReportEvent(getName());
return rpt;
}
@Override
public Map<String, Reportable> getSubMetrics() {
Map<String, Reportable> map = new HashMap<String, Reportable>();
map.put(sink.getName(), sink);
return map;
}
@Deprecated
@Override
public void getReports(String namePrefix, Map<String, ReportEvent> reports) {
super.getReports(namePrefix, reports);
sink.getReports(namePrefix + getName() + ".", reports);
}
}