You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flume.apache.org by ar...@apache.org on 2012/02/21 05:00:45 UTC

svn commit: r1291585 - in /incubator/flume/branches/flume-728: ./ flume-ng-clients/ flume-ng-clients/flume-ng-log4jappender/ flume-ng-clients/flume-ng-log4jappender/src/ flume-ng-clients/flume-ng-log4jappender/src/main/ flume-ng-clients/flume-ng-log4ja...

Author: arvind
Date: Tue Feb 21 04:00:44 2012
New Revision: 1291585

URL: http://svn.apache.org/viewvc?rev=1291585&view=rev
Log:
FLUME-886. Create Log4j Appender.

(Hari Shreedharan via Arvind Prabhakar)

Added:
    incubator/flume/branches/flume-728/flume-ng-clients/   (with props)
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/   (with props)
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/pom.xml
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java   (with props)
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java   (with props)
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/java/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppender.java   (with props)
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/resources/
    incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/resources/flume-log4jtest.properties
    incubator/flume/branches/flume-728/flume-ng-clients/pom.xml
Modified:
    incubator/flume/branches/flume-728/pom.xml

Propchange: incubator/flume/branches/flume-728/flume-ng-clients/
------------------------------------------------------------------------------
--- svn:ignore (added)
+++ svn:ignore Tue Feb 21 04:00:44 2012
@@ -0,0 +1 @@
+target

Propchange: incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/
------------------------------------------------------------------------------
--- svn:ignore (added)
+++ svn:ignore Tue Feb 21 04:00:44 2012
@@ -0,0 +1,4 @@
+.classpath
+.project
+.settings
+target

Added: incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/pom.xml
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/pom.xml?rev=1291585&view=auto
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/pom.xml (added)
+++ incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/pom.xml Tue Feb 21 04:00:44 2012
@@ -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.
+-->
+<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-clients</artifactId>
+    <groupId>org.apache.flume</groupId>
+    <version>1.1.0-incubating-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+
+  <groupId>org.apache.flume</groupId>
+  <artifactId>flume-ng-log4jappender</artifactId>
+  <name>Flume NG Log4j Appender</name>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro-ipc</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-core</artifactId>
+    </dependency>
+  </dependencies>
+
+
+</project>

