Skip to content

Commit

Permalink
[BEAM-1237] Add AmqpMessageCoderProviderRegistrar and AmqpMessageCode…
Browse files Browse the repository at this point in the history
…r deals the whole message (not only the body)

TODO: fix the Coder
  • Loading branch information
jbonofre committed Jun 14, 2017
1 parent 83b0061 commit e58dda8
Show file tree
Hide file tree
Showing 3 changed files with 60 additions and 4 deletions.
10 changes: 10 additions & 0 deletions sdks/java/io/amqp/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -61,13 +61,23 @@
<artifactId>proton-j</artifactId>
<version>0.13.1</version>
</dependency>
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
<version>2.5</version>
</dependency>

<!-- compile dependencies -->
<dependency>
<groupId>com.google.auto.value</groupId>
<artifactId>auto-value</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.google.auto.service</groupId>
<artifactId>auto-service</artifactId>
<optional>true</optional>
</dependency>

<!-- test dependencies -->
<dependency>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import org.apache.beam.sdk.coders.CoderException;
import org.apache.beam.sdk.coders.CustomCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.qpid.proton.amqp.messaging.AmqpValue;
import org.apache.commons.io.IOUtils;
import org.apache.qpid.proton.message.Message;

/**
Expand All @@ -40,14 +40,16 @@ static AmqpMessageCoder of() {

@Override
public void encode(Message value, OutputStream outStream) throws CoderException, IOException {
String body = new AmqpValue(value.getBody()).toString();
stringCoder.encode(body, outStream);
byte[] data = new byte[16384];
value.encode(data, 0, data.length);
outStream.write(data);
}

@Override
public Message decode(InputStream inStream) throws CoderException, IOException {
Message message = Message.Factory.create();
message.setBody(new AmqpValue(stringCoder.decode(inStream)));
byte[] data = IOUtils.toByteArray(inStream);
message.decode(data, 0, data.length);
return message;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -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.beam.sdk.io.amqp;

import com.google.auto.service.AutoService;
import com.google.common.collect.ImmutableList;

import java.util.List;

import org.apache.beam.sdk.coders.CoderProvider;
import org.apache.beam.sdk.coders.CoderProviderRegistrar;
import org.apache.beam.sdk.coders.CoderProviders;
import org.apache.beam.sdk.values.TypeDescriptor;
import org.apache.qpid.proton.message.Message;

/**
* A {@link CoderProviderRegistrar} for standard types used with {@link AmqpIO}.
*/
@AutoService(CoderProviderRegistrar.class)
public class AmqpMessageCoderProviderRegistrar implements CoderProviderRegistrar {

@Override
public List<CoderProvider> getCoderProviders() {
return ImmutableList.of(
CoderProviders.forCoder(TypeDescriptor.of(Message.class),
AmqpMessageCoder.of()));
}

}

0 comments on commit e58dda8

Please sign in to comment.