From 89b0c84f3b1124f62949e4987cd38b6d034dbc91 Mon Sep 17 00:00:00 2001 From: Jan Rudert Date: Fri, 12 Jul 2013 13:34:57 +0200 Subject: [PATCH 01/13] initial implementation of an protocol buffers firehose --- indexing-common/pom.xml | 6 + .../metamx/druid/indexer/data/DataSpec.java | 3 +- .../druid/indexer/data/InputRowParser.java | 7 + .../druid/indexer/data/ProtoBufDataSpec.java | 69 + .../indexer/data/ProtoBufInputRowParser.java | 108 ++ .../data/ProtoBufInputRowParserTest.java | 85 + .../indexer/data/ProtoTestEventWrapper.java | 1375 +++++++++++++++++ .../src/test/resources/ProtoTest.proto | 31 + .../src/test/resources/prototest.desc | Bin 0 -> 400 bytes .../firehose/KafkaFirehoseFactory.java | 301 ++-- 10 files changed, 1874 insertions(+), 111 deletions(-) create mode 100644 indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufDataSpec.java create mode 100644 indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java create mode 100644 indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java create mode 100644 indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoTestEventWrapper.java create mode 100644 indexing-common/src/test/resources/ProtoTest.proto create mode 100644 indexing-common/src/test/resources/prototest.desc diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index fd727328c49..d09ea33bd64 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -79,6 +79,12 @@ commons-io commons-io + + com.google.protobuf + protobuf-java + 2.5.0 + + diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java index 099d0c8d535..36d942d8d62 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java @@ -32,7 +32,8 @@ import java.util.List; @JsonSubTypes(value = { @JsonSubTypes.Type(name = "json", value = JSONDataSpec.class), @JsonSubTypes.Type(name = "csv", value = CSVDataSpec.class), - @JsonSubTypes.Type(name = "tsv", value = DelimitedDataSpec.class) + @JsonSubTypes.Type(name = "tsv", value = DelimitedDataSpec.class), + @JsonSubTypes.Type(name = "protobuf", value = ProtoBufDataSpec.class) }) public interface DataSpec { diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java index 82a3c947bc3..25957379d9a 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java @@ -1,7 +1,14 @@ package com.metamx.druid.indexer.data; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.input.InputRow; +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringInputRowParser.class) +@JsonSubTypes({ + @JsonSubTypes.Type(name = "string", value = StringInputRowParser.class), + @JsonSubTypes.Type(name = "protobuf", value = ProtoBufInputRowParser.class), +}) public interface InputRowParser { public InputRow parse(T input); diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufDataSpec.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufDataSpec.java new file mode 100644 index 00000000000..042b0603061 --- /dev/null +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufDataSpec.java @@ -0,0 +1,69 @@ +package com.metamx.druid.indexer.data; + +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Lists; +import com.metamx.common.parsers.Parser; +import com.metamx.druid.index.v1.SpatialDimensionSchema; + +/** + * @author jan.rudert + */ +public class ProtoBufDataSpec implements DataSpec{ + private final List dimensions; + private final List spatialDimensions; + private final String descriptorFileInClasspath; + + @JsonCreator + public ProtoBufDataSpec( + @JsonProperty("descriptor") String descriptorFileInClasspath, + @JsonProperty("dimensions") List dimensions, + @JsonProperty("spatialDimensions") List spatialDimensions + ) + { + this.descriptorFileInClasspath = descriptorFileInClasspath; + this.dimensions = dimensions; + this.spatialDimensions = (spatialDimensions == null) + ? Lists.newArrayList() + : spatialDimensions; + + } + + @JsonProperty("descriptor") + public String getDescriptorFileInClassPath() { + return descriptorFileInClasspath; + } + + @JsonProperty("dimensions") + @Override + public List getDimensions() + { + return dimensions; + } + + @JsonProperty("spatialDimensions") + @Override + public List getSpatialDimensions() + { + return spatialDimensions; + } + + @Override + public void verify(List usedCols) + { + } + + @Override + public boolean hasCustomDimensions() + { + return !(dimensions == null || dimensions.isEmpty()); + } + + @Override + public Parser getParser() + { + throw new UnsupportedOperationException("No String parser for protobuf data"); + } +} diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java new file mode 100644 index 00000000000..38ee99663e4 --- /dev/null +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java @@ -0,0 +1,108 @@ +package com.metamx.druid.indexer.data; + +import static com.google.protobuf.DescriptorProtos.FileDescriptorSet; +import static com.google.protobuf.Descriptors.Descriptor; +import static com.google.protobuf.Descriptors.FileDescriptor; + +import java.io.InputStream; +import java.util.List; +import java.util.Map; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Throwables; +import com.google.common.collect.Maps; +import com.google.protobuf.Descriptors; +import com.google.protobuf.DynamicMessage; +import com.google.protobuf.InvalidProtocolBufferException; +import com.metamx.druid.input.InputRow; + +/** + * @author jan.rudert + */ +public class ProtoBufInputRowParser implements InputRowParser +{ + + private final MapInputRowParser inputRowCreator; + private final Descriptor descriptor; + + @JsonCreator + public ProtoBufInputRowParser( + @JsonProperty("timestampSpec") TimestampSpec timestampSpec, + @JsonProperty("data") ProtoBufDataSpec dataSpec, + @JsonProperty("dimensionExclusions") List dimensionExclusions) + { + + descriptor = getDescriptor(dataSpec.getDescriptorFileInClassPath()); + + + this.inputRowCreator = new MapInputRowParser(timestampSpec, dataSpec, dimensionExclusions); + + } + + @Override + public InputRow parse(byte[] input) + { + + Map theMap = buildStringKeyMap(input); + + return inputRowCreator.parse(theMap); + } + + private Map buildStringKeyMap(byte[] input) + { + Map theMap = Maps.newHashMap(); + + try + { + DynamicMessage message = DynamicMessage.parseFrom(descriptor, input); + Map allFields = message.getAllFields(); + + for (Map.Entry entry : allFields.entrySet()) + { + String name = entry.getKey().getName(); + if (theMap.containsKey(name)) + { + continue; + // TODO + // throw new RuntimeException("dupicate key " + name + " in " + + // message); + } + Object value = entry.getValue(); + if(value instanceof Descriptors.EnumValueDescriptor) { + Descriptors.EnumValueDescriptor desc = (Descriptors.EnumValueDescriptor) value; + value = desc.getName(); + } + + theMap.put(name, value); + } + + } catch (InvalidProtocolBufferException e) + { + // TODO + e.printStackTrace(); + } + return theMap; + } + + private Descriptor getDescriptor(String descriptorFileInClassPath) + { + try + { + InputStream fin = this.getClass().getClassLoader().getResourceAsStream(descriptorFileInClassPath); + FileDescriptorSet set = FileDescriptorSet.parseFrom(fin); + FileDescriptor file = FileDescriptor.buildFrom(set.getFile(0), new FileDescriptor[] + {}); + return file.getMessageTypes().get(0); + } catch (Exception e) + { + throw Throwables.propagate(e); + } + } + + @Override + public void addDimensionExclusion(String dimension) + { + inputRowCreator.addDimensionExclusion(dimension); + } +} diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java new file mode 100644 index 00000000000..c6116af3946 --- /dev/null +++ b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java @@ -0,0 +1,85 @@ +package com.metamx.druid.indexer.data; + +import static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE; +import static org.junit.Assert.assertEquals; + +import java.io.ByteArrayOutputStream; +import java.util.Arrays; +import java.util.List; + +import org.joda.time.DateTime; +import org.junit.Test; + +import com.metamx.druid.input.InputRow; + +/** + * @author jan.rudert + */ +public class ProtoBufInputRowParserTest { + + public static final String[] DIMENSIONS = new String[]{"eventType", "id", "someOtherId", "isValid"}; + + /* + eventType = 1; + + required uint64 id = 2; + required string timestamp = 3; + optional uint32 someOtherId = 4; + optional bool isValid = 5; + optional string description = 6; + + optional float someFloatColumn = 7; + optional uint32 someIntColumn = 8; + optional uint64 someLongColumn = 9; + */ + + @Test + public void testParse() throws Exception { + + //configure pares with desc file + ProtoBufInputRowParser parser = new ProtoBufInputRowParser(new TimestampSpec("timestamp", "iso"), + new ProtoBufDataSpec("prototest.desc", Arrays.asList(DIMENSIONS), null), + Arrays.asList()); + + + //create binary of proto test event + DateTime dateTime = new DateTime(2012, 07, 12, 9, 30); + ProtoTestEventWrapper.ProtoTestEvent event = ProtoTestEventWrapper.ProtoTestEvent.newBuilder() + .setDescription("description") + .setEventType(CATEGORY_ONE) + .setId(4711L) + .setIsValid(true) + .setSomeOtherId(4712) + .setTimestamp(dateTime.toString()) + .setSomeFloatColumn(47.11F) + .setSomeIntColumn(815) + .setSomeLongColumn(816L) + .build(); + + ByteArrayOutputStream out = new ByteArrayOutputStream(); + event.writeTo(out); + + InputRow row = parser.parse(out.toByteArray()); + System.out.println(row); + assertEquals(Arrays.asList(DIMENSIONS), row.getDimensions()); + assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch()); + + assertDimensionEquals(row, "id", "4711"); + assertDimensionEquals(row, "isValid", "true"); + assertDimensionEquals(row, "someOtherId", "4712"); + assertDimensionEquals(row, "description", "description"); + assertDimensionEquals(row, "eventType", CATEGORY_ONE.name()); + + + assertEquals(47.11F, row.getFloatMetric("someFloatColumn"), 0.0); + assertEquals(815.0F, row.getFloatMetric("someIntColumn"), 0.0); + assertEquals(816.0F, row.getFloatMetric("someLongColumn"), 0.0); + + } + + private void assertDimensionEquals(InputRow row, String dimension, Object expected) { + List values = row.getDimension(dimension); + assertEquals(1, values.size()); + assertEquals(expected, values.get(0)); + } +} diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoTestEventWrapper.java b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoTestEventWrapper.java new file mode 100644 index 00000000000..307d8a93a9c --- /dev/null +++ b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoTestEventWrapper.java @@ -0,0 +1,1375 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: ProtoTest.proto + +package com.metamx.druid.indexer.data; + +public final class ProtoTestEventWrapper { + private ProtoTestEventWrapper() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + } + public interface ProtoTestEventOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .prototest.ProtoTestEvent.EventCategory eventType = 1; + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + boolean hasEventType(); + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType(); + + // required uint64 id = 2; + /** + * required uint64 id = 2; + */ + boolean hasId(); + /** + * required uint64 id = 2; + */ + long getId(); + + // required string timestamp = 3; + /** + * required string timestamp = 3; + */ + boolean hasTimestamp(); + /** + * required string timestamp = 3; + */ + java.lang.String getTimestamp(); + /** + * required string timestamp = 3; + */ + com.google.protobuf.ByteString + getTimestampBytes(); + + // optional uint32 someOtherId = 4; + /** + * optional uint32 someOtherId = 4; + */ + boolean hasSomeOtherId(); + /** + * optional uint32 someOtherId = 4; + */ + int getSomeOtherId(); + + // optional bool isValid = 5; + /** + * optional bool isValid = 5; + */ + boolean hasIsValid(); + /** + * optional bool isValid = 5; + */ + boolean getIsValid(); + + // optional string description = 6; + /** + * optional string description = 6; + */ + boolean hasDescription(); + /** + * optional string description = 6; + */ + java.lang.String getDescription(); + /** + * optional string description = 6; + */ + com.google.protobuf.ByteString + getDescriptionBytes(); + + // optional float someFloatColumn = 7; + /** + * optional float someFloatColumn = 7; + */ + boolean hasSomeFloatColumn(); + /** + * optional float someFloatColumn = 7; + */ + float getSomeFloatColumn(); + + // optional uint32 someIntColumn = 8; + /** + * optional uint32 someIntColumn = 8; + */ + boolean hasSomeIntColumn(); + /** + * optional uint32 someIntColumn = 8; + */ + int getSomeIntColumn(); + + // optional uint64 someLongColumn = 9; + /** + * optional uint64 someLongColumn = 9; + */ + boolean hasSomeLongColumn(); + /** + * optional uint64 someLongColumn = 9; + */ + long getSomeLongColumn(); + } + /** + * Protobuf type {@code prototest.ProtoTestEvent} + */ + public static final class ProtoTestEvent extends + com.google.protobuf.GeneratedMessage + implements ProtoTestEventOrBuilder { + // Use ProtoTestEvent.newBuilder() to construct. + private ProtoTestEvent(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private ProtoTestEvent(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final ProtoTestEvent defaultInstance; + public static ProtoTestEvent getDefaultInstance() { + return defaultInstance; + } + + public ProtoTestEvent getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private ProtoTestEvent( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + int rawValue = input.readEnum(); + com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory value = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(1, rawValue); + } else { + bitField0_ |= 0x00000001; + eventType_ = value; + } + break; + } + case 16: { + bitField0_ |= 0x00000002; + id_ = input.readUInt64(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + timestamp_ = input.readBytes(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + someOtherId_ = input.readUInt32(); + break; + } + case 40: { + bitField0_ |= 0x00000010; + isValid_ = input.readBool(); + break; + } + case 50: { + bitField0_ |= 0x00000020; + description_ = input.readBytes(); + break; + } + case 61: { + bitField0_ |= 0x00000040; + someFloatColumn_ = input.readFloat(); + break; + } + case 64: { + bitField0_ |= 0x00000080; + someIntColumn_ = input.readUInt32(); + break; + } + case 72: { + bitField0_ |= 0x00000100; + someLongColumn_ = input.readUInt64(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.class, com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public ProtoTestEvent parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new ProtoTestEvent(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + /** + * Protobuf enum {@code prototest.ProtoTestEvent.EventCategory} + */ + public enum EventCategory + implements com.google.protobuf.ProtocolMessageEnum { + /** + * CATEGORY_ZERO = 0; + */ + CATEGORY_ZERO(0, 0), + /** + * CATEGORY_ONE = 1; + */ + CATEGORY_ONE(1, 1), + /** + * CATEGORY_TWO = 2; + */ + CATEGORY_TWO(2, 2), + ; + + /** + * CATEGORY_ZERO = 0; + */ + public static final int CATEGORY_ZERO_VALUE = 0; + /** + * CATEGORY_ONE = 1; + */ + public static final int CATEGORY_ONE_VALUE = 1; + /** + * CATEGORY_TWO = 2; + */ + public static final int CATEGORY_TWO_VALUE = 2; + + + public final int getNumber() { return value; } + + public static EventCategory valueOf(int value) { + switch (value) { + case 0: return CATEGORY_ZERO; + case 1: return CATEGORY_ONE; + case 2: return CATEGORY_TWO; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static com.google.protobuf.Internal.EnumLiteMap + internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public EventCategory findValueByNumber(int number) { + return EventCategory.valueOf(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(index); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDescriptor().getEnumTypes().get(0); + } + + private static final EventCategory[] VALUES = values(); + + public static EventCategory valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int index; + private final int value; + + private EventCategory(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:prototest.ProtoTestEvent.EventCategory) + } + + private int bitField0_; + // required .prototest.ProtoTestEvent.EventCategory eventType = 1; + public static final int EVENTTYPE_FIELD_NUMBER = 1; + private com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_; + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + public boolean hasEventType() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() { + return eventType_; + } + + // required uint64 id = 2; + public static final int ID_FIELD_NUMBER = 2; + private long id_; + /** + * required uint64 id = 2; + */ + public boolean hasId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required uint64 id = 2; + */ + public long getId() { + return id_; + } + + // required string timestamp = 3; + public static final int TIMESTAMP_FIELD_NUMBER = 3; + private java.lang.Object timestamp_; + /** + * required string timestamp = 3; + */ + public boolean hasTimestamp() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required string timestamp = 3; + */ + public java.lang.String getTimestamp() { + java.lang.Object ref = timestamp_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + timestamp_ = s; + } + return s; + } + } + /** + * required string timestamp = 3; + */ + public com.google.protobuf.ByteString + getTimestampBytes() { + java.lang.Object ref = timestamp_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + timestamp_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // optional uint32 someOtherId = 4; + public static final int SOMEOTHERID_FIELD_NUMBER = 4; + private int someOtherId_; + /** + * optional uint32 someOtherId = 4; + */ + public boolean hasSomeOtherId() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint32 someOtherId = 4; + */ + public int getSomeOtherId() { + return someOtherId_; + } + + // optional bool isValid = 5; + public static final int ISVALID_FIELD_NUMBER = 5; + private boolean isValid_; + /** + * optional bool isValid = 5; + */ + public boolean hasIsValid() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional bool isValid = 5; + */ + public boolean getIsValid() { + return isValid_; + } + + // optional string description = 6; + public static final int DESCRIPTION_FIELD_NUMBER = 6; + private java.lang.Object description_; + /** + * optional string description = 6; + */ + public boolean hasDescription() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + /** + * optional string description = 6; + */ + public java.lang.String getDescription() { + java.lang.Object ref = description_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + description_ = s; + } + return s; + } + } + /** + * optional string description = 6; + */ + public com.google.protobuf.ByteString + getDescriptionBytes() { + java.lang.Object ref = description_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + description_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // optional float someFloatColumn = 7; + public static final int SOMEFLOATCOLUMN_FIELD_NUMBER = 7; + private float someFloatColumn_; + /** + * optional float someFloatColumn = 7; + */ + public boolean hasSomeFloatColumn() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + /** + * optional float someFloatColumn = 7; + */ + public float getSomeFloatColumn() { + return someFloatColumn_; + } + + // optional uint32 someIntColumn = 8; + public static final int SOMEINTCOLUMN_FIELD_NUMBER = 8; + private int someIntColumn_; + /** + * optional uint32 someIntColumn = 8; + */ + public boolean hasSomeIntColumn() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + /** + * optional uint32 someIntColumn = 8; + */ + public int getSomeIntColumn() { + return someIntColumn_; + } + + // optional uint64 someLongColumn = 9; + public static final int SOMELONGCOLUMN_FIELD_NUMBER = 9; + private long someLongColumn_; + /** + * optional uint64 someLongColumn = 9; + */ + public boolean hasSomeLongColumn() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + /** + * optional uint64 someLongColumn = 9; + */ + public long getSomeLongColumn() { + return someLongColumn_; + } + + private void initFields() { + eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; + id_ = 0L; + timestamp_ = ""; + someOtherId_ = 0; + isValid_ = false; + description_ = ""; + someFloatColumn_ = 0F; + someIntColumn_ = 0; + someLongColumn_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasEventType()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasTimestamp()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeEnum(1, eventType_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, id_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getTimestampBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt32(4, someOtherId_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBool(5, isValid_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeBytes(6, getDescriptionBytes()); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeFloat(7, someFloatColumn_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeUInt32(8, someIntColumn_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeUInt64(9, someLongColumn_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(1, eventType_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(2, id_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, getTimestampBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(4, someOtherId_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(5, isValid_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(6, getDescriptionBytes()); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += com.google.protobuf.CodedOutputStream + .computeFloatSize(7, someFloatColumn_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(8, someIntColumn_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(9, someLongColumn_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code prototest.ProtoTestEvent} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEventOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.class, com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.Builder.class); + } + + // Construct using com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; + bitField0_ = (bitField0_ & ~0x00000001); + id_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + timestamp_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + someOtherId_ = 0; + bitField0_ = (bitField0_ & ~0x00000008); + isValid_ = false; + bitField0_ = (bitField0_ & ~0x00000010); + description_ = ""; + bitField0_ = (bitField0_ & ~0x00000020); + someFloatColumn_ = 0F; + bitField0_ = (bitField0_ & ~0x00000040); + someIntColumn_ = 0; + bitField0_ = (bitField0_ & ~0x00000080); + someLongColumn_ = 0L; + bitField0_ = (bitField0_ & ~0x00000100); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; + } + + public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance(); + } + + public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent build() { + com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent buildPartial() { + com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent result = new com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.eventType_ = eventType_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.id_ = id_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.timestamp_ = timestamp_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.someOtherId_ = someOtherId_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.isValid_ = isValid_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.description_ = description_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000040; + } + result.someFloatColumn_ = someFloatColumn_; + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000080; + } + result.someIntColumn_ = someIntColumn_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000100; + } + result.someLongColumn_ = someLongColumn_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent) { + return mergeFrom((com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent other) { + if (other == com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance()) return this; + if (other.hasEventType()) { + setEventType(other.getEventType()); + } + if (other.hasId()) { + setId(other.getId()); + } + if (other.hasTimestamp()) { + bitField0_ |= 0x00000004; + timestamp_ = other.timestamp_; + onChanged(); + } + if (other.hasSomeOtherId()) { + setSomeOtherId(other.getSomeOtherId()); + } + if (other.hasIsValid()) { + setIsValid(other.getIsValid()); + } + if (other.hasDescription()) { + bitField0_ |= 0x00000020; + description_ = other.description_; + onChanged(); + } + if (other.hasSomeFloatColumn()) { + setSomeFloatColumn(other.getSomeFloatColumn()); + } + if (other.hasSomeIntColumn()) { + setSomeIntColumn(other.getSomeIntColumn()); + } + if (other.hasSomeLongColumn()) { + setSomeLongColumn(other.getSomeLongColumn()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasEventType()) { + + return false; + } + if (!hasId()) { + + return false; + } + if (!hasTimestamp()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .prototest.ProtoTestEvent.EventCategory eventType = 1; + private com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + public boolean hasEventType() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() { + return eventType_; + } + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + public Builder setEventType(com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + eventType_ = value; + onChanged(); + return this; + } + /** + * required .prototest.ProtoTestEvent.EventCategory eventType = 1; + */ + public Builder clearEventType() { + bitField0_ = (bitField0_ & ~0x00000001); + eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; + onChanged(); + return this; + } + + // required uint64 id = 2; + private long id_ ; + /** + * required uint64 id = 2; + */ + public boolean hasId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required uint64 id = 2; + */ + public long getId() { + return id_; + } + /** + * required uint64 id = 2; + */ + public Builder setId(long value) { + bitField0_ |= 0x00000002; + id_ = value; + onChanged(); + return this; + } + /** + * required uint64 id = 2; + */ + public Builder clearId() { + bitField0_ = (bitField0_ & ~0x00000002); + id_ = 0L; + onChanged(); + return this; + } + + // required string timestamp = 3; + private java.lang.Object timestamp_ = ""; + /** + * required string timestamp = 3; + */ + public boolean hasTimestamp() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required string timestamp = 3; + */ + public java.lang.String getTimestamp() { + java.lang.Object ref = timestamp_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((com.google.protobuf.ByteString) ref) + .toStringUtf8(); + timestamp_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string timestamp = 3; + */ + public com.google.protobuf.ByteString + getTimestampBytes() { + java.lang.Object ref = timestamp_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + timestamp_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * required string timestamp = 3; + */ + public Builder setTimestamp( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + timestamp_ = value; + onChanged(); + return this; + } + /** + * required string timestamp = 3; + */ + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000004); + timestamp_ = getDefaultInstance().getTimestamp(); + onChanged(); + return this; + } + /** + * required string timestamp = 3; + */ + public Builder setTimestampBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + timestamp_ = value; + onChanged(); + return this; + } + + // optional uint32 someOtherId = 4; + private int someOtherId_ ; + /** + * optional uint32 someOtherId = 4; + */ + public boolean hasSomeOtherId() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional uint32 someOtherId = 4; + */ + public int getSomeOtherId() { + return someOtherId_; + } + /** + * optional uint32 someOtherId = 4; + */ + public Builder setSomeOtherId(int value) { + bitField0_ |= 0x00000008; + someOtherId_ = value; + onChanged(); + return this; + } + /** + * optional uint32 someOtherId = 4; + */ + public Builder clearSomeOtherId() { + bitField0_ = (bitField0_ & ~0x00000008); + someOtherId_ = 0; + onChanged(); + return this; + } + + // optional bool isValid = 5; + private boolean isValid_ ; + /** + * optional bool isValid = 5; + */ + public boolean hasIsValid() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional bool isValid = 5; + */ + public boolean getIsValid() { + return isValid_; + } + /** + * optional bool isValid = 5; + */ + public Builder setIsValid(boolean value) { + bitField0_ |= 0x00000010; + isValid_ = value; + onChanged(); + return this; + } + /** + * optional bool isValid = 5; + */ + public Builder clearIsValid() { + bitField0_ = (bitField0_ & ~0x00000010); + isValid_ = false; + onChanged(); + return this; + } + + // optional string description = 6; + private java.lang.Object description_ = ""; + /** + * optional string description = 6; + */ + public boolean hasDescription() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + /** + * optional string description = 6; + */ + public java.lang.String getDescription() { + java.lang.Object ref = description_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((com.google.protobuf.ByteString) ref) + .toStringUtf8(); + description_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string description = 6; + */ + public com.google.protobuf.ByteString + getDescriptionBytes() { + java.lang.Object ref = description_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + description_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * optional string description = 6; + */ + public Builder setDescription( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + description_ = value; + onChanged(); + return this; + } + /** + * optional string description = 6; + */ + public Builder clearDescription() { + bitField0_ = (bitField0_ & ~0x00000020); + description_ = getDefaultInstance().getDescription(); + onChanged(); + return this; + } + /** + * optional string description = 6; + */ + public Builder setDescriptionBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + description_ = value; + onChanged(); + return this; + } + + // optional float someFloatColumn = 7; + private float someFloatColumn_ ; + /** + * optional float someFloatColumn = 7; + */ + public boolean hasSomeFloatColumn() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + /** + * optional float someFloatColumn = 7; + */ + public float getSomeFloatColumn() { + return someFloatColumn_; + } + /** + * optional float someFloatColumn = 7; + */ + public Builder setSomeFloatColumn(float value) { + bitField0_ |= 0x00000040; + someFloatColumn_ = value; + onChanged(); + return this; + } + /** + * optional float someFloatColumn = 7; + */ + public Builder clearSomeFloatColumn() { + bitField0_ = (bitField0_ & ~0x00000040); + someFloatColumn_ = 0F; + onChanged(); + return this; + } + + // optional uint32 someIntColumn = 8; + private int someIntColumn_ ; + /** + * optional uint32 someIntColumn = 8; + */ + public boolean hasSomeIntColumn() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + /** + * optional uint32 someIntColumn = 8; + */ + public int getSomeIntColumn() { + return someIntColumn_; + } + /** + * optional uint32 someIntColumn = 8; + */ + public Builder setSomeIntColumn(int value) { + bitField0_ |= 0x00000080; + someIntColumn_ = value; + onChanged(); + return this; + } + /** + * optional uint32 someIntColumn = 8; + */ + public Builder clearSomeIntColumn() { + bitField0_ = (bitField0_ & ~0x00000080); + someIntColumn_ = 0; + onChanged(); + return this; + } + + // optional uint64 someLongColumn = 9; + private long someLongColumn_ ; + /** + * optional uint64 someLongColumn = 9; + */ + public boolean hasSomeLongColumn() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + /** + * optional uint64 someLongColumn = 9; + */ + public long getSomeLongColumn() { + return someLongColumn_; + } + /** + * optional uint64 someLongColumn = 9; + */ + public Builder setSomeLongColumn(long value) { + bitField0_ |= 0x00000100; + someLongColumn_ = value; + onChanged(); + return this; + } + /** + * optional uint64 someLongColumn = 9; + */ + public Builder clearSomeLongColumn() { + bitField0_ = (bitField0_ & ~0x00000100); + someLongColumn_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:prototest.ProtoTestEvent) + } + + static { + defaultInstance = new ProtoTestEvent(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:prototest.ProtoTestEvent) + } + + private static com.google.protobuf.Descriptors.Descriptor + internal_static_prototest_ProtoTestEvent_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_prototest_ProtoTestEvent_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\017ProtoTest.proto\022\tprototest\"\266\002\n\016ProtoTe" + + "stEvent\022:\n\teventType\030\001 \002(\0162\'.prototest.P" + + "rotoTestEvent.EventCategory\022\n\n\002id\030\002 \002(\004\022" + + "\021\n\ttimestamp\030\003 \002(\t\022\023\n\013someOtherId\030\004 \001(\r\022" + + "\017\n\007isValid\030\005 \001(\010\022\023\n\013description\030\006 \001(\t\022\027\n" + + "\017someFloatColumn\030\007 \001(\002\022\025\n\rsomeIntColumn\030" + + "\010 \001(\r\022\026\n\016someLongColumn\030\t \001(\004\"F\n\rEventCa" + + "tegory\022\021\n\rCATEGORY_ZERO\020\000\022\020\n\014CATEGORY_ON" + + "E\020\001\022\020\n\014CATEGORY_TWO\020\002B6\n\035com.metamx.drui" + + "d.indexer.dataB\025ProtoTestEventWrapper" + }; + com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public com.google.protobuf.ExtensionRegistry assignDescriptors( + com.google.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + internal_static_prototest_ProtoTestEvent_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_prototest_ProtoTestEvent_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_prototest_ProtoTestEvent_descriptor, + new java.lang.String[] { "EventType", "Id", "Timestamp", "SomeOtherId", "IsValid", "Description", "SomeFloatColumn", "SomeIntColumn", "SomeLongColumn", }); + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }, assigner); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/indexing-common/src/test/resources/ProtoTest.proto b/indexing-common/src/test/resources/ProtoTest.proto new file mode 100644 index 00000000000..c7822f9ff1b --- /dev/null +++ b/indexing-common/src/test/resources/ProtoTest.proto @@ -0,0 +1,31 @@ + +package prototest; +option java_package = "com.metamx.druid.indexer.data"; +option java_outer_classname = "ProtoTestEventWrapper"; + + + +message ProtoTestEvent { + + +enum EventCategory { + CATEGORY_ZERO = 0; + CATEGORY_ONE = 1; + CATEGORY_TWO = 2; +} + + required EventCategory eventType = 1; + + required uint64 id = 2; + required string timestamp = 3; + optional uint32 someOtherId = 4; + optional bool isValid = 5; + optional string description = 6; + + optional float someFloatColumn = 7; + optional uint32 someIntColumn = 8; + optional uint64 someLongColumn = 9; + + + +} diff --git a/indexing-common/src/test/resources/prototest.desc b/indexing-common/src/test/resources/prototest.desc new file mode 100644 index 0000000000000000000000000000000000000000..649ce5bcb8fe5df743b9fff73e1835e4467efd4e GIT binary patch literal 400 zcmY+APfNov7>8|F-TG?(EHdpy7`wPr4jw#sbET7kSQ*8k7YR$4fu<>GCfkSb>-hOh zst$U}%k%p^;Z5MH2hp>VS{AI)xV97W+d~^L`F0^Z#MvjSG`a(y*`e6iEOEx}G`zX~ z6|I50ZQMF3(QG5t9s#(VCa#e@Gz4FBVH6_jq-T(i&f&C?g5~;ysRd2Eu`>-&1byDT zkkW|91`KSAvSzJ#t+}j{69av836WLjr6f9)1S^I~`K2^4Oi-!qr*|ESP!T4pX(| X6!VI*9aESRO=hF+N39g8Yo>kxsG4&{ literal 0 HcmV?d00001 diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java index 1c189ed95e9..3c6d528d6a6 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java @@ -19,22 +19,8 @@ package com.metamx.druid.realtime.firehose; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Charsets; -import com.google.common.collect.ImmutableMap; -import com.metamx.common.exception.FormattedException; -import com.metamx.common.logger.Logger; -import com.metamx.druid.indexer.data.StringInputRowParser; -import com.metamx.druid.input.InputRow; -import kafka.consumer.Consumer; -import kafka.consumer.ConsumerConfig; -import kafka.consumer.KafkaStream; -import kafka.javaapi.consumer.ConsumerConnector; -import kafka.message.Message; -import kafka.message.MessageAndMetadata; - import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.CharBuffer; import java.nio.charset.CoderResult; import java.nio.charset.CodingErrorAction; @@ -43,122 +29,217 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import kafka.consumer.Consumer; +import kafka.consumer.ConsumerConfig; +import kafka.consumer.KafkaStream; +import kafka.javaapi.consumer.ConsumerConnector; +import kafka.message.Message; +import kafka.message.MessageAndMetadata; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; +import com.google.common.collect.ImmutableMap; +import com.metamx.common.exception.FormattedException; +import com.metamx.common.logger.Logger; +import com.metamx.druid.indexer.data.InputRowParser; +import com.metamx.druid.indexer.data.ProtoBufInputRowParser; +import com.metamx.druid.indexer.data.StringInputRowParser; +import com.metamx.druid.input.InputRow; + /** */ public class KafkaFirehoseFactory implements FirehoseFactory { - private static final Logger log = new Logger(KafkaFirehoseFactory.class); + private static final Logger log = new Logger(KafkaFirehoseFactory.class); - @JsonProperty - private final Properties consumerProps; + @JsonProperty + private final Properties consumerProps; - @JsonProperty - private final String feed; + @JsonProperty + private final String feed; - @JsonProperty - private final StringInputRowParser parser; + @JsonProperty + private final InputRowParser parser; - @JsonCreator - public KafkaFirehoseFactory( - @JsonProperty("consumerProps") Properties consumerProps, - @JsonProperty("feed") String feed, - @JsonProperty("parser") StringInputRowParser parser - ) - { - this.consumerProps = consumerProps; - this.feed = feed; - this.parser = parser; + @JsonCreator + public KafkaFirehoseFactory( + @JsonProperty("consumerProps") Properties consumerProps, + @JsonProperty("feed") String feed, + @JsonProperty("parser") InputRowParser parser) + { + this.consumerProps = consumerProps; + this.feed = feed; + this.parser = parser; - parser.addDimensionExclusion("feed"); - } + parser.addDimensionExclusion("feed"); + } - @Override - public Firehose connect() throws IOException - { - final ConsumerConnector connector = Consumer.createJavaConsumerConnector(new ConsumerConfig(consumerProps)); + @Override + public Firehose connect() throws IOException + { + final ConsumerConnector connector = Consumer.createJavaConsumerConnector(new ConsumerConfig(consumerProps)); - final Map>> streams = connector.createMessageStreams(ImmutableMap.of(feed, 1)); + final Map>> streams = connector.createMessageStreams(ImmutableMap.of(feed, 1)); - final List> streamList = streams.get(feed); - if (streamList == null || streamList.size() != 1) { - return null; - } + final List> streamList = streams.get(feed); + if (streamList == null || streamList.size() != 1) + { + return null; + } - final KafkaStream stream = streamList.get(0); + final KafkaStream stream = streamList.get(0); - return new Firehose() - { - Iterator> iter = stream.iterator(); - private CharBuffer chars = null; + if (parser instanceof StringInputRowParser) + { - @Override - public boolean hasMore() - { - return iter.hasNext(); - } + return new StringMessageFirehose(connector, stream); + } else if (parser instanceof ProtoBufInputRowParser) + { + return new ProtoBufMessageFirehose(stream, connector); + } + throw new RuntimeException("No Firehose for parser: " + parser.getClass().getName()); + } - @Override - public InputRow nextRow() throws FormattedException - { - final Message message = iter.next().message(); + private abstract static class AbstractKafkaFirehose implements Firehose + { + protected final ConsumerConnector connector; + protected final Iterator> iter; - if (message == null) { - return null; - } + public AbstractKafkaFirehose(ConsumerConnector connector, KafkaStream stream) + { + iter = stream.iterator(); + this.connector = connector; + } - int payloadSize = message.payloadSize(); - if (chars == null || chars.remaining() < payloadSize) { - chars = CharBuffer.allocate(payloadSize); - } + @Override + public boolean hasMore() + { + return iter.hasNext(); + } - final CoderResult coderResult = Charsets.UTF_8.newDecoder() - .onMalformedInput(CodingErrorAction.REPLACE) - .onUnmappableCharacter(CodingErrorAction.REPLACE) - .decode(message.payload(), chars, true); + @Override + public InputRow nextRow() throws FormattedException + { + final Message message = iter.next().message(); - if (coderResult.isUnderflow()) { - chars.flip(); - try { - return parser.parse(chars.toString()); - } - finally { - chars.clear(); - } - } - else { - throw new FormattedException.Builder() - .withErrorCode(FormattedException.ErrorCode.UNPARSABLE_ROW) - .withMessage(String.format("Failed with CoderResult[%s]", coderResult)) - .build(); - } - } + if (message == null) + { + return null; + } - @Override - public Runnable commit() - { - return new Runnable() - { - @Override - public void run() - { - /* - This is actually not going to do exactly what we want, cause it will be called asynchronously - after the persist is complete. So, it's going to commit that it's processed more than was actually - persisted. This is unfortunate, but good enough for now. Should revisit along with an upgrade - of our Kafka version. - */ + return parseMessage(message); + } - log.info("committing offsets"); - connector.commitOffsets(); - } - }; - } + protected abstract InputRow parseMessage(Message message); - @Override - public void close() throws IOException - { - connector.shutdown(); - } - }; - } + @Override + public Runnable commit() + { + return new Runnable() + { + @Override + public void run() + { + /* + * This is actually not going to do exactly what we want, cause it + * will be called asynchronously after the persist is complete. So, + * it's going to commit that it's processed more than was actually + * persisted. This is unfortunate, but good enough for now. Should + * revisit along with an upgrade of our Kafka version. + */ + + log.info("committing offsets"); + connector.commitOffsets(); + } + }; + } + + @Override + public void close() throws IOException + { + connector.shutdown(); + } + } + + private class StringMessageFirehose extends AbstractKafkaFirehose + { + + private CharBuffer chars = null; + + public StringMessageFirehose(ConsumerConnector connector, KafkaStream stream) + { + super(connector, stream); + } + + @Override + public InputRow parseMessage(Message message) throws FormattedException + { + + int payloadSize = message.payloadSize(); + if (chars == null || chars.remaining() < payloadSize) + { + chars = CharBuffer.allocate(payloadSize); + } + + final CoderResult coderResult = Charsets.UTF_8.newDecoder() + .onMalformedInput(CodingErrorAction.REPLACE) + .onUnmappableCharacter(CodingErrorAction.REPLACE) + .decode(message.payload(), chars, true); + + if (coderResult.isUnderflow()) + { + chars.flip(); + try + { + return parser.parse(chars.toString()); + } finally + { + chars.clear(); + } + } + else + { + throw new FormattedException.Builder() + .withErrorCode(FormattedException.ErrorCode.UNPARSABLE_ROW) + .withMessage(String.format("Failed with CoderResult[%s]", coderResult)) + .build(); + } + } + + } + + private class ProtoBufMessageFirehose extends AbstractKafkaFirehose + { + + private ByteBuffer bytes = null; + + public ProtoBufMessageFirehose(KafkaStream stream, ConsumerConnector connector) + { + super(connector, stream); + } + + @Override + public InputRow parseMessage(Message message) throws FormattedException + { + + int payloadSize = message.payloadSize(); + if (bytes == null || bytes.remaining() < payloadSize) + { + bytes = ByteBuffer.allocate(payloadSize); + } + + bytes.put(message.payload()); + + bytes.flip(); + try + { + return parser.parse(bytes); + } finally + { + bytes.clear(); + } + } + + } } From a8039107d8fb63867af3e22e5a179bdfcbd6f924 Mon Sep 17 00:00:00 2001 From: Jan Rudert Date: Fri, 12 Jul 2013 16:02:15 +0200 Subject: [PATCH 02/13] no bytebuffer in the middle --- .../indexer/data/ProtoBufInputRowParser.java | 9 ++++---- .../data/ProtoBufInputRowParserTest.java | 3 ++- .../firehose/KafkaFirehoseFactory.java | 21 ++----------------- 3 files changed, 9 insertions(+), 24 deletions(-) diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java index 38ee99663e4..7e7b7494083 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java @@ -12,6 +12,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Throwables; import com.google.common.collect.Maps; +import com.google.protobuf.ByteString; import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; import com.google.protobuf.InvalidProtocolBufferException; @@ -20,7 +21,7 @@ import com.metamx.druid.input.InputRow; /** * @author jan.rudert */ -public class ProtoBufInputRowParser implements InputRowParser +public class ProtoBufInputRowParser implements InputRowParser { private final MapInputRowParser inputRowCreator; @@ -41,7 +42,7 @@ public class ProtoBufInputRowParser implements InputRowParser } @Override - public InputRow parse(byte[] input) + public InputRow parse(ByteString input) { Map theMap = buildStringKeyMap(input); @@ -49,7 +50,7 @@ public class ProtoBufInputRowParser implements InputRowParser return inputRowCreator.parse(theMap); } - private Map buildStringKeyMap(byte[] input) + private Map buildStringKeyMap(ByteString input) { Map theMap = Maps.newHashMap(); @@ -60,7 +61,7 @@ public class ProtoBufInputRowParser implements InputRowParser for (Map.Entry entry : allFields.entrySet()) { - String name = entry.getKey().getName(); + String name = entry.getKey().getName().toLowerCase(); if (theMap.containsKey(name)) { continue; diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java index c6116af3946..f17d9cf9749 100644 --- a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java @@ -7,6 +7,7 @@ import java.io.ByteArrayOutputStream; import java.util.Arrays; import java.util.List; +import com.google.protobuf.ByteString; import org.joda.time.DateTime; import org.junit.Test; @@ -59,7 +60,7 @@ public class ProtoBufInputRowParserTest { ByteArrayOutputStream out = new ByteArrayOutputStream(); event.writeTo(out); - InputRow row = parser.parse(out.toByteArray()); + InputRow row = parser.parse(ByteString.copyFrom(out.toByteArray())); System.out.println(row); assertEquals(Arrays.asList(DIMENSIONS), row.getDimensions()); assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch()); diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java index 3c6d528d6a6..514cb1025b2 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import com.google.protobuf.ByteString; import kafka.consumer.Consumer; import kafka.consumer.ConsumerConfig; import kafka.consumer.KafkaStream; @@ -212,8 +213,6 @@ public class KafkaFirehoseFactory implements FirehoseFactory private class ProtoBufMessageFirehose extends AbstractKafkaFirehose { - private ByteBuffer bytes = null; - public ProtoBufMessageFirehose(KafkaStream stream, ConsumerConnector connector) { super(connector, stream); @@ -222,23 +221,7 @@ public class KafkaFirehoseFactory implements FirehoseFactory @Override public InputRow parseMessage(Message message) throws FormattedException { - - int payloadSize = message.payloadSize(); - if (bytes == null || bytes.remaining() < payloadSize) - { - bytes = ByteBuffer.allocate(payloadSize); - } - - bytes.put(message.payload()); - - bytes.flip(); - try - { - return parser.parse(bytes); - } finally - { - bytes.clear(); - } + return parser.parse(ByteString.copyFrom(message.payload())); } } From 527c6c81c59d37d9a500f918e1c6eb8b45c2bd4b Mon Sep 17 00:00:00 2001 From: Jan Rudert Date: Fri, 19 Jul 2013 14:12:53 +0200 Subject: [PATCH 03/13] refactored StringInputRowParser to handle a ByteBuffer, better wiring and configuration of KafkaFirehose --- .../flights/FlightsFirehoseFactory.java | 3 +- .../metamx/druid/indexer/data/DataSpec.java | 3 +- .../druid/indexer/data/MapInputRowParser.java | 23 ++-- .../druid/indexer/data/ProtoBufDataSpec.java | 69 ------------ .../indexer/data/ProtoBufInputRowParser.java | 22 ++-- .../indexer/data/StringInputRowParser.java | 105 +++++++++++++----- .../data/ProtoBufInputRowParserTest.java | 8 +- .../data/StringInputRowParserTest.java | 23 ++++ .../indexer/HadoopDruidIndexerMapper.java | 3 +- .../druid/indexer/IndexGeneratorJob.java | 2 +- .../common/index/StaticS3FirehoseFactory.java | 3 +- .../firehose/KafkaFirehoseFactory.java | 99 +++-------------- .../com/metamx/druid/index/v1/TestIndex.java | 3 +- 13 files changed, 151 insertions(+), 215 deletions(-) delete mode 100644 indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufDataSpec.java create mode 100644 indexing-common/src/test/java/com/metamx/druid/indexer/data/StringInputRowParserTest.java diff --git a/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java b/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java index 66b8de4450c..7413d56eeea 100644 --- a/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java +++ b/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java @@ -36,6 +36,7 @@ import java.io.FileInputStream; import java.io.FileReader; import java.io.IOException; import java.io.InputStreamReader; +import java.nio.ByteBuffer; import java.util.Iterator; import java.util.zip.GZIPInputStream; @@ -118,7 +119,7 @@ public class FlightsFirehoseFactory implements FirehoseFactory @Override public InputRow nextRow() { - final InputRow retVal = parser.parse(line); + final InputRow retVal = parser.parse(ByteBuffer.wrap(line.getBytes())); line = null; return retVal; } diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java index 36d942d8d62..099d0c8d535 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java @@ -32,8 +32,7 @@ import java.util.List; @JsonSubTypes(value = { @JsonSubTypes.Type(name = "json", value = JSONDataSpec.class), @JsonSubTypes.Type(name = "csv", value = CSVDataSpec.class), - @JsonSubTypes.Type(name = "tsv", value = DelimitedDataSpec.class), - @JsonSubTypes.Type(name = "protobuf", value = ProtoBufDataSpec.class) + @JsonSubTypes.Type(name = "tsv", value = DelimitedDataSpec.class) }) public interface DataSpec { diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java index 60a97c131bf..93c221cfaf9 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java @@ -8,6 +8,7 @@ import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import org.joda.time.DateTime; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -15,18 +16,20 @@ import java.util.Set; public class MapInputRowParser implements InputRowParser> { private final TimestampSpec timestampSpec; - private final DataSpec dataSpec; + private List dimensions; private final Set dimensionExclusions; @JsonCreator public MapInputRowParser( @JsonProperty("timestampSpec") TimestampSpec timestampSpec, - @JsonProperty("data") DataSpec dataSpec, + @JsonProperty("dimensions") List dimensions, @JsonProperty("dimensionExclusions") List dimensionExclusions ) { this.timestampSpec = timestampSpec; - this.dataSpec = dataSpec; + if (dimensions != null) { + this.dimensions = Collections.unmodifiableList(Lists.newArrayList(dimensions)); + } this.dimensionExclusions = Sets.newHashSet(); if (dimensionExclusions != null) { for (String dimensionExclusion : dimensionExclusions) { @@ -39,8 +42,8 @@ public class MapInputRowParser implements InputRowParser> @Override public InputRow parse(Map theMap) { - final List dimensions = dataSpec.hasCustomDimensions() - ? dataSpec.getDimensions() + final List dimensions = hasCustomDimensions() + ? this.dimensions : Lists.newArrayList(Sets.difference(theMap.keySet(), dimensionExclusions)); final DateTime timestamp = timestampSpec.extractTimestamp(theMap); @@ -57,6 +60,10 @@ public class MapInputRowParser implements InputRowParser> return new MapBasedInputRow(timestamp.getMillis(), dimensions, theMap); } + private boolean hasCustomDimensions() { + return dimensions != null; + } + @Override public void addDimensionExclusion(String dimension) { @@ -69,10 +76,10 @@ public class MapInputRowParser implements InputRowParser> return timestampSpec; } - @JsonProperty("data") - public DataSpec getDataSpec() + @JsonProperty + public List getDimensions() { - return dataSpec; + return dimensions; } @JsonProperty diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufDataSpec.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufDataSpec.java deleted file mode 100644 index 042b0603061..00000000000 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufDataSpec.java +++ /dev/null @@ -1,69 +0,0 @@ -package com.metamx.druid.indexer.data; - -import java.util.List; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.Lists; -import com.metamx.common.parsers.Parser; -import com.metamx.druid.index.v1.SpatialDimensionSchema; - -/** - * @author jan.rudert - */ -public class ProtoBufDataSpec implements DataSpec{ - private final List dimensions; - private final List spatialDimensions; - private final String descriptorFileInClasspath; - - @JsonCreator - public ProtoBufDataSpec( - @JsonProperty("descriptor") String descriptorFileInClasspath, - @JsonProperty("dimensions") List dimensions, - @JsonProperty("spatialDimensions") List spatialDimensions - ) - { - this.descriptorFileInClasspath = descriptorFileInClasspath; - this.dimensions = dimensions; - this.spatialDimensions = (spatialDimensions == null) - ? Lists.newArrayList() - : spatialDimensions; - - } - - @JsonProperty("descriptor") - public String getDescriptorFileInClassPath() { - return descriptorFileInClasspath; - } - - @JsonProperty("dimensions") - @Override - public List getDimensions() - { - return dimensions; - } - - @JsonProperty("spatialDimensions") - @Override - public List getSpatialDimensions() - { - return spatialDimensions; - } - - @Override - public void verify(List usedCols) - { - } - - @Override - public boolean hasCustomDimensions() - { - return !(dimensions == null || dimensions.isEmpty()); - } - - @Override - public Parser getParser() - { - throw new UnsupportedOperationException("No String parser for protobuf data"); - } -} diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java index 7e7b7494083..a96e0abc8ed 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java @@ -5,6 +5,7 @@ import static com.google.protobuf.Descriptors.Descriptor; import static com.google.protobuf.Descriptors.FileDescriptor; import java.io.InputStream; +import java.nio.ByteBuffer; import java.util.List; import java.util.Map; @@ -21,7 +22,7 @@ import com.metamx.druid.input.InputRow; /** * @author jan.rudert */ -public class ProtoBufInputRowParser implements InputRowParser +public class ProtoBufInputRowParser implements InputRowParser { private final MapInputRowParser inputRowCreator; @@ -30,19 +31,18 @@ public class ProtoBufInputRowParser implements InputRowParser @JsonCreator public ProtoBufInputRowParser( @JsonProperty("timestampSpec") TimestampSpec timestampSpec, - @JsonProperty("data") ProtoBufDataSpec dataSpec, - @JsonProperty("dimensionExclusions") List dimensionExclusions) + @JsonProperty("dimensions") List dimensions, + @JsonProperty("dimensionExclusions") List dimensionExclusions, + @JsonProperty("descriptor") String descriptorFileInClasspath) { - descriptor = getDescriptor(dataSpec.getDescriptorFileInClassPath()); - - - this.inputRowCreator = new MapInputRowParser(timestampSpec, dataSpec, dimensionExclusions); + descriptor = getDescriptor(descriptorFileInClasspath); + inputRowCreator = new MapInputRowParser(timestampSpec, dimensions, dimensionExclusions); } @Override - public InputRow parse(ByteString input) + public InputRow parse(ByteBuffer input) { Map theMap = buildStringKeyMap(input); @@ -50,18 +50,18 @@ public class ProtoBufInputRowParser implements InputRowParser return inputRowCreator.parse(theMap); } - private Map buildStringKeyMap(ByteString input) + private Map buildStringKeyMap(ByteBuffer input) { Map theMap = Maps.newHashMap(); try { - DynamicMessage message = DynamicMessage.parseFrom(descriptor, input); + DynamicMessage message = DynamicMessage.parseFrom(descriptor, ByteString.copyFrom(input)); Map allFields = message.getAllFields(); for (Map.Entry entry : allFields.entrySet()) { - String name = entry.getKey().getName().toLowerCase(); + String name = entry.getKey().getName(); if (theMap.containsKey(name)) { continue; diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java index 1905e5a5ded..14a56a43f97 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java @@ -19,49 +19,94 @@ package com.metamx.druid.indexer.data; +import java.nio.ByteBuffer; +import java.nio.CharBuffer; +import java.nio.charset.CoderResult; +import java.nio.charset.CodingErrorAction; +import java.util.List; +import java.util.Map; + import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.base.Charsets; import com.metamx.common.exception.FormattedException; import com.metamx.common.parsers.Parser; import com.metamx.common.parsers.ToLowerCaseParser; import com.metamx.druid.input.InputRow; -import java.util.List; -import java.util.Map; - /** */ -public class StringInputRowParser implements InputRowParser +public class StringInputRowParser implements InputRowParser { - private final InputRowParser> inputRowCreator; - private final Parser parser; + private final InputRowParser> inputRowCreator; + private final Parser parser; - @JsonCreator - public StringInputRowParser( - @JsonProperty("timestampSpec") TimestampSpec timestampSpec, - @JsonProperty("data") DataSpec dataSpec, - @JsonProperty("dimensionExclusions") List dimensionExclusions - ) - { - this.inputRowCreator = new MapInputRowParser(timestampSpec, dataSpec, dimensionExclusions); - this.parser = new ToLowerCaseParser(dataSpec.getParser()); - } + private CharBuffer chars = null; - public void addDimensionExclusion(String dimension) - { - inputRowCreator.addDimensionExclusion(dimension); - } + @JsonCreator + public StringInputRowParser( + @JsonProperty("timestampSpec") TimestampSpec timestampSpec, + @JsonProperty("data") DataSpec dataSpec, + @JsonProperty("dimensionExclusions") List dimensionExclusions) + { + this.inputRowCreator = new MapInputRowParser(timestampSpec, dataSpec.getDimensions(), dimensionExclusions); + this.parser = new ToLowerCaseParser(dataSpec.getParser()); + } - @Override - public InputRow parse(String input) throws FormattedException - { - return inputRowCreator.parse(parser.parse(input)); - } + public void addDimensionExclusion(String dimension) + { + inputRowCreator.addDimensionExclusion(dimension); + } - @JsonValue - public InputRowParser> getInputRowCreator() - { - return inputRowCreator; - } + @Override + public InputRow parse(ByteBuffer input) throws FormattedException + { + + Map theMap = buildStringKeyMap(input); + + return inputRowCreator.parse(theMap); + } + + private Map buildStringKeyMap(ByteBuffer input) + { + int payloadSize = input.limit(); + + if (chars == null || chars.remaining() < payloadSize) + { + chars = CharBuffer.allocate(payloadSize); + } + + final CoderResult coderResult = Charsets.UTF_8.newDecoder() + .onMalformedInput(CodingErrorAction.REPLACE) + .onUnmappableCharacter(CodingErrorAction.REPLACE) + .decode(input, chars, true); + + Map theMap; + if (coderResult.isUnderflow()) + { + chars.flip(); + try + { + theMap = parser.parse(chars.toString()); + } finally + { + chars.clear(); + } + } + else + { + throw new FormattedException.Builder() + .withErrorCode(FormattedException.ErrorCode.UNPARSABLE_ROW) + .withMessage(String.format("Failed with CoderResult[%s]", coderResult)) + .build(); + } + return theMap; + } + + @JsonValue + public InputRowParser> getInputRowCreator() + { + return inputRowCreator; + } } diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java index f17d9cf9749..bb0a6021f6e 100644 --- a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java @@ -4,6 +4,7 @@ import static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent import static org.junit.Assert.assertEquals; import java.io.ByteArrayOutputStream; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; @@ -37,10 +38,9 @@ public class ProtoBufInputRowParserTest { @Test public void testParse() throws Exception { - //configure pares with desc file + //configure parser with desc file ProtoBufInputRowParser parser = new ProtoBufInputRowParser(new TimestampSpec("timestamp", "iso"), - new ProtoBufDataSpec("prototest.desc", Arrays.asList(DIMENSIONS), null), - Arrays.asList()); + Arrays.asList(DIMENSIONS), Arrays.asList(), "prototest.desc"); //create binary of proto test event @@ -60,7 +60,7 @@ public class ProtoBufInputRowParserTest { ByteArrayOutputStream out = new ByteArrayOutputStream(); event.writeTo(out); - InputRow row = parser.parse(ByteString.copyFrom(out.toByteArray())); + InputRow row = parser.parse(ByteBuffer.wrap(out.toByteArray())); System.out.println(row); assertEquals(Arrays.asList(DIMENSIONS), row.getDimensions()); assertEquals(dateTime.getMillis(), row.getTimestampFromEpoch()); diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/StringInputRowParserTest.java b/indexing-common/src/test/java/com/metamx/druid/indexer/data/StringInputRowParserTest.java new file mode 100644 index 00000000000..a3f072b5a0e --- /dev/null +++ b/indexing-common/src/test/java/com/metamx/druid/indexer/data/StringInputRowParserTest.java @@ -0,0 +1,23 @@ +package com.metamx.druid.indexer.data; + +import junit.framework.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; + +/** + * @author jan.rudert + */ +public class StringInputRowParserTest { + + + + @Test + public void testPayloadSize() { + ByteBuffer payload = ByteBuffer.allocate(10); + payload.position(2); + payload.limit(5); + payload.rewind(); + Assert.assertEquals(5, payload.limit()); + } +} diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java index 651cb757023..1e71d8125fb 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java @@ -9,6 +9,7 @@ import org.apache.hadoop.mapreduce.Mapper; import org.joda.time.DateTime; import java.io.IOException; +import java.nio.ByteBuffer; public abstract class HadoopDruidIndexerMapper extends Mapper { @@ -41,7 +42,7 @@ public abstract class HadoopDruidIndexerMapper extends Mapper< try { final InputRow inputRow; try { - inputRow = parser.parse(value.toString()); + inputRow = parser.parse(ByteBuffer.wrap(value.getBytes())); } catch (IllegalArgumentException e) { if (config.isIgnoreInvalidRows()) { diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java index 583cafcae4f..08ae59fa670 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java @@ -280,7 +280,7 @@ public class IndexGeneratorJob implements Jobby for (final Text value : values) { context.progress(); - final InputRow inputRow = index.getSpatialDimensionRowFormatter().formatRow(parser.parse(value.toString())); + final InputRow inputRow = index.getSpatialDimensionRowFormatter().formatRow(parser.parse(ByteBuffer.wrap(value.getBytes()))); allDimensionNames.addAll(inputRow.getDimensions()); int numRows = index.add(inputRow); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/StaticS3FirehoseFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/StaticS3FirehoseFactory.java index b7289c017b8..a15c8c0e86d 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/StaticS3FirehoseFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/StaticS3FirehoseFactory.java @@ -44,6 +44,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; import java.net.URI; +import java.nio.ByteBuffer; import java.util.List; import java.util.NoSuchElementException; import java.util.Queue; @@ -186,7 +187,7 @@ public class StaticS3FirehoseFactory implements FirehoseFactory throw new NoSuchElementException(); } - return parser.parse(lineIterator.next()); + return parser.parse(ByteBuffer.wrap(lineIterator.next().getBytes())); } @Override diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java index 514cb1025b2..a5babf091bf 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java @@ -21,15 +21,11 @@ package com.metamx.druid.realtime.firehose; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.CharBuffer; -import java.nio.charset.CoderResult; -import java.nio.charset.CodingErrorAction; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Properties; -import com.google.protobuf.ByteString; import kafka.consumer.Consumer; import kafka.consumer.ConsumerConfig; import kafka.consumer.KafkaStream; @@ -39,13 +35,10 @@ import kafka.message.MessageAndMetadata; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; import com.metamx.common.exception.FormattedException; import com.metamx.common.logger.Logger; import com.metamx.druid.indexer.data.InputRowParser; -import com.metamx.druid.indexer.data.ProtoBufInputRowParser; -import com.metamx.druid.indexer.data.StringInputRowParser; import com.metamx.druid.input.InputRow; /** @@ -61,13 +54,13 @@ public class KafkaFirehoseFactory implements FirehoseFactory private final String feed; @JsonProperty - private final InputRowParser parser; + private final InputRowParser parser; @JsonCreator public KafkaFirehoseFactory( @JsonProperty("consumerProps") Properties consumerProps, @JsonProperty("feed") String feed, - @JsonProperty("parser") InputRowParser parser) + @JsonProperty("parser") InputRowParser parser) { this.consumerProps = consumerProps; this.feed = feed; @@ -91,26 +84,20 @@ public class KafkaFirehoseFactory implements FirehoseFactory final KafkaStream stream = streamList.get(0); - if (parser instanceof StringInputRowParser) - { - - return new StringMessageFirehose(connector, stream); - } else if (parser instanceof ProtoBufInputRowParser) - { - return new ProtoBufMessageFirehose(stream, connector); - } - throw new RuntimeException("No Firehose for parser: " + parser.getClass().getName()); + return new DefaultFirehose(connector, stream, parser); } - private abstract static class AbstractKafkaFirehose implements Firehose + private static class DefaultFirehose implements Firehose { - protected final ConsumerConnector connector; - protected final Iterator> iter; + private final ConsumerConnector connector; + private final Iterator> iter; + private final InputRowParser parser; - public AbstractKafkaFirehose(ConsumerConnector connector, KafkaStream stream) + public DefaultFirehose(ConsumerConnector connector, KafkaStream stream, InputRowParser parser) { iter = stream.iterator(); this.connector = connector; + this.parser = parser; } @Override @@ -132,7 +119,10 @@ public class KafkaFirehoseFactory implements FirehoseFactory return parseMessage(message); } - protected abstract InputRow parseMessage(Message message); + public InputRow parseMessage(Message message) throws FormattedException + { + return parser.parse(message.payload()); + } @Override public Runnable commit() @@ -162,67 +152,4 @@ public class KafkaFirehoseFactory implements FirehoseFactory connector.shutdown(); } } - - private class StringMessageFirehose extends AbstractKafkaFirehose - { - - private CharBuffer chars = null; - - public StringMessageFirehose(ConsumerConnector connector, KafkaStream stream) - { - super(connector, stream); - } - - @Override - public InputRow parseMessage(Message message) throws FormattedException - { - - int payloadSize = message.payloadSize(); - if (chars == null || chars.remaining() < payloadSize) - { - chars = CharBuffer.allocate(payloadSize); - } - - final CoderResult coderResult = Charsets.UTF_8.newDecoder() - .onMalformedInput(CodingErrorAction.REPLACE) - .onUnmappableCharacter(CodingErrorAction.REPLACE) - .decode(message.payload(), chars, true); - - if (coderResult.isUnderflow()) - { - chars.flip(); - try - { - return parser.parse(chars.toString()); - } finally - { - chars.clear(); - } - } - else - { - throw new FormattedException.Builder() - .withErrorCode(FormattedException.ErrorCode.UNPARSABLE_ROW) - .withMessage(String.format("Failed with CoderResult[%s]", coderResult)) - .build(); - } - } - - } - - private class ProtoBufMessageFirehose extends AbstractKafkaFirehose - { - - public ProtoBufMessageFirehose(KafkaStream stream, ConsumerConnector connector) - { - super(connector, stream); - } - - @Override - public InputRow parseMessage(Message message) throws FormattedException - { - return parser.parse(ByteString.copyFrom(message.payload())); - } - - } } diff --git a/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java b/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java index df5761681af..9c56f5cfdd0 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java +++ b/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java @@ -43,6 +43,7 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; import java.net.URL; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Map; import java.util.concurrent.atomic.AtomicLong; @@ -176,7 +177,7 @@ public class TestIndex runOnce = true; } - retVal.add(parser.parse(line)); + retVal.add(parser.parse(ByteBuffer.wrap(line.getBytes()))); ++lineCount; return true; From bba83ec53202e040b416eb92b1e4c3f3ca17dc6b Mon Sep 17 00:00:00 2001 From: Jan Rudert Date: Mon, 29 Jul 2013 11:59:08 +0200 Subject: [PATCH 04/13] introduced a ByteBufferInputRowParser to keep the old StringInputRowParser in places outside the KafkaFireHose --- .../flights/FlightsFirehoseFactory.java | 3 +-- .../data/ByteBufferInputRowParser.java | 17 +++++++++++++ .../indexer/data/ProtoBufInputRowParser.java | 2 +- .../indexer/data/StringInputRowParser.java | 24 ++++++++++++++----- .../indexer/HadoopDruidIndexerMapper.java | 3 +-- .../druid/indexer/IndexGeneratorJob.java | 2 +- .../common/index/StaticS3FirehoseFactory.java | 3 +-- .../firehose/KafkaFirehoseFactory.java | 5 ++-- .../com/metamx/druid/index/v1/TestIndex.java | 3 +-- 9 files changed, 44 insertions(+), 18 deletions(-) create mode 100644 indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java diff --git a/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java b/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java index 7413d56eeea..66b8de4450c 100644 --- a/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java +++ b/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java @@ -36,7 +36,6 @@ import java.io.FileInputStream; import java.io.FileReader; import java.io.IOException; import java.io.InputStreamReader; -import java.nio.ByteBuffer; import java.util.Iterator; import java.util.zip.GZIPInputStream; @@ -119,7 +118,7 @@ public class FlightsFirehoseFactory implements FirehoseFactory @Override public InputRow nextRow() { - final InputRow retVal = parser.parse(ByteBuffer.wrap(line.getBytes())); + final InputRow retVal = parser.parse(line); line = null; return retVal; } diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java new file mode 100644 index 00000000000..f785e3f3a12 --- /dev/null +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java @@ -0,0 +1,17 @@ +package com.metamx.druid.indexer.data; + +import java.nio.ByteBuffer; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +/** + * @author jan.rudert + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringInputRowParser.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "protobuf", value = ProtoBufInputRowParser.class), + @JsonSubTypes.Type(name = "string", value = StringInputRowParser.class) +}) +public interface ByteBufferInputRowParser extends InputRowParser { +} diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java index a96e0abc8ed..89c6eca4a3c 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java @@ -22,7 +22,7 @@ import com.metamx.druid.input.InputRow; /** * @author jan.rudert */ -public class ProtoBufInputRowParser implements InputRowParser +public class ProtoBufInputRowParser implements ByteBufferInputRowParser { private final MapInputRowParser inputRowCreator; diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java index 14a56a43f97..b3942d67d27 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java @@ -37,7 +37,7 @@ import com.metamx.druid.input.InputRow; /** */ -public class StringInputRowParser implements InputRowParser +public class StringInputRowParser implements ByteBufferInputRowParser { private final InputRowParser> inputRowCreator; private final Parser parser; @@ -62,10 +62,7 @@ public class StringInputRowParser implements InputRowParser @Override public InputRow parse(ByteBuffer input) throws FormattedException { - - Map theMap = buildStringKeyMap(input); - - return inputRowCreator.parse(theMap); + return parseMap(buildStringKeyMap(input)); } private Map buildStringKeyMap(ByteBuffer input) @@ -88,7 +85,7 @@ public class StringInputRowParser implements InputRowParser chars.flip(); try { - theMap = parser.parse(chars.toString()); + theMap = parseString(chars.toString()); } finally { chars.clear(); @@ -104,6 +101,21 @@ public class StringInputRowParser implements InputRowParser return theMap; } + private Map parseString(String inputString) + { + return parser.parse(inputString); + } + + public InputRow parse(String input) throws FormattedException + { + return parseMap(parseString(input)); + } + + private InputRow parseMap(Map theMap) + { + return inputRowCreator.parse(theMap); + } + @JsonValue public InputRowParser> getInputRowCreator() { diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java index 1e71d8125fb..651cb757023 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java @@ -9,7 +9,6 @@ import org.apache.hadoop.mapreduce.Mapper; import org.joda.time.DateTime; import java.io.IOException; -import java.nio.ByteBuffer; public abstract class HadoopDruidIndexerMapper extends Mapper { @@ -42,7 +41,7 @@ public abstract class HadoopDruidIndexerMapper extends Mapper< try { final InputRow inputRow; try { - inputRow = parser.parse(ByteBuffer.wrap(value.getBytes())); + inputRow = parser.parse(value.toString()); } catch (IllegalArgumentException e) { if (config.isIgnoreInvalidRows()) { diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java index 08ae59fa670..583cafcae4f 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java @@ -280,7 +280,7 @@ public class IndexGeneratorJob implements Jobby for (final Text value : values) { context.progress(); - final InputRow inputRow = index.getSpatialDimensionRowFormatter().formatRow(parser.parse(ByteBuffer.wrap(value.getBytes()))); + final InputRow inputRow = index.getSpatialDimensionRowFormatter().formatRow(parser.parse(value.toString())); allDimensionNames.addAll(inputRow.getDimensions()); int numRows = index.add(inputRow); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/StaticS3FirehoseFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/StaticS3FirehoseFactory.java index a15c8c0e86d..b7289c017b8 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/StaticS3FirehoseFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/StaticS3FirehoseFactory.java @@ -44,7 +44,6 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; import java.net.URI; -import java.nio.ByteBuffer; import java.util.List; import java.util.NoSuchElementException; import java.util.Queue; @@ -187,7 +186,7 @@ public class StaticS3FirehoseFactory implements FirehoseFactory throw new NoSuchElementException(); } - return parser.parse(ByteBuffer.wrap(lineIterator.next().getBytes())); + return parser.parse(lineIterator.next()); } @Override diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java index a5babf091bf..01debf2a0a2 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import com.metamx.druid.indexer.data.ByteBufferInputRowParser; import kafka.consumer.Consumer; import kafka.consumer.ConsumerConfig; import kafka.consumer.KafkaStream; @@ -54,13 +55,13 @@ public class KafkaFirehoseFactory implements FirehoseFactory private final String feed; @JsonProperty - private final InputRowParser parser; + private final ByteBufferInputRowParser parser; @JsonCreator public KafkaFirehoseFactory( @JsonProperty("consumerProps") Properties consumerProps, @JsonProperty("feed") String feed, - @JsonProperty("parser") InputRowParser parser) + @JsonProperty("parser") ByteBufferInputRowParser parser) { this.consumerProps = consumerProps; this.feed = feed; diff --git a/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java b/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java index 9c56f5cfdd0..df5761681af 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java +++ b/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java @@ -43,7 +43,6 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; import java.net.URL; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Map; import java.util.concurrent.atomic.AtomicLong; @@ -177,7 +176,7 @@ public class TestIndex runOnce = true; } - retVal.add(parser.parse(ByteBuffer.wrap(line.getBytes()))); + retVal.add(parser.parse(line)); ++lineCount; return true; From 31c048ccb681672a8210e6466c5ebe92eec8d16a Mon Sep 17 00:00:00 2001 From: Jan Rudert Date: Thu, 1 Aug 2013 17:27:33 +0200 Subject: [PATCH 05/13] use hadoop cdh4-4.1.1 and protobuf 2.4.0 --- build.sh | 3 +- indexing-common/pom.xml | 1 - .../indexer/data/ProtoTestEventWrapper.java | 1669 +++++++---------- indexing-hadoop/pom.xml | 6 +- pom.xml | 21 +- server/pom.xml | 7 +- 6 files changed, 695 insertions(+), 1012 deletions(-) diff --git a/build.sh b/build.sh index fe534e319f7..796e958cd9d 100755 --- a/build.sh +++ b/build.sh @@ -16,7 +16,8 @@ VERSION=`cat pom.xml | grep version | head -4 | tail -1 | sed 's_.*\([^ echo Using Version[${VERSION}] mvn clean -mvn package +#mvn install -U +mvn install -U -DskipTests if [ $? -ne "0" ]; then echo "mvn package failed" diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 2249e107f10..f16b7fb1ef9 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -82,7 +82,6 @@ com.google.protobuf protobuf-java - 2.5.0 diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoTestEventWrapper.java b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoTestEventWrapper.java index 307d8a93a9c..9c86246398a 100644 --- a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoTestEventWrapper.java +++ b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoTestEventWrapper.java @@ -10,155 +10,684 @@ public final class ProtoTestEventWrapper { } public interface ProtoTestEventOrBuilder extends com.google.protobuf.MessageOrBuilder { - + // required .prototest.ProtoTestEvent.EventCategory eventType = 1; - /** - * required .prototest.ProtoTestEvent.EventCategory eventType = 1; - */ boolean hasEventType(); - /** - * required .prototest.ProtoTestEvent.EventCategory eventType = 1; - */ com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType(); - + // required uint64 id = 2; - /** - * required uint64 id = 2; - */ boolean hasId(); - /** - * required uint64 id = 2; - */ long getId(); - + // required string timestamp = 3; - /** - * required string timestamp = 3; - */ boolean hasTimestamp(); - /** - * required string timestamp = 3; - */ - java.lang.String getTimestamp(); - /** - * required string timestamp = 3; - */ - com.google.protobuf.ByteString - getTimestampBytes(); - + String getTimestamp(); + // optional uint32 someOtherId = 4; - /** - * optional uint32 someOtherId = 4; - */ boolean hasSomeOtherId(); - /** - * optional uint32 someOtherId = 4; - */ int getSomeOtherId(); - + // optional bool isValid = 5; - /** - * optional bool isValid = 5; - */ boolean hasIsValid(); - /** - * optional bool isValid = 5; - */ boolean getIsValid(); - + // optional string description = 6; - /** - * optional string description = 6; - */ boolean hasDescription(); - /** - * optional string description = 6; - */ - java.lang.String getDescription(); - /** - * optional string description = 6; - */ - com.google.protobuf.ByteString - getDescriptionBytes(); - + String getDescription(); + // optional float someFloatColumn = 7; - /** - * optional float someFloatColumn = 7; - */ boolean hasSomeFloatColumn(); - /** - * optional float someFloatColumn = 7; - */ float getSomeFloatColumn(); - + // optional uint32 someIntColumn = 8; - /** - * optional uint32 someIntColumn = 8; - */ boolean hasSomeIntColumn(); - /** - * optional uint32 someIntColumn = 8; - */ int getSomeIntColumn(); - + // optional uint64 someLongColumn = 9; - /** - * optional uint64 someLongColumn = 9; - */ boolean hasSomeLongColumn(); - /** - * optional uint64 someLongColumn = 9; - */ long getSomeLongColumn(); } - /** - * Protobuf type {@code prototest.ProtoTestEvent} - */ public static final class ProtoTestEvent extends com.google.protobuf.GeneratedMessage implements ProtoTestEventOrBuilder { // Use ProtoTestEvent.newBuilder() to construct. - private ProtoTestEvent(com.google.protobuf.GeneratedMessage.Builder builder) { + private ProtoTestEvent(Builder builder) { super(builder); - this.unknownFields = builder.getUnknownFields(); } - private ProtoTestEvent(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } - + private ProtoTestEvent(boolean noInit) {} + private static final ProtoTestEvent defaultInstance; public static ProtoTestEvent getDefaultInstance() { return defaultInstance; } - + public ProtoTestEvent getDefaultInstanceForType() { return defaultInstance; } - - private final com.google.protobuf.UnknownFieldSet unknownFields; - @java.lang.Override - public final com.google.protobuf.UnknownFieldSet - getUnknownFields() { - return this.unknownFields; + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; } - private ProtoTestEvent( - com.google.protobuf.CodedInputStream input, + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable; + } + + public enum EventCategory + implements com.google.protobuf.ProtocolMessageEnum { + CATEGORY_ZERO(0, 0), + CATEGORY_ONE(1, 1), + CATEGORY_TWO(2, 2), + ; + + public static final int CATEGORY_ZERO_VALUE = 0; + public static final int CATEGORY_ONE_VALUE = 1; + public static final int CATEGORY_TWO_VALUE = 2; + + + public final int getNumber() { return value; } + + public static EventCategory valueOf(int value) { + switch (value) { + case 0: return CATEGORY_ZERO; + case 1: return CATEGORY_ONE; + case 2: return CATEGORY_TWO; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static com.google.protobuf.Internal.EnumLiteMap + internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public EventCategory findValueByNumber(int number) { + return EventCategory.valueOf(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(index); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDescriptor().getEnumTypes().get(0); + } + + private static final EventCategory[] VALUES = { + CATEGORY_ZERO, CATEGORY_ONE, CATEGORY_TWO, + }; + + public static EventCategory valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int index; + private final int value; + + private EventCategory(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:prototest.ProtoTestEvent.EventCategory) + } + + private int bitField0_; + // required .prototest.ProtoTestEvent.EventCategory eventType = 1; + public static final int EVENTTYPE_FIELD_NUMBER = 1; + private com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_; + public boolean hasEventType() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() { + return eventType_; + } + + // required uint64 id = 2; + public static final int ID_FIELD_NUMBER = 2; + private long id_; + public boolean hasId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getId() { + return id_; + } + + // required string timestamp = 3; + public static final int TIMESTAMP_FIELD_NUMBER = 3; + private java.lang.Object timestamp_; + public boolean hasTimestamp() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getTimestamp() { + java.lang.Object ref = timestamp_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + timestamp_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getTimestampBytes() { + java.lang.Object ref = timestamp_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + timestamp_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // optional uint32 someOtherId = 4; + public static final int SOMEOTHERID_FIELD_NUMBER = 4; + private int someOtherId_; + public boolean hasSomeOtherId() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public int getSomeOtherId() { + return someOtherId_; + } + + // optional bool isValid = 5; + public static final int ISVALID_FIELD_NUMBER = 5; + private boolean isValid_; + public boolean hasIsValid() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public boolean getIsValid() { + return isValid_; + } + + // optional string description = 6; + public static final int DESCRIPTION_FIELD_NUMBER = 6; + private java.lang.Object description_; + public boolean hasDescription() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public String getDescription() { + java.lang.Object ref = description_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + description_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getDescriptionBytes() { + java.lang.Object ref = description_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + description_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // optional float someFloatColumn = 7; + public static final int SOMEFLOATCOLUMN_FIELD_NUMBER = 7; + private float someFloatColumn_; + public boolean hasSomeFloatColumn() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public float getSomeFloatColumn() { + return someFloatColumn_; + } + + // optional uint32 someIntColumn = 8; + public static final int SOMEINTCOLUMN_FIELD_NUMBER = 8; + private int someIntColumn_; + public boolean hasSomeIntColumn() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public int getSomeIntColumn() { + return someIntColumn_; + } + + // optional uint64 someLongColumn = 9; + public static final int SOMELONGCOLUMN_FIELD_NUMBER = 9; + private long someLongColumn_; + public boolean hasSomeLongColumn() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public long getSomeLongColumn() { + return someLongColumn_; + } + + private void initFields() { + eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; + id_ = 0L; + timestamp_ = ""; + someOtherId_ = 0; + isValid_ = false; + description_ = ""; + someFloatColumn_ = 0F; + someIntColumn_ = 0; + someLongColumn_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasEventType()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasTimestamp()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeEnum(1, eventType_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, id_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getTimestampBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt32(4, someOtherId_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBool(5, isValid_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeBytes(6, getDescriptionBytes()); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeFloat(7, someFloatColumn_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeUInt32(8, someIntColumn_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeUInt64(9, someLongColumn_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(1, eventType_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(2, id_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, getTimestampBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(4, someOtherId_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(5, isValid_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(6, getDescriptionBytes()); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += com.google.protobuf.CodedOutputStream + .computeFloatSize(7, someFloatColumn_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(8, someIntColumn_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(9, someLongColumn_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { - initFields(); - int mutable_bitField0_ = 0; - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder(); - try { - boolean done = false; - while (!done) { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEventOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable; + } + + // Construct using com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; + bitField0_ = (bitField0_ & ~0x00000001); + id_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + timestamp_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + someOtherId_ = 0; + bitField0_ = (bitField0_ & ~0x00000008); + isValid_ = false; + bitField0_ = (bitField0_ & ~0x00000010); + description_ = ""; + bitField0_ = (bitField0_ & ~0x00000020); + someFloatColumn_ = 0F; + bitField0_ = (bitField0_ & ~0x00000040); + someIntColumn_ = 0; + bitField0_ = (bitField0_ & ~0x00000080); + someLongColumn_ = 0L; + bitField0_ = (bitField0_ & ~0x00000100); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDescriptor(); + } + + public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType() { + return com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance(); + } + + public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent build() { + com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent buildPartial() { + com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent result = new com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.eventType_ = eventType_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.id_ = id_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.timestamp_ = timestamp_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.someOtherId_ = someOtherId_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.isValid_ = isValid_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.description_ = description_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000040; + } + result.someFloatColumn_ = someFloatColumn_; + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000080; + } + result.someIntColumn_ = someIntColumn_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000100; + } + result.someLongColumn_ = someLongColumn_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent) { + return mergeFrom((com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent other) { + if (other == com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance()) return this; + if (other.hasEventType()) { + setEventType(other.getEventType()); + } + if (other.hasId()) { + setId(other.getId()); + } + if (other.hasTimestamp()) { + setTimestamp(other.getTimestamp()); + } + if (other.hasSomeOtherId()) { + setSomeOtherId(other.getSomeOtherId()); + } + if (other.hasIsValid()) { + setIsValid(other.getIsValid()); + } + if (other.hasDescription()) { + setDescription(other.getDescription()); + } + if (other.hasSomeFloatColumn()) { + setSomeFloatColumn(other.getSomeFloatColumn()); + } + if (other.hasSomeIntColumn()) { + setSomeIntColumn(other.getSomeIntColumn()); + } + if (other.hasSomeLongColumn()) { + setSomeLongColumn(other.getSomeLongColumn()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasEventType()) { + + return false; + } + if (!hasId()) { + + return false; + } + if (!hasTimestamp()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { int tag = input.readTag(); switch (tag) { case 0: - done = true; - break; + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { - done = true; + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; } break; } @@ -215,741 +744,18 @@ public final class ProtoTestEventWrapper { } } } - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - throw e.setUnfinishedMessage(this); - } catch (java.io.IOException e) { - throw new com.google.protobuf.InvalidProtocolBufferException( - e.getMessage()).setUnfinishedMessage(this); - } finally { - this.unknownFields = unknownFields.build(); - makeExtensionsImmutable(); - } - } - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.class, com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.Builder.class); - } - - public static com.google.protobuf.Parser PARSER = - new com.google.protobuf.AbstractParser() { - public ProtoTestEvent parsePartialFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new ProtoTestEvent(input, extensionRegistry); - } - }; - - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } - - /** - * Protobuf enum {@code prototest.ProtoTestEvent.EventCategory} - */ - public enum EventCategory - implements com.google.protobuf.ProtocolMessageEnum { - /** - * CATEGORY_ZERO = 0; - */ - CATEGORY_ZERO(0, 0), - /** - * CATEGORY_ONE = 1; - */ - CATEGORY_ONE(1, 1), - /** - * CATEGORY_TWO = 2; - */ - CATEGORY_TWO(2, 2), - ; - - /** - * CATEGORY_ZERO = 0; - */ - public static final int CATEGORY_ZERO_VALUE = 0; - /** - * CATEGORY_ONE = 1; - */ - public static final int CATEGORY_ONE_VALUE = 1; - /** - * CATEGORY_TWO = 2; - */ - public static final int CATEGORY_TWO_VALUE = 2; - - - public final int getNumber() { return value; } - - public static EventCategory valueOf(int value) { - switch (value) { - case 0: return CATEGORY_ZERO; - case 1: return CATEGORY_ONE; - case 2: return CATEGORY_TWO; - default: return null; - } - } - - public static com.google.protobuf.Internal.EnumLiteMap - internalGetValueMap() { - return internalValueMap; - } - private static com.google.protobuf.Internal.EnumLiteMap - internalValueMap = - new com.google.protobuf.Internal.EnumLiteMap() { - public EventCategory findValueByNumber(int number) { - return EventCategory.valueOf(number); - } - }; - - public final com.google.protobuf.Descriptors.EnumValueDescriptor - getValueDescriptor() { - return getDescriptor().getValues().get(index); - } - public final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptorForType() { - return getDescriptor(); - } - public static final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptor() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDescriptor().getEnumTypes().get(0); - } - - private static final EventCategory[] VALUES = values(); - - public static EventCategory valueOf( - com.google.protobuf.Descriptors.EnumValueDescriptor desc) { - if (desc.getType() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "EnumValueDescriptor is not for this type."); - } - return VALUES[desc.getIndex()]; - } - - private final int index; - private final int value; - - private EventCategory(int index, int value) { - this.index = index; - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:prototest.ProtoTestEvent.EventCategory) - } - - private int bitField0_; - // required .prototest.ProtoTestEvent.EventCategory eventType = 1; - public static final int EVENTTYPE_FIELD_NUMBER = 1; - private com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_; - /** - * required .prototest.ProtoTestEvent.EventCategory eventType = 1; - */ - public boolean hasEventType() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - /** - * required .prototest.ProtoTestEvent.EventCategory eventType = 1; - */ - public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() { - return eventType_; - } - - // required uint64 id = 2; - public static final int ID_FIELD_NUMBER = 2; - private long id_; - /** - * required uint64 id = 2; - */ - public boolean hasId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - /** - * required uint64 id = 2; - */ - public long getId() { - return id_; - } - - // required string timestamp = 3; - public static final int TIMESTAMP_FIELD_NUMBER = 3; - private java.lang.Object timestamp_; - /** - * required string timestamp = 3; - */ - public boolean hasTimestamp() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - /** - * required string timestamp = 3; - */ - public java.lang.String getTimestamp() { - java.lang.Object ref = timestamp_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - if (bs.isValidUtf8()) { - timestamp_ = s; - } - return s; - } - } - /** - * required string timestamp = 3; - */ - public com.google.protobuf.ByteString - getTimestampBytes() { - java.lang.Object ref = timestamp_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - timestamp_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - // optional uint32 someOtherId = 4; - public static final int SOMEOTHERID_FIELD_NUMBER = 4; - private int someOtherId_; - /** - * optional uint32 someOtherId = 4; - */ - public boolean hasSomeOtherId() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - /** - * optional uint32 someOtherId = 4; - */ - public int getSomeOtherId() { - return someOtherId_; - } - - // optional bool isValid = 5; - public static final int ISVALID_FIELD_NUMBER = 5; - private boolean isValid_; - /** - * optional bool isValid = 5; - */ - public boolean hasIsValid() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - /** - * optional bool isValid = 5; - */ - public boolean getIsValid() { - return isValid_; - } - - // optional string description = 6; - public static final int DESCRIPTION_FIELD_NUMBER = 6; - private java.lang.Object description_; - /** - * optional string description = 6; - */ - public boolean hasDescription() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - /** - * optional string description = 6; - */ - public java.lang.String getDescription() { - java.lang.Object ref = description_; - if (ref instanceof java.lang.String) { - return (java.lang.String) ref; - } else { - com.google.protobuf.ByteString bs = - (com.google.protobuf.ByteString) ref; - java.lang.String s = bs.toStringUtf8(); - if (bs.isValidUtf8()) { - description_ = s; - } - return s; - } - } - /** - * optional string description = 6; - */ - public com.google.protobuf.ByteString - getDescriptionBytes() { - java.lang.Object ref = description_; - if (ref instanceof java.lang.String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - description_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - - // optional float someFloatColumn = 7; - public static final int SOMEFLOATCOLUMN_FIELD_NUMBER = 7; - private float someFloatColumn_; - /** - * optional float someFloatColumn = 7; - */ - public boolean hasSomeFloatColumn() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - /** - * optional float someFloatColumn = 7; - */ - public float getSomeFloatColumn() { - return someFloatColumn_; - } - - // optional uint32 someIntColumn = 8; - public static final int SOMEINTCOLUMN_FIELD_NUMBER = 8; - private int someIntColumn_; - /** - * optional uint32 someIntColumn = 8; - */ - public boolean hasSomeIntColumn() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - /** - * optional uint32 someIntColumn = 8; - */ - public int getSomeIntColumn() { - return someIntColumn_; - } - - // optional uint64 someLongColumn = 9; - public static final int SOMELONGCOLUMN_FIELD_NUMBER = 9; - private long someLongColumn_; - /** - * optional uint64 someLongColumn = 9; - */ - public boolean hasSomeLongColumn() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - /** - * optional uint64 someLongColumn = 9; - */ - public long getSomeLongColumn() { - return someLongColumn_; - } - - private void initFields() { - eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; - id_ = 0L; - timestamp_ = ""; - someOtherId_ = 0; - isValid_ = false; - description_ = ""; - someFloatColumn_ = 0F; - someIntColumn_ = 0; - someLongColumn_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasEventType()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasTimestamp()) { - memoizedIsInitialized = 0; - return false; - } - memoizedIsInitialized = 1; - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeEnum(1, eventType_.getNumber()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, id_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBytes(3, getTimestampBytes()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeUInt32(4, someOtherId_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBool(5, isValid_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - output.writeBytes(6, getDescriptionBytes()); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - output.writeFloat(7, someFloatColumn_); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - output.writeUInt32(8, someIntColumn_); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - output.writeUInt64(9, someLongColumn_); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (((bitField0_ & 0x00000001) == 0x00000001)) { - size += com.google.protobuf.CodedOutputStream - .computeEnumSize(1, eventType_.getNumber()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(2, id_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += com.google.protobuf.CodedOutputStream - .computeBytesSize(3, getTimestampBytes()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += com.google.protobuf.CodedOutputStream - .computeUInt32Size(4, someOtherId_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += com.google.protobuf.CodedOutputStream - .computeBoolSize(5, isValid_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - size += com.google.protobuf.CodedOutputStream - .computeBytesSize(6, getDescriptionBytes()); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - size += com.google.protobuf.CodedOutputStream - .computeFloatSize(7, someFloatColumn_); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - size += com.google.protobuf.CodedOutputStream - .computeUInt32Size(8, someIntColumn_); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(9, someLongColumn_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - private static final long serialVersionUID = 0L; - @java.lang.Override - protected java.lang.Object writeReplace() - throws java.io.ObjectStreamException { - return super.writeReplace(); - } - - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data); - } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return PARSER.parseFrom(data, extensionRegistry); - } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom(java.io.InputStream input) - throws java.io.IOException { - return PARSER.parseFrom(input); - } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return PARSER.parseFrom(input, extensionRegistry); - } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - return PARSER.parseDelimitedFrom(input); - } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return PARSER.parseDelimitedFrom(input, extensionRegistry); - } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return PARSER.parseFrom(input); - } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return PARSER.parseFrom(input, extensionRegistry); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - /** - * Protobuf type {@code prototest.ProtoTestEvent} - */ - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder - implements com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEventOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable - .ensureFieldAccessorsInitialized( - com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.class, com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.Builder.class); - } - - // Construct using com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - } - } - private static Builder create() { - return new Builder(); - } - - public Builder clear() { - super.clear(); - eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; - bitField0_ = (bitField0_ & ~0x00000001); - id_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - timestamp_ = ""; - bitField0_ = (bitField0_ & ~0x00000004); - someOtherId_ = 0; - bitField0_ = (bitField0_ & ~0x00000008); - isValid_ = false; - bitField0_ = (bitField0_ & ~0x00000010); - description_ = ""; - bitField0_ = (bitField0_ & ~0x00000020); - someFloatColumn_ = 0F; - bitField0_ = (bitField0_ & ~0x00000040); - someIntColumn_ = 0; - bitField0_ = (bitField0_ & ~0x00000080); - someLongColumn_ = 0L; - bitField0_ = (bitField0_ & ~0x00000100); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; - } - - public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance(); - } - - public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent build() { - com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent buildPartial() { - com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent result = new com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent(this); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.eventType_ = eventType_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.id_ = id_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.timestamp_ = timestamp_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.someOtherId_ = someOtherId_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.isValid_ = isValid_; - if (((from_bitField0_ & 0x00000020) == 0x00000020)) { - to_bitField0_ |= 0x00000020; - } - result.description_ = description_; - if (((from_bitField0_ & 0x00000040) == 0x00000040)) { - to_bitField0_ |= 0x00000040; - } - result.someFloatColumn_ = someFloatColumn_; - if (((from_bitField0_ & 0x00000080) == 0x00000080)) { - to_bitField0_ |= 0x00000080; - } - result.someIntColumn_ = someIntColumn_; - if (((from_bitField0_ & 0x00000100) == 0x00000100)) { - to_bitField0_ |= 0x00000100; - } - result.someLongColumn_ = someLongColumn_; - result.bitField0_ = to_bitField0_; - onBuilt(); - return result; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent) { - return mergeFrom((com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent other) { - if (other == com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance()) return this; - if (other.hasEventType()) { - setEventType(other.getEventType()); - } - if (other.hasId()) { - setId(other.getId()); - } - if (other.hasTimestamp()) { - bitField0_ |= 0x00000004; - timestamp_ = other.timestamp_; - onChanged(); - } - if (other.hasSomeOtherId()) { - setSomeOtherId(other.getSomeOtherId()); - } - if (other.hasIsValid()) { - setIsValid(other.getIsValid()); - } - if (other.hasDescription()) { - bitField0_ |= 0x00000020; - description_ = other.description_; - onChanged(); - } - if (other.hasSomeFloatColumn()) { - setSomeFloatColumn(other.getSomeFloatColumn()); - } - if (other.hasSomeIntColumn()) { - setSomeIntColumn(other.getSomeIntColumn()); - } - if (other.hasSomeLongColumn()) { - setSomeLongColumn(other.getSomeLongColumn()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public final boolean isInitialized() { - if (!hasEventType()) { - - return false; - } - if (!hasId()) { - - return false; - } - if (!hasTimestamp()) { - - return false; - } - return true; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parsedMessage = null; - try { - parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); - } catch (com.google.protobuf.InvalidProtocolBufferException e) { - parsedMessage = (com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent) e.getUnfinishedMessage(); - throw e; - } finally { - if (parsedMessage != null) { - mergeFrom(parsedMessage); - } - } - return this; } + private int bitField0_; - + // required .prototest.ProtoTestEvent.EventCategory eventType = 1; private com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; - /** - * required .prototest.ProtoTestEvent.EventCategory eventType = 1; - */ public boolean hasEventType() { return ((bitField0_ & 0x00000001) == 0x00000001); } - /** - * required .prototest.ProtoTestEvent.EventCategory eventType = 1; - */ public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() { return eventType_; } - /** - * required .prototest.ProtoTestEvent.EventCategory eventType = 1; - */ public Builder setEventType(com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory value) { if (value == null) { throw new NullPointerException(); @@ -959,92 +765,50 @@ public final class ProtoTestEventWrapper { onChanged(); return this; } - /** - * required .prototest.ProtoTestEvent.EventCategory eventType = 1; - */ public Builder clearEventType() { bitField0_ = (bitField0_ & ~0x00000001); eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; onChanged(); return this; } - + // required uint64 id = 2; private long id_ ; - /** - * required uint64 id = 2; - */ public boolean hasId() { return ((bitField0_ & 0x00000002) == 0x00000002); } - /** - * required uint64 id = 2; - */ public long getId() { return id_; } - /** - * required uint64 id = 2; - */ public Builder setId(long value) { bitField0_ |= 0x00000002; id_ = value; onChanged(); return this; } - /** - * required uint64 id = 2; - */ public Builder clearId() { bitField0_ = (bitField0_ & ~0x00000002); id_ = 0L; onChanged(); return this; } - + // required string timestamp = 3; private java.lang.Object timestamp_ = ""; - /** - * required string timestamp = 3; - */ public boolean hasTimestamp() { return ((bitField0_ & 0x00000004) == 0x00000004); } - /** - * required string timestamp = 3; - */ - public java.lang.String getTimestamp() { + public String getTimestamp() { java.lang.Object ref = timestamp_; - if (!(ref instanceof java.lang.String)) { - java.lang.String s = ((com.google.protobuf.ByteString) ref) - .toStringUtf8(); + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); timestamp_ = s; return s; } else { - return (java.lang.String) ref; + return (String) ref; } } - /** - * required string timestamp = 3; - */ - public com.google.protobuf.ByteString - getTimestampBytes() { - java.lang.Object ref = timestamp_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - timestamp_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * required string timestamp = 3; - */ - public Builder setTimestamp( - java.lang.String value) { + public Builder setTimestamp(String value) { if (value == null) { throw new NullPointerException(); } @@ -1053,138 +817,76 @@ public final class ProtoTestEventWrapper { onChanged(); return this; } - /** - * required string timestamp = 3; - */ public Builder clearTimestamp() { bitField0_ = (bitField0_ & ~0x00000004); timestamp_ = getDefaultInstance().getTimestamp(); onChanged(); return this; } - /** - * required string timestamp = 3; - */ - public Builder setTimestampBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; + void setTimestamp(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000004; timestamp_ = value; onChanged(); - return this; } - + // optional uint32 someOtherId = 4; private int someOtherId_ ; - /** - * optional uint32 someOtherId = 4; - */ public boolean hasSomeOtherId() { return ((bitField0_ & 0x00000008) == 0x00000008); } - /** - * optional uint32 someOtherId = 4; - */ public int getSomeOtherId() { return someOtherId_; } - /** - * optional uint32 someOtherId = 4; - */ public Builder setSomeOtherId(int value) { bitField0_ |= 0x00000008; someOtherId_ = value; onChanged(); return this; } - /** - * optional uint32 someOtherId = 4; - */ public Builder clearSomeOtherId() { bitField0_ = (bitField0_ & ~0x00000008); someOtherId_ = 0; onChanged(); return this; } - + // optional bool isValid = 5; private boolean isValid_ ; - /** - * optional bool isValid = 5; - */ public boolean hasIsValid() { return ((bitField0_ & 0x00000010) == 0x00000010); } - /** - * optional bool isValid = 5; - */ public boolean getIsValid() { return isValid_; } - /** - * optional bool isValid = 5; - */ public Builder setIsValid(boolean value) { bitField0_ |= 0x00000010; isValid_ = value; onChanged(); return this; } - /** - * optional bool isValid = 5; - */ public Builder clearIsValid() { bitField0_ = (bitField0_ & ~0x00000010); isValid_ = false; onChanged(); return this; } - + // optional string description = 6; private java.lang.Object description_ = ""; - /** - * optional string description = 6; - */ public boolean hasDescription() { return ((bitField0_ & 0x00000020) == 0x00000020); } - /** - * optional string description = 6; - */ - public java.lang.String getDescription() { + public String getDescription() { java.lang.Object ref = description_; - if (!(ref instanceof java.lang.String)) { - java.lang.String s = ((com.google.protobuf.ByteString) ref) - .toStringUtf8(); + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); description_ = s; return s; } else { - return (java.lang.String) ref; + return (String) ref; } } - /** - * optional string description = 6; - */ - public com.google.protobuf.ByteString - getDescriptionBytes() { - java.lang.Object ref = description_; - if (ref instanceof String) { - com.google.protobuf.ByteString b = - com.google.protobuf.ByteString.copyFromUtf8( - (java.lang.String) ref); - description_ = b; - return b; - } else { - return (com.google.protobuf.ByteString) ref; - } - } - /** - * optional string description = 6; - */ - public Builder setDescription( - java.lang.String value) { + public Builder setDescription(String value) { if (value == null) { throw new NullPointerException(); } @@ -1193,145 +895,98 @@ public final class ProtoTestEventWrapper { onChanged(); return this; } - /** - * optional string description = 6; - */ public Builder clearDescription() { bitField0_ = (bitField0_ & ~0x00000020); description_ = getDefaultInstance().getDescription(); onChanged(); return this; } - /** - * optional string description = 6; - */ - public Builder setDescriptionBytes( - com.google.protobuf.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000020; + void setDescription(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000020; description_ = value; onChanged(); - return this; } - + // optional float someFloatColumn = 7; private float someFloatColumn_ ; - /** - * optional float someFloatColumn = 7; - */ public boolean hasSomeFloatColumn() { return ((bitField0_ & 0x00000040) == 0x00000040); } - /** - * optional float someFloatColumn = 7; - */ public float getSomeFloatColumn() { return someFloatColumn_; } - /** - * optional float someFloatColumn = 7; - */ public Builder setSomeFloatColumn(float value) { bitField0_ |= 0x00000040; someFloatColumn_ = value; onChanged(); return this; } - /** - * optional float someFloatColumn = 7; - */ public Builder clearSomeFloatColumn() { bitField0_ = (bitField0_ & ~0x00000040); someFloatColumn_ = 0F; onChanged(); return this; } - + // optional uint32 someIntColumn = 8; private int someIntColumn_ ; - /** - * optional uint32 someIntColumn = 8; - */ public boolean hasSomeIntColumn() { return ((bitField0_ & 0x00000080) == 0x00000080); } - /** - * optional uint32 someIntColumn = 8; - */ public int getSomeIntColumn() { return someIntColumn_; } - /** - * optional uint32 someIntColumn = 8; - */ public Builder setSomeIntColumn(int value) { bitField0_ |= 0x00000080; someIntColumn_ = value; onChanged(); return this; } - /** - * optional uint32 someIntColumn = 8; - */ public Builder clearSomeIntColumn() { bitField0_ = (bitField0_ & ~0x00000080); someIntColumn_ = 0; onChanged(); return this; } - + // optional uint64 someLongColumn = 9; private long someLongColumn_ ; - /** - * optional uint64 someLongColumn = 9; - */ public boolean hasSomeLongColumn() { return ((bitField0_ & 0x00000100) == 0x00000100); } - /** - * optional uint64 someLongColumn = 9; - */ public long getSomeLongColumn() { return someLongColumn_; } - /** - * optional uint64 someLongColumn = 9; - */ public Builder setSomeLongColumn(long value) { bitField0_ |= 0x00000100; someLongColumn_ = value; onChanged(); return this; } - /** - * optional uint64 someLongColumn = 9; - */ public Builder clearSomeLongColumn() { bitField0_ = (bitField0_ & ~0x00000100); someLongColumn_ = 0L; onChanged(); return this; } - + // @@protoc_insertion_point(builder_scope:prototest.ProtoTestEvent) } - + static { defaultInstance = new ProtoTestEvent(true); defaultInstance.initFields(); } - + // @@protoc_insertion_point(class_scope:prototest.ProtoTestEvent) } - + private static com.google.protobuf.Descriptors.Descriptor internal_static_prototest_ProtoTestEvent_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_prototest_ProtoTestEvent_fieldAccessorTable; - + public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { return descriptor; @@ -1361,7 +1016,9 @@ public final class ProtoTestEventWrapper { internal_static_prototest_ProtoTestEvent_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_prototest_ProtoTestEvent_descriptor, - new java.lang.String[] { "EventType", "Id", "Timestamp", "SomeOtherId", "IsValid", "Description", "SomeFloatColumn", "SomeIntColumn", "SomeLongColumn", }); + new java.lang.String[] { "EventType", "Id", "Timestamp", "SomeOtherId", "IsValid", "Description", "SomeFloatColumn", "SomeIntColumn", "SomeLongColumn", }, + com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.class, + com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.Builder.class); return null; } }; @@ -1370,6 +1027,6 @@ public final class ProtoTestEventWrapper { new com.google.protobuf.Descriptors.FileDescriptor[] { }, assigner); } - + // @@protoc_insertion_point(outer_class_scope) } diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 4f130f14fa8..c0dc442e3f7 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -77,7 +77,11 @@ org.apache.hadoop - hadoop-core + hadoop-client + + + org.apache.hadoop + hadoop-hdfs com.fasterxml.jackson.core diff --git a/pom.xml b/pom.xml index 9c3cc0f3231..4e2be406fd3 100644 --- a/pom.xml +++ b/pom.xml @@ -145,8 +145,19 @@ org.apache.hadoop - hadoop-core - 0.20.2 + hadoop-client + 2.0.0-mr1-cdh4.1.1 + + + org.mortbay.jetty + servlet-api-2.5 + + + + + org.apache.hadoop + hadoop-hdfs + 2.0.0-cdh4.1.1 org.mortbay.jetty @@ -325,6 +336,12 @@ lz4 1.1.2 + + com.google.protobuf + protobuf-java + 2.4.0a + + diff --git a/server/pom.xml b/server/pom.xml index f5aa2785d8e..9c1cab92a33 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -170,8 +170,13 @@ org.apache.hadoop - hadoop-core + hadoop-client + + org.apache.hadoop + hadoop-hdfs + + From de61937167d403eda7c1242c6c005a2f55098c11 Mon Sep 17 00:00:00 2001 From: Jan Rudert Date: Thu, 1 Aug 2013 17:58:32 +0200 Subject: [PATCH 06/13] reverted accident change --- build.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/build.sh b/build.sh index 796e958cd9d..fe534e319f7 100755 --- a/build.sh +++ b/build.sh @@ -16,8 +16,7 @@ VERSION=`cat pom.xml | grep version | head -4 | tail -1 | sed 's_.*\([^ echo Using Version[${VERSION}] mvn clean -#mvn install -U -mvn install -U -DskipTests +mvn package if [ $? -ne "0" ]; then echo "mvn package failed" From 7195637a41c8de06f9b2b4e07e83bed89bd724f0 Mon Sep 17 00:00:00 2001 From: Jan Rudert Date: Mon, 5 Aug 2013 09:16:57 +0200 Subject: [PATCH 07/13] use same hadoop version as druid master (0.20.2) --- indexing-hadoop/pom.xml | 6 +----- pom.xml | 15 ++------------- server/pom.xml | 7 +------ 3 files changed, 4 insertions(+), 24 deletions(-) diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 04fa6f06f8f..2abed0a074a 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -77,11 +77,7 @@ org.apache.hadoop - hadoop-client - - - org.apache.hadoop - hadoop-hdfs + hadoop-core com.fasterxml.jackson.core diff --git a/pom.xml b/pom.xml index c92b20a45b3..e96f2b4b8b7 100644 --- a/pom.xml +++ b/pom.xml @@ -145,19 +145,8 @@ org.apache.hadoop - hadoop-client - 2.0.0-mr1-cdh4.1.1 - - - org.mortbay.jetty - servlet-api-2.5 - - - - - org.apache.hadoop - hadoop-hdfs - 2.0.0-cdh4.1.1 + hadoop-core + 0.20.2 org.mortbay.jetty diff --git a/server/pom.xml b/server/pom.xml index bad640290c6..549334ca652 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -170,13 +170,8 @@ org.apache.hadoop - hadoop-client + hadoop-core - - org.apache.hadoop - hadoop-hdfs - - From 55267ca0e4ba8e52fb5e9d3fcce7a167718eec81 Mon Sep 17 00:00:00 2001 From: jan Date: Sun, 11 Aug 2013 13:44:13 +0200 Subject: [PATCH 08/13] removed @author tags --- .../metamx/druid/indexer/data/ByteBufferInputRowParser.java | 3 --- .../com/metamx/druid/indexer/data/ProtoBufInputRowParser.java | 3 --- .../metamx/druid/indexer/data/ProtoBufInputRowParserTest.java | 3 --- .../metamx/druid/indexer/data/StringInputRowParserTest.java | 3 --- .../com/metamx/druid/loading/DataSegmentPusherUtilTest.java | 3 --- 5 files changed, 15 deletions(-) diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java index f785e3f3a12..bc3721bf719 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java @@ -5,9 +5,6 @@ import java.nio.ByteBuffer; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -/** - * @author jan.rudert - */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringInputRowParser.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "protobuf", value = ProtoBufInputRowParser.class), diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java index 89c6eca4a3c..47f56047cb1 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java @@ -19,9 +19,6 @@ import com.google.protobuf.DynamicMessage; import com.google.protobuf.InvalidProtocolBufferException; import com.metamx.druid.input.InputRow; -/** - * @author jan.rudert - */ public class ProtoBufInputRowParser implements ByteBufferInputRowParser { diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java index bb0a6021f6e..237f89edf14 100644 --- a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java @@ -14,9 +14,6 @@ import org.junit.Test; import com.metamx.druid.input.InputRow; -/** - * @author jan.rudert - */ public class ProtoBufInputRowParserTest { public static final String[] DIMENSIONS = new String[]{"eventType", "id", "someOtherId", "isValid"}; diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/StringInputRowParserTest.java b/indexing-common/src/test/java/com/metamx/druid/indexer/data/StringInputRowParserTest.java index a3f072b5a0e..bee550a9d6c 100644 --- a/indexing-common/src/test/java/com/metamx/druid/indexer/data/StringInputRowParserTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/indexer/data/StringInputRowParserTest.java @@ -5,9 +5,6 @@ import org.junit.Test; import java.nio.ByteBuffer; -/** - * @author jan.rudert - */ public class StringInputRowParserTest { diff --git a/server/src/test/java/com/metamx/druid/loading/DataSegmentPusherUtilTest.java b/server/src/test/java/com/metamx/druid/loading/DataSegmentPusherUtilTest.java index f367fceab47..2337f882098 100644 --- a/server/src/test/java/com/metamx/druid/loading/DataSegmentPusherUtilTest.java +++ b/server/src/test/java/com/metamx/druid/loading/DataSegmentPusherUtilTest.java @@ -10,9 +10,6 @@ import org.junit.Test; import java.util.Arrays; -/** - * @author jan.rudert - */ public class DataSegmentPusherUtilTest { @Test public void shouldNotHaveColonsInHdfsStorageDir() throws Exception { From d6992d96d624cbb86a17ba33edd257a1eaa61e81 Mon Sep 17 00:00:00 2001 From: jan Date: Sun, 11 Aug 2013 13:50:53 +0200 Subject: [PATCH 09/13] use ImmutableList's factory method to build a defensive copy of dimensions --- .../java/com/metamx/druid/indexer/data/MapInputRowParser.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java index 93c221cfaf9..da82f946261 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java @@ -2,13 +2,13 @@ package com.metamx.druid.indexer.data; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import org.joda.time.DateTime; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -28,7 +28,7 @@ public class MapInputRowParser implements InputRowParser> { this.timestampSpec = timestampSpec; if (dimensions != null) { - this.dimensions = Collections.unmodifiableList(Lists.newArrayList(dimensions)); + this.dimensions = ImmutableList.copyOf(dimensions); } this.dimensionExclusions = Sets.newHashSet(); if (dimensionExclusions != null) { From 5084fc0642a4a2470becee0eb1a5618aa03b6270 Mon Sep 17 00:00:00 2001 From: jan Date: Sun, 11 Aug 2013 13:51:53 +0200 Subject: [PATCH 10/13] fixed ByteBuffer usage --- .../indexer/data/StringInputRowParser.java | 2 +- .../data/StringInputRowParserTest.java | 20 ------------------- 2 files changed, 1 insertion(+), 21 deletions(-) delete mode 100644 indexing-common/src/test/java/com/metamx/druid/indexer/data/StringInputRowParserTest.java diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java index b3942d67d27..c1b52d142c6 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java @@ -67,7 +67,7 @@ public class StringInputRowParser implements ByteBufferInputRowParser private Map buildStringKeyMap(ByteBuffer input) { - int payloadSize = input.limit(); + int payloadSize = input.remaining(); if (chars == null || chars.remaining() < payloadSize) { diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/StringInputRowParserTest.java b/indexing-common/src/test/java/com/metamx/druid/indexer/data/StringInputRowParserTest.java deleted file mode 100644 index bee550a9d6c..00000000000 --- a/indexing-common/src/test/java/com/metamx/druid/indexer/data/StringInputRowParserTest.java +++ /dev/null @@ -1,20 +0,0 @@ -package com.metamx.druid.indexer.data; - -import junit.framework.Assert; -import org.junit.Test; - -import java.nio.ByteBuffer; - -public class StringInputRowParserTest { - - - - @Test - public void testPayloadSize() { - ByteBuffer payload = ByteBuffer.allocate(10); - payload.position(2); - payload.limit(5); - payload.rewind(); - Assert.assertEquals(5, payload.limit()); - } -} From 3ec2761a101af894811c7dcc9c5d1f09debbb1d0 Mon Sep 17 00:00:00 2001 From: jan Date: Sun, 11 Aug 2013 13:55:52 +0200 Subject: [PATCH 11/13] removed illegal start of expression error --- .../main/java/com/metamx/druid/indexer/data/InputRowParser.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java index 25957379d9a..960b16c13ed 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java @@ -7,7 +7,7 @@ import com.metamx.druid.input.InputRow; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringInputRowParser.class) @JsonSubTypes({ @JsonSubTypes.Type(name = "string", value = StringInputRowParser.class), - @JsonSubTypes.Type(name = "protobuf", value = ProtoBufInputRowParser.class), + @JsonSubTypes.Type(name = "protobuf", value = ProtoBufInputRowParser.class) }) public interface InputRowParser { From 7842ade02cddbe549f527e3a782e25c36c15756f Mon Sep 17 00:00:00 2001 From: jan Date: Sun, 11 Aug 2013 21:11:25 +0200 Subject: [PATCH 12/13] added comment on a possible protobuf message based in put row optimization --- .../com/metamx/druid/indexer/data/ProtoBufInputRowParser.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java index 47f56047cb1..d3d2dab6d5f 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java @@ -41,7 +41,8 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser @Override public InputRow parse(ByteBuffer input) { - + // TODO there should be a ProtoBufBasedInputRow that does not need an intermediate map but accesses + // the DynamicMessage directly Map theMap = buildStringKeyMap(input); return inputRowCreator.parse(theMap); From e8ab64d6a8e85ed3e7743e19ba56a39e6d5c63fb Mon Sep 17 00:00:00 2001 From: Jan Rudert Date: Mon, 12 Aug 2013 17:43:09 +0200 Subject: [PATCH 13/13] adapted changes after merging of branch 'master' of github.com:metamx/druid into kafka-protobuf --- .../java/com/metamx/druid/indexer/data/InputRowParser.java | 3 --- .../com/metamx/druid/indexer/data/ProtoBufInputRowParser.java | 3 ++- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java index 8aa8751dfc8..960ac8719a7 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java @@ -1,11 +1,8 @@ package com.metamx.druid.indexer.data; -<<<<<<< HEAD import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -======= import com.metamx.common.exception.FormattedException; ->>>>>>> 68a3f1ab79feb1677e4174728e5209208a54bad9 import com.metamx.druid.input.InputRow; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringInputRowParser.class) diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java index d3d2dab6d5f..b5cb4e2e0c9 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java @@ -17,6 +17,7 @@ import com.google.protobuf.ByteString; import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; import com.google.protobuf.InvalidProtocolBufferException; +import com.metamx.common.exception.FormattedException; import com.metamx.druid.input.InputRow; public class ProtoBufInputRowParser implements ByteBufferInputRowParser @@ -39,7 +40,7 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser } @Override - public InputRow parse(ByteBuffer input) + public InputRow parse(ByteBuffer input) throws FormattedException { // TODO there should be a ProtoBufBasedInputRow that does not need an intermediate map but accesses // the DynamicMessage directly