Added: incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java?rev=1291585&view=auto
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java (added)
+++ incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java Tue Feb 21 04:00:44 2012
@@ -0,0 +1,204 @@
+/*
+ * 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.flume.clients.log4jappender;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.avro.AvroRemoteException;
+import org.apache.avro.ipc.NettyTransceiver;
+import org.apache.avro.ipc.Transceiver;
+import org.apache.avro.ipc.specific.SpecificRequestor;
+import org.apache.flume.FlumeException;
+import org.apache.flume.source.avro.AvroFlumeEvent;
+import org.apache.flume.source.avro.AvroSourceProtocol;
+import org.apache.log4j.AppenderSkeleton;
+import org.apache.log4j.helpers.LogLog;
+import org.apache.log4j.spi.LoggingEvent;
+import org.apache.flume.clients.log4jappender.Log4jAvroHeaders;
+/**
+ *
+ * Appends Log4j Events to an external Flume client which is decribed by
+ * the Log4j configuration file. The appender takes two parameters
+ *<p>
+ *<strong>Hostname</strong> : This is the hostname of the first hop
+ *at which Flume (through an AvroSource) is listening for events.
+ *</p>
+ *<p>
+ *<strong>Port</strong> : This the port on the above host where the Flume
+ *Source is listening for events.
+ *</p>
+ *A sample log4j properties file which appends to a source would look like:
+ *<pre><p>
+ *log4j.appender.out2 = org.apache.flume.clients.log4jappender.Log4jAppender
+ *log4j.appender.out2.Port = 25430
+ *log4j.appender.out2.Hostname = foobarflumesource.com
+ *log4j.logger.org.apache.flume.clients.log4jappender = DEBUG,out2</p></pre>
+ *<p><i>Note: Change the last line to the package of the class(es), that will
+ *do the appending.For example if classes from the package
+ *com.bar.foo are appending, the last line would be:</i></p>
+ *<pre><p>log4j.logger.com.bar.foo = DEBUG,out2</p></pre>
+ *
+ *
+ */
+public class Log4jAppender extends AppenderSkeleton {
+
+  private String hostname;
+  private int port;
+  private AvroSourceProtocol protocolClient = null;
+  private Transceiver transceiver = null;
+
+
+  /**
+   * If this constructor is used programatically, rather than from a log4j conf,
+   * you must set the <tt>port</tt> and <tt>hostname</tt> and then call
+   * <tt>activateOptions()</tt> before calling <tt>append()</tt>.
+   */
+  public Log4jAppender(){
+  }
+  /**
+   * Sets the hostname and port. Even if these are passed the
+   * <tt>activateOptions()</tt> function must be called before calling
+   * <tt>append()</tt>, else <tt>append()</tt> will throw an Exception.
+   * @param hostname The first hop where the client should connect to.
+   * @param port The port to connect on the host.
+   *
+   */
+  public Log4jAppender(String hostname, int port){
+    this.hostname = hostname;
+    this.port = port;
+  }
+
+  /**
+   * Append the LoggingEvent, to send to the first Flume hop.
+   * @param event The LoggingEvent to be appended to the flume.
+   * @throws FlumeException if the appender was closed
+   * or the hostname and port were not setup.
+   */
+  @Override
+  public synchronized void append(LoggingEvent event) throws FlumeException{
+    //If protocolClient is null, it means either this appender object was never
+    //setup by setting hostname and port and then calling activateOptions
+    //or this appender object was closed by calling close(), so we throw an
+    //exception to show the appender is no longer accessible.
+    if(protocolClient == null){
+      throw new FlumeException("Cannot Append to Appender!" +
+          "Appender either closed or not setup correctly!");
+    }
+
+    //Client created first time append is called.
+    AvroFlumeEvent avroEvent = new AvroFlumeEvent();
+    Map<CharSequence, CharSequence> hdrs =
+        new HashMap<CharSequence, CharSequence>();
+    hdrs.put(Log4jAvroHeaders.LOGGER_NAME.toString(), event.getLoggerName());
+    hdrs.put(Log4jAvroHeaders.TIMESTAMP.toString(),
+        String.valueOf(event.getTimeStamp()));
+
+    //To get the level back simply use
+    //LoggerEvent.toLevel(hdrs.get(Integer.parseInt(
+    //Log4jAvroHeaders.LOG_LEVEL.toString()))
+    hdrs.put(Log4jAvroHeaders.LOG_LEVEL.toString(),
+        String.valueOf(event.getLevel().toInt()));
+    hdrs.put(Log4jAvroHeaders.MESSAGE_ENCODING.toString(), "UTF8");
+    avroEvent.setHeaders(hdrs);
+    try {
+      avroEvent.setBody(ByteBuffer.wrap(
+          event.getMessage().toString().getBytes("UTF8")));
+      protocolClient.append(avroEvent);
+    } catch (UnsupportedEncodingException e) {
+      String errormsg = "Unable to encode body of event! Event lost! " +
+          e.getMessage();
+      LogLog.error(errormsg);
+      throw new FlumeException(errormsg,e);
+    } catch (AvroRemoteException e) {
+      String errormsg = "Avro Remote Exception: " + e.getMessage();
+      LogLog.error(errormsg);
+      throw new FlumeException(errormsg,e);
+    }
+  }
+
+  //This function should be synchronized to make sure one thread
+  //does not close an appender another thread is using, and hence risking
+  //a null pointer exception.
+  /**
+   * Closes underlying client.
+   * If <tt>append()</tt> is called after this function is called,
+   * it will throw an exception.
+   * @throws FlumeException if appender was closed once or not setup.
+   */
+  @Override
+  public synchronized void close() throws FlumeException{
+    //Simply set protocol client to null and let Java do the cleanup
+    //when the client is no longer accessible.
+    //Any append calls after this will result in an Exception.
+    try {
+      transceiver.close();
+    } catch (IOException e) {
+      throw new FlumeException("Attempting to close " +
+          "Appender which is already closed or one which was never opened", e);
+    }
+    protocolClient = null;
+  }
+
+  @Override
+  public boolean requiresLayout() {
+    return false;
+  }
+  /**
+   * Set the first flume hop hostname.
+   * @param hostname The first hop where the client should connect to.
+   */
+  public void setHostname(String hostname){
+    this.hostname = hostname;
+  }
+  /**
+   * Set the port on the hostname to connect to.
+   * @param port The port to connect on the host.
+   */
+  public void setPort(int port){
+    this.port = port;
+  }
+
+  /**
+   * Activate the options set using <tt>setPort()</tt>
+   * and <tt>setHostname()</tt>
+   * @throws FlumeException if the <tt>hostname</tt> and
+   *  <tt>port</tt> combination is invalid.
+   */
+  @Override
+  public void activateOptions() throws FlumeException{
+    try {
+      transceiver = new NettyTransceiver(
+          new InetSocketAddress(hostname, port));
+      protocolClient = SpecificRequestor.getClient(
+          AvroSourceProtocol.class, transceiver);
+    } catch (IOException e) {
+      String errormsg = "Avro Client creation failed! "+
+          e.getMessage();
+      LogLog.error(errormsg);
+      throw new FlumeException(errormsg,e);
+
+    }
+  }
+}
+

