Skip to content
Permalink
Browse files
FLUME-942: Support event compatibility with Flume 0.9x
  • Loading branch information
pmujumdar committed Feb 6, 2012
1 parent dfc60fe commit 6b877b272fafb45eed308785b1e56e4a70f6c84e
Show file tree
Hide file tree
Showing 14 changed files with 3,106 additions and 0 deletions.
@@ -0,0 +1,94 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">

<modelVersion>4.0.0</modelVersion>

<parent>
<artifactId>flume-ng-legacy-sources</artifactId>
<groupId>org.apache.flume</groupId>
<version>1.1.0-incubating-SNAPSHOT</version>
</parent>

<groupId>org.apache.flume.flume-ng-legacy-sources</groupId>
<artifactId>flume-avro-source</artifactId>
<name>Flume legacy Avro source</name>

<properties>
<thrift.executable>${env.THRIFT_HOME}/bin/thrift</thrift.executable>
</properties>

<build>
<plugins>
<plugin>
<groupId>org.apache.rat</groupId>
<artifactId>apache-rat-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>idl-protocol</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>

<dependencies>

<dependency>
<groupId>org.apache.flume</groupId>
<artifactId>flume-ng-core</artifactId>
</dependency>

<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</dependency>

<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</dependency>

<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro-ipc</artifactId>
</dependency>

</dependencies>

</project>
@@ -0,0 +1,36 @@
/**
* 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.
*/

/* Avro protocol for Flume 0.9.x compatibility */
@namespace("com.cloudera.flume.handlers.avro")
protocol FlumeOGEventAvroServer{
enum Priority { FATAL, ERROR, WARN,
INFO, DEBUG, TRACE}

record AvroFlumeOGEvent {
long timestamp;
Priority priority;
bytes body;
long nanos;
string host;
map<bytes> fields;
}

void append( AvroFlumeOGEvent evt ) ;
}
@@ -0,0 +1,137 @@
package org.apache.flume.source.avroLegacy;

import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;

import org.apache.avro.ipc.HttpServer;
import org.apache.avro.ipc.specific.SpecificResponder;
import org.apache.flume.Context;
import org.apache.flume.CounterGroup;
import org.apache.flume.Event;
import org.apache.flume.EventDrivenSource;
import org.apache.flume.Source;
import org.apache.flume.conf.Configurable;
import org.apache.flume.event.EventBuilder;
import org.apache.flume.source.AbstractSource;

import com.cloudera.flume.handlers.avro.AvroFlumeOGEvent;
import com.cloudera.flume.handlers.avro.FlumeOGEventAvroServer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.flume.ChannelException;

/**
* <p>
* A {@link Source} implementation that receives Avro events from Avro sink of Flume OG
* </p>
* <p>
* <b>Configuration options</b>
* </p>
* <table>
* <tr>
* <th>Parameter</th>
* <th>Description</th>
* <th>Unit / Type</th>
* <th>Default</th>
* </tr>
* <tr>
* <td><tt>host</tt></td>
* <td>The hostname or IP to which the source will bind.</td>
* <td>Hostname or IP / String</td>
* <td>none (required)</td>
* </tr>
* <tr>
* <td><tt>port</tt></td>
* <td>The port to which the source will bind and listen for events.</td>
* <td>TCP port / int</td>
* <td>none (required)</td>
* </tr>
* </table>
* <p>
* <b>Metrics</b>
* </p>
* <p>
* TODO
* </p>
*/

public class AvroLegacySource extends AbstractSource implements EventDrivenSource,
Configurable, FlumeOGEventAvroServer {

static final Logger LOG = LoggerFactory.getLogger(AvroLegacySource.class);

// Flume OG event fields
public static final String HOST = "host";
public static final String TIMESTAMP = "timestamp";
public static final String PRIORITY = "pri";
public static final String NANOS = "nanos";
public static final String OG_EVENT = "FlumeOG";

private CounterGroup counterGroup;
protected FlumeOGEventAvroServer avroClient;
private String host;
private int port;
private SpecificResponder res;
private HttpServer http;

public AvroLegacySource() {
counterGroup = new CounterGroup();
}

@Override
public void start() {
// setup http server to receive OG events
res = new SpecificResponder(FlumeOGEventAvroServer.class, this);
try {
http = new HttpServer(res, host, port);
} catch (IOException eI) {
LOG.warn("Failed to start server", eI);
return;
}
http.start();
super.start();
}

@Override
public void stop() {
http.close();
super.stop();
}

@Override
public Void append( AvroFlumeOGEvent evt ) throws org.apache.avro.AvroRemoteException {
counterGroup.incrementAndGet("rpc.received");
Map<String, String> headers = new HashMap<String, String>();

// extract Flume OG event headers
headers.put(HOST, evt.getHost().toString());
headers.put(TIMESTAMP, evt.getTimestamp().toString());
headers.put(PRIORITY, evt.getPriority().toString());
headers.put(NANOS, evt.getNanos().toString());
for (Entry<CharSequence, ByteBuffer> entry: evt.getFields().entrySet()) {
headers.put(entry.getKey().toString(), entry.getValue().toString());
}
headers.put(OG_EVENT, "yes");

Event event = EventBuilder.withBody(evt.getBody().array(), headers);
try {
getChannelProcessor().processEvent(event);
counterGroup.incrementAndGet("rpc.events");
} catch (ChannelException ex) {
return null;
}

counterGroup.incrementAndGet("rpc.successful");
return null;
}

@Override
public void configure(Context context) {
port = Integer.parseInt(context.get("port", String.class));
host = context.get("host", String.class);
}

}

0 comments on commit 6b877b2

Please sign in to comment.