Return-Path: X-Original-To: apmail-incubator-flume-commits-archive@minotaur.apache.org Delivered-To: apmail-incubator-flume-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 0CA7B916D for ; Tue, 13 Mar 2012 23:38:34 +0000 (UTC) Received: (qmail 74513 invoked by uid 500); 13 Mar 2012 23:38:34 -0000 Delivered-To: apmail-incubator-flume-commits-archive@incubator.apache.org Received: (qmail 74461 invoked by uid 500); 13 Mar 2012 23:38:33 -0000 Mailing-List: contact flume-commits-help@incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: flume-dev@incubator.apache.org Delivered-To: mailing list flume-commits@incubator.apache.org Received: (qmail 74452 invoked by uid 99); 13 Mar 2012 23:38:33 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 13 Mar 2012 23:38:33 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 13 Mar 2012 23:38:29 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 94E23238896F; Tue, 13 Mar 2012 23:38:08 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1300411 - in /incubator/flume/trunk: ./ flume-ng-core/src/main/java/org/apache/flume/source/ flume-ng-core/src/test/java/org/apache/flume/source/ flume-ng-legacy-sources/flume-avro-source/src/main/java/org/apache/flume/source/avroLegacy/ f... Date: Tue, 13 Mar 2012 23:38:08 -0000 To: flume-commits@incubator.apache.org From: arvind@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20120313233808.94E23238896F@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: arvind Date: Tue Mar 13 23:38:07 2012 New Revision: 1300411 URL: http://svn.apache.org/viewvc?rev=1300411&view=rev Log: FLUME-1028. Fix jenkins build after addition of submodule. (Mike Percy via Arvind Prabhakar) Modified: incubator/flume/trunk/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java incubator/flume/trunk/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java incubator/flume/trunk/flume-ng-legacy-sources/flume-avro-source/src/main/java/org/apache/flume/source/avroLegacy/AvroLegacySource.java incubator/flume/trunk/flume-ng-legacy-sources/flume-avro-source/src/test/java/org/apache/flume/source/avroLegacy/TestLegacyAvroSource.java incubator/flume/trunk/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java incubator/flume/trunk/pom.xml Modified: incubator/flume/trunk/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java URL: http://svn.apache.org/viewvc/incubator/flume/trunk/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java?rev=1300411&r1=1300410&r2=1300411&view=diff ============================================================================== --- incubator/flume/trunk/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java (original) +++ incubator/flume/trunk/flume-ng-core/src/main/java/org/apache/flume/source/AvroSource.java Tue Mar 13 23:38:07 2012 @@ -24,7 +24,6 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import org.apache.avro.ipc.NettyServer; import org.apache.avro.ipc.Responder; @@ -154,6 +153,19 @@ public class AvroSource extends Abstract return "AvroSource: { bindAddress:" + bindAddress + " port:" + port + " }"; } + /** + * Helper function to convert a map of CharSequence to a map of String. + */ + private static Map toStringMap( + Map charSeqMap) { + Map stringMap = + new HashMap(); + for (Map.Entry entry : charSeqMap.entrySet()) { + stringMap.put(entry.getKey().toString(), entry.getValue().toString()); + } + return stringMap; + } + @Override public Status append(AvroFlumeEvent avroEvent) { logger.debug("Received avro event:{}", avroEvent); @@ -161,7 +173,7 @@ public class AvroSource extends Abstract counterGroup.incrementAndGet("rpc.received"); Event event = EventBuilder.withBody(avroEvent.getBody().array(), - avroEvent.getHeaders()); + toStringMap(avroEvent.getHeaders())); try { getChannelProcessor().processEvent(event); @@ -182,7 +194,7 @@ public class AvroSource extends Abstract for (AvroFlumeEvent avroEvent : events) { Event event = EventBuilder.withBody(avroEvent.getBody().array(), - avroEvent.getHeaders()); + toStringMap(avroEvent.getHeaders())); counterGroup.incrementAndGet("rpc.events"); batch.add(event); Modified: incubator/flume/trunk/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java URL: http://svn.apache.org/viewvc/incubator/flume/trunk/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java?rev=1300411&r1=1300410&r2=1300411&view=diff ============================================================================== --- incubator/flume/trunk/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java (original) +++ incubator/flume/trunk/flume-ng-core/src/test/java/org/apache/flume/source/TestAvroSource.java Tue Mar 13 23:38:07 2012 @@ -145,7 +145,7 @@ public class TestAvroSource { AvroFlumeEvent avroEvent = new AvroFlumeEvent(); - avroEvent.setHeaders(new HashMap()); + avroEvent.setHeaders(new HashMap()); avroEvent.setBody(ByteBuffer.wrap("Hello avro".getBytes())); Status status = client.append(avroEvent); Modified: incubator/flume/trunk/flume-ng-legacy-sources/flume-avro-source/src/main/java/org/apache/flume/source/avroLegacy/AvroLegacySource.java URL: http://svn.apache.org/viewvc/incubator/flume/trunk/flume-ng-legacy-sources/flume-avro-source/src/main/java/org/apache/flume/source/avroLegacy/AvroLegacySource.java?rev=1300411&r1=1300410&r2=1300411&view=diff ============================================================================== --- incubator/flume/trunk/flume-ng-legacy-sources/flume-avro-source/src/main/java/org/apache/flume/source/avroLegacy/AvroLegacySource.java (original) +++ incubator/flume/trunk/flume-ng-legacy-sources/flume-avro-source/src/main/java/org/apache/flume/source/avroLegacy/AvroLegacySource.java Tue Mar 13 23:38:07 2012 @@ -131,7 +131,7 @@ public class AvroLegacySource extends Ab headers.put(TIMESTAMP, evt.getTimestamp().toString()); headers.put(PRIORITY, evt.getPriority().toString()); headers.put(NANOS, evt.getNanos().toString()); - for (Entry entry: evt.getFields().entrySet()) { + for (Entry entry : evt.getFields().entrySet()) { headers.put(entry.getKey().toString(), entry.getValue().toString()); } headers.put(OG_EVENT, "yes"); Modified: incubator/flume/trunk/flume-ng-legacy-sources/flume-avro-source/src/test/java/org/apache/flume/source/avroLegacy/TestLegacyAvroSource.java URL: http://svn.apache.org/viewvc/incubator/flume/trunk/flume-ng-legacy-sources/flume-avro-source/src/test/java/org/apache/flume/source/avroLegacy/TestLegacyAvroSource.java?rev=1300411&r1=1300410&r2=1300411&view=diff ============================================================================== --- incubator/flume/trunk/flume-ng-legacy-sources/flume-avro-source/src/test/java/org/apache/flume/source/avroLegacy/TestLegacyAvroSource.java (original) +++ incubator/flume/trunk/flume-ng-legacy-sources/flume-avro-source/src/test/java/org/apache/flume/source/avroLegacy/TestLegacyAvroSource.java Tue Mar 13 23:38:07 2012 @@ -145,7 +145,7 @@ public class TestLegacyAvroSource { AvroFlumeOGEvent avroEvent = AvroFlumeOGEvent.newBuilder().setHost("foo"). setPriority(Priority.INFO).setNanos(0).setTimestamp(1). - setFields(new HashMap ()). + setFields(new HashMap ()). setBody(ByteBuffer.wrap("foo".getBytes())).build(); client.append(avroEvent); Modified: incubator/flume/trunk/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java URL: http://svn.apache.org/viewvc/incubator/flume/trunk/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java?rev=1300411&r1=1300410&r2=1300411&view=diff ============================================================================== --- incubator/flume/trunk/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java (original) +++ incubator/flume/trunk/flume-ng-sdk/src/main/java/org/apache/flume/api/NettyAvroRpcClient.java Tue Mar 13 23:38:07 2012 @@ -157,7 +157,7 @@ public class NettyAvroRpcClient implemen try { AvroFlumeEvent avroEvent = new AvroFlumeEvent(); avroEvent.setBody(ByteBuffer.wrap(event.getBody())); - avroEvent.setHeaders(event.getHeaders()); + avroEvent.setHeaders(toCharSeqMap(event.getHeaders())); avroClient.append(avroEvent, callFuture); } catch (IOException ex) { throw new EventDeliveryException("RPC request IO exception. " + @@ -196,7 +196,7 @@ public class NettyAvroRpcClient implemen Event event = iter.next(); AvroFlumeEvent avroEvent = new AvroFlumeEvent(); avroEvent.setBody(ByteBuffer.wrap(event.getBody())); - avroEvent.setHeaders(event.getHeaders()); + avroEvent.setHeaders(toCharSeqMap(event.getHeaders())); avroEvents.add(avroEvent); } @@ -279,6 +279,19 @@ public class NettyAvroRpcClient implemen } } + /** + * Helper function to convert a map of String to a map of CharSequence. + */ + private static Map toCharSeqMap( + Map stringMap) { + Map charSeqMap = + new HashMap(); + for (Map.Entry entry : stringMap.entrySet()) { + charSeqMap.put(entry.getKey(), entry.getValue()); + } + return charSeqMap; + } + @Override public boolean isActive() { stateLock.lock(); Modified: incubator/flume/trunk/pom.xml URL: http://svn.apache.org/viewvc/incubator/flume/trunk/pom.xml?rev=1300411&r1=1300410&r2=1300411&view=diff ============================================================================== --- incubator/flume/trunk/pom.xml (original) +++ incubator/flume/trunk/pom.xml Tue Mar 13 23:38:07 2012 @@ -437,10 +437,12 @@ limitations under the License. org.apache.avro avro-maven-plugin + + 1.6.1 @@ -535,19 +537,19 @@ limitations under the License. org.apache.avro avro - 1.6.2 + 1.6.1 org.apache.avro avro-compiler - 1.6.2 + 1.6.1 org.apache.avro avro-ipc - 1.6.2 + 1.6.1