Propchange: incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAppender.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java?rev=1291585&view=auto
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java (added)
+++ incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java Tue Feb 21 04:00:44 2012
@@ -0,0 +1,52 @@
+/*
+ * 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.flume.clients.log4jappender;
+
+public enum Log4jAvroHeaders {
+  OTHER("flume.client.log4j.logger.other"),
+  LOGGER_NAME("flume.client.log4j.logger.name"),
+  LOG_LEVEL("flume.client.log4j.log.level"),
+  MESSAGE_ENCODING("flume.client.log4j.message.encoding"),
+  TIMESTAMP("flume.client.log4j.timestamp");
+
+  private String headerName;
+  private Log4jAvroHeaders(String headerName){
+    this.headerName = headerName;
+  }
+
+  public String getName(){
+    return headerName;
+  }
+
+  public String toString(){
+    return getName();
+  }
+
+  public static Log4jAvroHeaders getByName(String headerName){
+    Log4jAvroHeaders hdrs = null;
+    try{
+      hdrs = Log4jAvroHeaders.valueOf(headerName.toLowerCase().trim());
+    }
+    catch(IllegalArgumentException e){
+      hdrs = Log4jAvroHeaders.OTHER;
+    }
+    return hdrs;
+  }
+
+}
\ No newline at end of file

Propchange: incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/main/java/org/apache/flume/clients/log4jappender/Log4jAvroHeaders.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppender.java
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppender.java?rev=1291585&view=auto
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppender.java (added)
+++ incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppender.java Tue Feb 21 04:00:44 2012
@@ -0,0 +1,126 @@
+/*
+ * 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.flume.clients.log4jappender;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import junit.framework.Assert;
+
+import org.apache.flume.Channel;
+import org.apache.flume.ChannelSelector;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.Transaction;
+import org.apache.flume.channel.ChannelProcessor;
+import org.apache.flume.channel.MemoryChannel;
+import org.apache.flume.channel.ReplicatingChannelSelector;
+import org.apache.flume.conf.Configurables;
+import org.apache.flume.source.AvroSource;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.log4j.PropertyConfigurator;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestLog4jAppender{
+  private AvroSource source;
+  private Channel ch;
+
+  @Before
+  public void initiate() throws InterruptedException{
+    int port = 25430;
+    source = new AvroSource();
+    ch = new MemoryChannel();
+    Configurables.configure(ch, new Context());
+
+    Context context = new Context();
+    context.put("port", String.valueOf(port));
+    context.put("bind", "localhost");
+    Configurables.configure(source, context);
+
+    List<Channel> channels = new ArrayList<Channel>();
+    channels.add(ch);
+
+    ChannelSelector rcs = new ReplicatingChannelSelector();
+    rcs.setChannels(channels);
+
+    source.setChannelProcessor(new ChannelProcessor(rcs));
+
+    source.start();
+
+  }
+  @Test
+  public void testLog4jAppender() throws IOException {
+    //The properties file having Avro port info should be loaded only
+    //after the test begins, else log4j tries to connect to the source
+    //before the source has started up in the above function, since
+    //log4j setup is completed before the @Before calls also.
+    //This will cause the test to fail even before it starts!
+    File TESTFILE = new File(
+        TestLog4jAppender.class.getClassLoader()
+        .getResource("flume-log4jtest.properties").getFile());
+    FileReader reader = new FileReader(TESTFILE);
+    Properties props = new Properties();
+    props.load(reader);
+    PropertyConfigurator.configure(props);
+    Logger logger = LogManager.getLogger(TestLog4jAppender.class);
+    for(int count = 0; count <= 1000; count++){
+      int level = count % 5;
+      String msg = "This is log message number" + String.valueOf(count);
+
+      logger.log(Level.toLevel(level), msg);
+      Transaction transaction = ch.getTransaction();
+      transaction.begin();
+      Event event = ch.take();
+      Assert.assertNotNull(event);
+      Assert.assertEquals(new String(event.getBody(), "UTF8"), msg);
+
+      Map<String, String> hdrs = event.getHeaders();
+
+      Assert.assertNotNull(hdrs.get(Log4jAvroHeaders.TIMESTAMP.toString()));
+
+      Assert.assertEquals(Level.toLevel(level),
+          Level.toLevel(hdrs.get(Log4jAvroHeaders.LOG_LEVEL.toString())));
+
+      Assert.assertEquals(logger.getName(),
+          hdrs.get(Log4jAvroHeaders.LOGGER_NAME.toString()));
+
+      Assert.assertEquals("UTF8",
+          hdrs.get(Log4jAvroHeaders.MESSAGE_ENCODING.toString()));
+      //To confirm on console we actually got the body
+      System.out.println("Got body: "+new String(event.getBody(), "UTF8"));
+      transaction.commit();
+      transaction.close();
+    }
+
+  }
+
+  @After
+  public void cleanUp(){
+  }
+
+}

Propchange: incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/java/org/apache/flume/clients/log4jappender/TestLog4jAppender.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/resources/flume-log4jtest.properties
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/resources/flume-log4jtest.properties?rev=1291585&view=auto
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/resources/flume-log4jtest.properties (added)
+++ incubator/flume/branches/flume-728/flume-ng-clients/flume-ng-log4jappender/src/test/resources/flume-log4jtest.properties Tue Feb 21 04:00:44 2012
@@ -0,0 +1,20 @@
+# 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.appender.out2 = org.apache.flume.clients.log4jappender.Log4jAppender
+log4j.appender.out2.Port = 25430
+log4j.appender.out2.Hostname = localhost
+log4j.logger.org.apache.flume.clients.log4jappender = DEBUG,out2
\ No newline at end of file

Added: incubator/flume/branches/flume-728/flume-ng-clients/pom.xml
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/flume-ng-clients/pom.xml?rev=1291585&view=auto
==============================================================================
--- incubator/flume/branches/flume-728/flume-ng-clients/pom.xml (added)
+++ incubator/flume/branches/flume-728/flume-ng-clients/pom.xml Tue Feb 21 04:00:44 2012
@@ -0,0 +1,33 @@
+<!--
+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-parent</artifactId>
+    <groupId>org.apache.flume</groupId>
+    <version>1.1.0-incubating-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+  <artifactId>flume-ng-clients</artifactId>
+  <packaging>pom</packaging>
+  <name>Flume NG Clients</name>
+  <description>All flume NG clients will come under this module</description>
+  <modules>
+    <module>flume-ng-log4jappender</module>
+  </modules>
+</project>
\ No newline at end of file

Modified: incubator/flume/branches/flume-728/pom.xml
URL: http://svn.apache.org/viewvc/incubator/flume/branches/flume-728/pom.xml?rev=1291585&r1=1291584&r2=1291585&view=diff
==============================================================================
--- incubator/flume/branches/flume-728/pom.xml (original)
+++ incubator/flume/branches/flume-728/pom.xml Tue Feb 21 04:00:44 2012
@@ -48,6 +48,7 @@ limitations under the License.
     <module>flume-ng-dist</module>
     <module>flume-ng-channels</module>
     <module>flume-ng-legacy-sources</module>
+    <module>flume-ng-clients</module>
   </modules>
 
   <profiles>
@@ -126,7 +127,7 @@ limitations under the License.
     <profile>
       <id>compileThrift</id>
       <activation>
-       <activeByDefault>false</activeByDefault>
+        <activeByDefault>false</activeByDefault>
       </activation>
       <properties>
         <thrift.executable>${env.THRIFT_HOME}/bin/thrift</thrift.executable>