mirror of https://github.com/apache/druid.git
Kinesis Input Format for timestamp, and payload parsing (#16813)
* SQL syntax error should target USER persona * * revert change to queryHandler and related tests, based on review comments * * add test * Introduce KinesisRecordEntity to support Kinesis headers in InputFormats * * add kinesisInputFormat and Reader, and tests * * bind KinesisInputFormat class to module * * improve test coverage * * remove references to kafka * * resolve review comments * * remove comment * * fix grammer of comment * * fix comment again * * fix comment again * * more review comments * * add partitionKey * * add check for same timestamp and partitionKey column name * * fix intellij inspection
This commit is contained in:
parent
63ba5a4113
commit
9b731e8f0a
|
@ -33,8 +33,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord;
|
|||
* key, and timestamp.
|
||||
* <p>
|
||||
* NOTE: Any records with null values will be skipped, even if they contain non-null keys, or headers
|
||||
* <p>
|
||||
* This functionality is not yet exposed through any built-in InputFormats, but is available for use in extensions.
|
||||
*/
|
||||
public class KafkaRecordEntity extends ByteEntity
|
||||
{
|
||||
|
|
|
@ -0,0 +1,157 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.data.input.kinesis;
|
||||
|
||||
import com.amazonaws.services.kinesis.model.Record;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.data.input.InputEntity;
|
||||
import org.apache.druid.data.input.InputEntityReader;
|
||||
import org.apache.druid.data.input.InputFormat;
|
||||
import org.apache.druid.data.input.InputRowSchema;
|
||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.indexing.seekablestream.SettableByteEntity;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Kinesis aware InputFormat. Allows for reading kinesis specific values that are stored in the {@link Record}. At
|
||||
* this time, this input format only supports reading data from the following record components
|
||||
* <p>
|
||||
* - {@link Record#data}
|
||||
* - {@link Record#approximateArrivalTimestamp}
|
||||
* - {@link Record#partitionKey}
|
||||
* <p>
|
||||
* This class can be extended easily to read other fields available in the kinesis record.
|
||||
*/
|
||||
public class KinesisInputFormat implements InputFormat
|
||||
{
|
||||
private static final String DEFAULT_TIMESTAMP_COLUMN_NAME = "kinesis.timestamp";
|
||||
private static final String DEFAULT_PARTITION_KEY_COLUMN_NAME = "kinesis.partitionKey";
|
||||
|
||||
// Since KinesisInputFormat blends data from record properties, and payload, timestamp spec can be pointing to an
|
||||
// attribute within one of these 2 sections. To handle scenarios where there is no timestamp value in the payload, we
|
||||
// induce an artificial timestamp value to avoid unnecessary parser barf out. Users in such situations can use the
|
||||
// inputFormat's kinesis record timestamp as its primary timestamp.
|
||||
public static final String DEFAULT_AUTO_TIMESTAMP_STRING = "__kif_auto_timestamp";
|
||||
private final TimestampSpec dummyTimestampSpec = new TimestampSpec(DEFAULT_AUTO_TIMESTAMP_STRING, "auto", DateTimes.EPOCH);
|
||||
|
||||
private final InputFormat valueFormat;
|
||||
private final String timestampColumnName;
|
||||
private final String partitionKeyColumnName;
|
||||
|
||||
public KinesisInputFormat(
|
||||
@JsonProperty("valueFormat") InputFormat valueFormat,
|
||||
@JsonProperty("partitionKeyColumnName") @Nullable String partitionKeyColumnName,
|
||||
@JsonProperty("timestampColumnName") @Nullable String timestampColumnName
|
||||
)
|
||||
{
|
||||
this.valueFormat = Preconditions.checkNotNull(valueFormat, "valueFormat must not be null");
|
||||
Preconditions.checkState(
|
||||
!(timestampColumnName != null && timestampColumnName.equals(partitionKeyColumnName)),
|
||||
"timestampColumnName and partitionKeyColumnName must be different"
|
||||
);
|
||||
this.partitionKeyColumnName = partitionKeyColumnName != null
|
||||
? partitionKeyColumnName
|
||||
: DEFAULT_PARTITION_KEY_COLUMN_NAME;
|
||||
this.timestampColumnName = timestampColumnName != null ? timestampColumnName : DEFAULT_TIMESTAMP_COLUMN_NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isSplittable()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputEntityReader createReader(InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory)
|
||||
{
|
||||
final SettableByteEntity<KinesisRecordEntity> settableByteEntitySource;
|
||||
if (source instanceof SettableByteEntity) {
|
||||
settableByteEntitySource = (SettableByteEntity<KinesisRecordEntity>) source;
|
||||
} else {
|
||||
settableByteEntitySource = new SettableByteEntity<>();
|
||||
settableByteEntitySource.setEntity((KinesisRecordEntity) source);
|
||||
}
|
||||
InputRowSchema newInputRowSchema = new InputRowSchema(
|
||||
dummyTimestampSpec,
|
||||
inputRowSchema.getDimensionsSpec(),
|
||||
inputRowSchema.getColumnsFilter(),
|
||||
inputRowSchema.getMetricNames()
|
||||
);
|
||||
return new KinesisInputReader(
|
||||
inputRowSchema,
|
||||
settableByteEntitySource,
|
||||
JsonInputFormat.withLineSplittable(valueFormat, false).createReader(
|
||||
newInputRowSchema,
|
||||
source,
|
||||
temporaryDirectory
|
||||
),
|
||||
partitionKeyColumnName,
|
||||
timestampColumnName
|
||||
);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public InputFormat getValueFormat()
|
||||
{
|
||||
return valueFormat;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
public String getTimestampColumnName()
|
||||
{
|
||||
return timestampColumnName;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
public String getPartitionKeyColumnName()
|
||||
{
|
||||
return partitionKeyColumnName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
KinesisInputFormat that = (KinesisInputFormat) o;
|
||||
return Objects.equals(valueFormat, that.valueFormat)
|
||||
&& Objects.equals(timestampColumnName, that.timestampColumnName)
|
||||
&& Objects.equals(partitionKeyColumnName, that.partitionKeyColumnName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(valueFormat, timestampColumnName, partitionKeyColumnName);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,256 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.data.input.kinesis;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.data.input.InputEntityReader;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.InputRowListPlusRawValues;
|
||||
import org.apache.druid.data.input.InputRowSchema;
|
||||
import org.apache.druid.data.input.MapBasedInputRow;
|
||||
import org.apache.druid.data.input.impl.MapInputRowParser;
|
||||
import org.apache.druid.indexing.seekablestream.SettableByteEntity;
|
||||
import org.apache.druid.java.util.common.CloseableIterators;
|
||||
import org.apache.druid.java.util.common.parsers.CloseableIterator;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.AbstractMap;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class KinesisInputReader implements InputEntityReader
|
||||
{
|
||||
|
||||
private final InputRowSchema inputRowSchema;
|
||||
private final SettableByteEntity<KinesisRecordEntity> source;
|
||||
private final InputEntityReader valueParser;
|
||||
private final String partitionKeyColumnName;
|
||||
private final String timestampColumnName;
|
||||
|
||||
public KinesisInputReader(
|
||||
InputRowSchema inputRowSchema,
|
||||
SettableByteEntity<KinesisRecordEntity> source,
|
||||
InputEntityReader valueParser,
|
||||
String partitionKeyColumnName,
|
||||
String timestampColumnName
|
||||
)
|
||||
{
|
||||
this.inputRowSchema = inputRowSchema;
|
||||
this.source = source;
|
||||
this.valueParser = valueParser;
|
||||
this.partitionKeyColumnName = partitionKeyColumnName;
|
||||
this.timestampColumnName = timestampColumnName;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public CloseableIterator<InputRow> read() throws IOException
|
||||
{
|
||||
final KinesisRecordEntity record = source.getEntity();
|
||||
final Map<String, Object> mergedHeaderMap = extractHeaders(record);
|
||||
|
||||
if (record.getRecord().getData() != null) {
|
||||
return buildBlendedRows(valueParser, mergedHeaderMap);
|
||||
} else {
|
||||
return CloseableIterators.withEmptyBaggage(buildInputRowsForMap(mergedHeaderMap).iterator());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CloseableIterator<InputRowListPlusRawValues> sample() throws IOException
|
||||
{
|
||||
final KinesisRecordEntity record = source.getEntity();
|
||||
InputRowListPlusRawValues headers = extractHeaderSample(record);
|
||||
if (record.getRecord().getData() != null) {
|
||||
return buildBlendedRowsSample(valueParser, headers.getRawValues());
|
||||
} else {
|
||||
final List<InputRowListPlusRawValues> rows = Collections.singletonList(headers);
|
||||
return CloseableIterators.withEmptyBaggage(rows.iterator());
|
||||
}
|
||||
}
|
||||
|
||||
private Map<String, Object> extractHeaders(KinesisRecordEntity record)
|
||||
{
|
||||
final Map<String, Object> mergedHeaderMap = new HashMap<>();
|
||||
mergedHeaderMap.put(timestampColumnName, record.getRecord().getApproximateArrivalTimestamp().getTime());
|
||||
mergedHeaderMap.put(partitionKeyColumnName, record.getRecord().getPartitionKey());
|
||||
return mergedHeaderMap;
|
||||
}
|
||||
|
||||
private CloseableIterator<InputRow> buildBlendedRows(
|
||||
InputEntityReader valueParser,
|
||||
Map<String, Object> headerKeyList
|
||||
) throws IOException
|
||||
{
|
||||
return valueParser.read().map(
|
||||
r -> {
|
||||
final HashSet<String> newDimensions = new HashSet<>(r.getDimensions());
|
||||
final Map<String, Object> event = buildBlendedEventMap(r::getRaw, newDimensions, headerKeyList);
|
||||
newDimensions.addAll(headerKeyList.keySet());
|
||||
// Remove the dummy timestamp added in KinesisInputFormat
|
||||
newDimensions.remove(KinesisInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING);
|
||||
|
||||
final DateTime timestamp = MapInputRowParser.parseTimestamp(inputRowSchema.getTimestampSpec(), event);
|
||||
return new MapBasedInputRow(
|
||||
timestamp,
|
||||
MapInputRowParser.findDimensions(
|
||||
inputRowSchema.getTimestampSpec(),
|
||||
inputRowSchema.getDimensionsSpec(),
|
||||
newDimensions
|
||||
),
|
||||
event
|
||||
);
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private InputRowListPlusRawValues extractHeaderSample(KinesisRecordEntity record)
|
||||
{
|
||||
Map<String, Object> mergedHeaderMap = extractHeaders(record);
|
||||
return InputRowListPlusRawValues.of(buildInputRowsForMap(mergedHeaderMap), mergedHeaderMap);
|
||||
}
|
||||
|
||||
private CloseableIterator<InputRowListPlusRawValues> buildBlendedRowsSample(
|
||||
InputEntityReader valueParser,
|
||||
Map<String, Object> headerKeyList
|
||||
) throws IOException
|
||||
{
|
||||
return valueParser.sample().map(
|
||||
rowAndValues -> {
|
||||
if (rowAndValues.getParseException() != null) {
|
||||
return rowAndValues;
|
||||
}
|
||||
List<InputRow> newInputRows = Lists.newArrayListWithCapacity(rowAndValues.getInputRows().size());
|
||||
List<Map<String, Object>> newRawRows = Lists.newArrayListWithCapacity(rowAndValues.getRawValues().size());
|
||||
|
||||
for (Map<String, Object> raw : rowAndValues.getRawValuesList()) {
|
||||
newRawRows.add(buildBlendedEventMap(raw::get, raw.keySet(), headerKeyList));
|
||||
}
|
||||
for (InputRow r : rowAndValues.getInputRows()) {
|
||||
if (r != null) {
|
||||
final HashSet<String> newDimensions = new HashSet<>(r.getDimensions());
|
||||
final Map<String, Object> event = buildBlendedEventMap(
|
||||
r::getRaw,
|
||||
newDimensions,
|
||||
headerKeyList
|
||||
);
|
||||
newDimensions.addAll(headerKeyList.keySet());
|
||||
// Remove the dummy timestamp added in KinesisInputFormat
|
||||
newDimensions.remove(KinesisInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING);
|
||||
newInputRows.add(
|
||||
new MapBasedInputRow(
|
||||
inputRowSchema.getTimestampSpec().extractTimestamp(event),
|
||||
MapInputRowParser.findDimensions(
|
||||
inputRowSchema.getTimestampSpec(),
|
||||
inputRowSchema.getDimensionsSpec(),
|
||||
newDimensions
|
||||
),
|
||||
event
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
return InputRowListPlusRawValues.ofList(newRawRows, newInputRows, null);
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private List<InputRow> buildInputRowsForMap(Map<String, Object> headerKeyList)
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new MapBasedInputRow(
|
||||
inputRowSchema.getTimestampSpec().extractTimestamp(headerKeyList),
|
||||
MapInputRowParser.findDimensions(
|
||||
inputRowSchema.getTimestampSpec(),
|
||||
inputRowSchema.getDimensionsSpec(),
|
||||
headerKeyList.keySet()
|
||||
),
|
||||
headerKeyList
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private Map<String, Object> buildBlendedEventMap(
|
||||
Function<String, Object> getRowValue,
|
||||
Set<String> rowDimensions,
|
||||
Map<String, Object> fallback
|
||||
)
|
||||
{
|
||||
final Set<String> keySet = new HashSet<>(fallback.keySet());
|
||||
keySet.addAll(rowDimensions);
|
||||
|
||||
return new AbstractMap<String, Object>()
|
||||
{
|
||||
@Override
|
||||
public Object get(Object key)
|
||||
{
|
||||
final String skey = (String) key;
|
||||
final Object val = getRowValue.apply(skey);
|
||||
if (val == null) {
|
||||
return fallback.get(skey);
|
||||
}
|
||||
return val;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> keySet()
|
||||
{
|
||||
return keySet;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Entry<String, Object>> entrySet()
|
||||
{
|
||||
return keySet().stream()
|
||||
.map(
|
||||
field -> new Entry<String, Object>()
|
||||
{
|
||||
@Override
|
||||
public String getKey()
|
||||
{
|
||||
return field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getValue()
|
||||
{
|
||||
return get(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object setValue(final Object value)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
)
|
||||
.collect(Collectors.toCollection(LinkedHashSet::new));
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.data.input.kinesis;
|
||||
|
||||
import com.amazonaws.services.kinesis.model.Record;
|
||||
import org.apache.druid.data.input.InputFormat;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.indexing.kinesis.KinesisRecordSupplier;
|
||||
|
||||
/**
|
||||
* A {@link ByteEntity} generated by {@link KinesisRecordSupplier} and fed to any {@link InputFormat} used by kinesis
|
||||
* indexing tasks.
|
||||
* <p>
|
||||
* It can be used as a regular ByteEntity, in which case the kinesis record value is returned, but the {@link #getRecord}
|
||||
* method also allows Kinesis-aware {@link InputFormat} implementations to read the full kinesis record, including
|
||||
* timestamp, encrytion key, patition key, and sequence number
|
||||
* <p>
|
||||
* NOTE: Any records with null values will be returned as records with just only kinesis properties and no data payload
|
||||
*/
|
||||
public class KinesisRecordEntity extends ByteEntity
|
||||
{
|
||||
private final Record record;
|
||||
|
||||
public KinesisRecordEntity(Record record)
|
||||
{
|
||||
super(record.getData());
|
||||
this.record = record;
|
||||
}
|
||||
|
||||
public Record getRecord()
|
||||
{
|
||||
return record;
|
||||
}
|
||||
}
|
|
@ -27,7 +27,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.inject.name.Named;
|
||||
import org.apache.druid.common.aws.AWSCredentialsConfig;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.task.TaskResource;
|
||||
|
@ -46,7 +46,7 @@ import java.util.Collections;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class KinesisIndexTask extends SeekableStreamIndexTask<String, String, ByteEntity>
|
||||
public class KinesisIndexTask extends SeekableStreamIndexTask<String, String, KinesisRecordEntity>
|
||||
{
|
||||
private static final String TYPE = "index_kinesis";
|
||||
|
||||
|
@ -100,7 +100,7 @@ public class KinesisIndexTask extends SeekableStreamIndexTask<String, String, By
|
|||
}
|
||||
|
||||
@Override
|
||||
protected SeekableStreamIndexTaskRunner<String, String, ByteEntity> createTaskRunner()
|
||||
protected SeekableStreamIndexTaskRunner<String, String, KinesisRecordEntity> createTaskRunner()
|
||||
{
|
||||
//noinspection unchecked
|
||||
return new KinesisIndexTaskRunner(
|
||||
|
|
|
@ -21,8 +21,8 @@ package org.apache.druid.indexing.kinesis;
|
|||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexing.common.LockGranularity;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
|
||||
|
@ -49,7 +49,7 @@ import java.util.Set;
|
|||
import java.util.TreeMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String, String, ByteEntity>
|
||||
public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String, String, KinesisRecordEntity>
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(KinesisIndexTaskRunner.class);
|
||||
private static final long POLL_TIMEOUT = 100;
|
||||
|
@ -81,8 +81,8 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
|
|||
|
||||
@Nonnull
|
||||
@Override
|
||||
protected List<OrderedPartitionableRecord<String, String, ByteEntity>> getRecords(
|
||||
RecordSupplier<String, String, ByteEntity> recordSupplier, TaskToolbox toolbox
|
||||
protected List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> getRecords(
|
||||
RecordSupplier<String, String, KinesisRecordEntity> recordSupplier, TaskToolbox toolbox
|
||||
)
|
||||
{
|
||||
return recordSupplier.poll(POLL_TIMEOUT);
|
||||
|
@ -119,7 +119,7 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
|
|||
@Override
|
||||
protected void possiblyResetDataSourceMetadata(
|
||||
TaskToolbox toolbox,
|
||||
RecordSupplier<String, String, ByteEntity> recordSupplier,
|
||||
RecordSupplier<String, String, KinesisRecordEntity> recordSupplier,
|
||||
Set<StreamPartition<String>> assignment
|
||||
)
|
||||
{
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.inject.Binder;
|
||||
import com.google.inject.name.Names;
|
||||
import org.apache.druid.common.aws.AWSCredentialsConfig;
|
||||
import org.apache.druid.data.input.kinesis.KinesisInputFormat;
|
||||
import org.apache.druid.guice.JsonConfigProvider;
|
||||
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec;
|
||||
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig;
|
||||
|
@ -50,7 +51,8 @@ public class KinesisIndexingServiceModule implements DruidModule
|
|||
new NamedType(KinesisIndexTaskIOConfig.class, SCHEME),
|
||||
new NamedType(KinesisSupervisorTuningConfig.class, SCHEME),
|
||||
new NamedType(KinesisSupervisorSpec.class, SCHEME),
|
||||
new NamedType(KinesisSamplerSpec.class, SCHEME)
|
||||
new NamedType(KinesisSamplerSpec.class, SCHEME),
|
||||
new NamedType(KinesisInputFormat.class, SCHEME)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
|
|||
import com.amazonaws.client.builder.AwsClientBuilder;
|
||||
import com.amazonaws.services.kinesis.AmazonKinesis;
|
||||
import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder;
|
||||
import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord;
|
||||
import com.amazonaws.services.kinesis.model.DescribeStreamRequest;
|
||||
import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
|
||||
import com.amazonaws.services.kinesis.model.GetRecordsRequest;
|
||||
|
@ -49,7 +50,7 @@ import com.google.common.collect.Maps;
|
|||
import org.apache.druid.common.aws.AWSClientUtil;
|
||||
import org.apache.druid.common.aws.AWSCredentialsConfig;
|
||||
import org.apache.druid.common.aws.AWSCredentialsUtils;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisor;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
|
||||
|
@ -69,7 +70,6 @@ import javax.annotation.Nullable;
|
|||
import java.lang.invoke.MethodHandle;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.lang.reflect.Method;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
@ -94,7 +94,7 @@ import java.util.stream.Collectors;
|
|||
* This class implements a local buffer for storing fetched Kinesis records. Fetching is done
|
||||
* in background threads.
|
||||
*/
|
||||
public class KinesisRecordSupplier implements RecordSupplier<String, String, ByteEntity>
|
||||
public class KinesisRecordSupplier implements RecordSupplier<String, String, KinesisRecordEntity>
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(KinesisRecordSupplier.class);
|
||||
private static final long PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS = 3000;
|
||||
|
@ -210,7 +210,7 @@ public class KinesisRecordSupplier implements RecordSupplier<String, String, Byt
|
|||
|
||||
// used for retrying on InterruptedException
|
||||
GetRecordsResult recordsResult = null;
|
||||
OrderedPartitionableRecord<String, String, ByteEntity> currRecord;
|
||||
OrderedPartitionableRecord<String, String, KinesisRecordEntity> currRecord;
|
||||
long recordBufferOfferWaitMillis;
|
||||
try {
|
||||
|
||||
|
@ -248,7 +248,7 @@ public class KinesisRecordSupplier implements RecordSupplier<String, String, Byt
|
|||
|
||||
// list will come back empty if there are no records
|
||||
for (Record kinesisRecord : recordsResult.getRecords()) {
|
||||
final List<ByteEntity> data;
|
||||
final List<KinesisRecordEntity> data;
|
||||
|
||||
if (deaggregateHandle == null || getDataHandle == null) {
|
||||
throw new ISE("deaggregateHandle or getDataHandle is null!");
|
||||
|
@ -256,15 +256,15 @@ public class KinesisRecordSupplier implements RecordSupplier<String, String, Byt
|
|||
|
||||
data = new ArrayList<>();
|
||||
|
||||
final List userRecords = (List) deaggregateHandle.invokeExact(
|
||||
final List<UserRecord> userRecords = (List<UserRecord>) deaggregateHandle.invokeExact(
|
||||
Collections.singletonList(kinesisRecord)
|
||||
);
|
||||
|
||||
int recordSize = 0;
|
||||
for (Object userRecord : userRecords) {
|
||||
ByteEntity byteEntity = new ByteEntity((ByteBuffer) getDataHandle.invoke(userRecord));
|
||||
recordSize += byteEntity.getBuffer().array().length;
|
||||
data.add(byteEntity);
|
||||
for (UserRecord userRecord : userRecords) {
|
||||
KinesisRecordEntity kinesisRecordEntity = new KinesisRecordEntity(userRecord);
|
||||
recordSize += kinesisRecordEntity.getBuffer().array().length;
|
||||
data.add(kinesisRecordEntity);
|
||||
}
|
||||
|
||||
|
||||
|
@ -408,7 +408,7 @@ public class KinesisRecordSupplier implements RecordSupplier<String, String, Byt
|
|||
|
||||
private final ConcurrentMap<StreamPartition<String>, PartitionResource> partitionResources =
|
||||
new ConcurrentHashMap<>();
|
||||
private MemoryBoundLinkedBlockingQueue<OrderedPartitionableRecord<String, String, ByteEntity>> records;
|
||||
private MemoryBoundLinkedBlockingQueue<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> records;
|
||||
|
||||
private final boolean backgroundFetchEnabled;
|
||||
private volatile boolean closed = false;
|
||||
|
@ -615,12 +615,12 @@ public class KinesisRecordSupplier implements RecordSupplier<String, String, Byt
|
|||
|
||||
@Nonnull
|
||||
@Override
|
||||
public List<OrderedPartitionableRecord<String, String, ByteEntity>> poll(long timeout)
|
||||
public List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> poll(long timeout)
|
||||
{
|
||||
start();
|
||||
|
||||
try {
|
||||
List<MemoryBoundLinkedBlockingQueue.ObjectContainer<OrderedPartitionableRecord<String, String, ByteEntity>>> polledRecords = new ArrayList<>();
|
||||
List<MemoryBoundLinkedBlockingQueue.ObjectContainer<OrderedPartitionableRecord<String, String, KinesisRecordEntity>>> polledRecords = new ArrayList<>();
|
||||
|
||||
records.drain(
|
||||
polledRecords,
|
||||
|
@ -1040,7 +1040,7 @@ public class KinesisRecordSupplier implements RecordSupplier<String, String, Byt
|
|||
}
|
||||
|
||||
// filter records in buffer and only retain ones whose partition was not seeked
|
||||
MemoryBoundLinkedBlockingQueue<OrderedPartitionableRecord<String, String, ByteEntity>> newQ =
|
||||
MemoryBoundLinkedBlockingQueue<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> newQ =
|
||||
new MemoryBoundLinkedBlockingQueue<>(recordBufferSizeBytes);
|
||||
|
||||
records.stream()
|
||||
|
|
|
@ -25,7 +25,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.common.aws.AWSCredentialsConfig;
|
||||
import org.apache.druid.common.utils.IdUtils;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.apache.druid.indexing.common.task.TaskResource;
|
||||
import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata;
|
||||
|
@ -74,7 +74,7 @@ import java.util.stream.Collectors;
|
|||
* tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of
|
||||
* Kinesis sequences.
|
||||
*/
|
||||
public class KinesisSupervisor extends SeekableStreamSupervisor<String, String, ByteEntity>
|
||||
public class KinesisSupervisor extends SeekableStreamSupervisor<String, String, KinesisRecordEntity>
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(KinesisSupervisor.class);
|
||||
|
||||
|
@ -150,7 +150,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String,
|
|||
}
|
||||
|
||||
@Override
|
||||
protected List<SeekableStreamIndexTask<String, String, ByteEntity>> createIndexTasks(
|
||||
protected List<SeekableStreamIndexTask<String, String, KinesisRecordEntity>> createIndexTasks(
|
||||
int replicas,
|
||||
String baseSequenceName,
|
||||
ObjectMapper sortingMapper,
|
||||
|
@ -164,7 +164,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String,
|
|||
final Map<String, Object> context = createBaseTaskContexts();
|
||||
context.put(CHECKPOINTS_CTX_KEY, checkpoints);
|
||||
|
||||
List<SeekableStreamIndexTask<String, String, ByteEntity>> taskList = new ArrayList<>();
|
||||
List<SeekableStreamIndexTask<String, String, KinesisRecordEntity>> taskList = new ArrayList<>();
|
||||
for (int i = 0; i < replicas; i++) {
|
||||
String taskId = IdUtils.getRandomIdWithPrefix(baseSequenceName);
|
||||
taskList.add(new KinesisIndexTask(
|
||||
|
@ -183,7 +183,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String,
|
|||
|
||||
|
||||
@Override
|
||||
protected RecordSupplier<String, String, ByteEntity> setupRecordSupplier() throws RuntimeException
|
||||
protected RecordSupplier<String, String, KinesisRecordEntity> setupRecordSupplier() throws RuntimeException
|
||||
{
|
||||
KinesisSupervisorIOConfig ioConfig = spec.getIoConfig();
|
||||
KinesisIndexTaskTuningConfig taskTuningConfig = spec.getTuningConfig();
|
||||
|
|
|
@ -0,0 +1,940 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.data.input.kinesis;
|
||||
|
||||
import com.amazonaws.services.kinesis.model.Record;
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.ColumnsFilter;
|
||||
import org.apache.druid.data.input.InputEntityReader;
|
||||
import org.apache.druid.data.input.InputFormat;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.InputRowListPlusRawValues;
|
||||
import org.apache.druid.data.input.InputRowSchema;
|
||||
import org.apache.druid.data.input.impl.CsvInputFormat;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.indexing.common.TestUtils;
|
||||
import org.apache.druid.indexing.seekablestream.SettableByteEntity;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.parsers.CloseableIterator;
|
||||
import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec;
|
||||
import org.apache.druid.java.util.common.parsers.JSONPathFieldType;
|
||||
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
||||
import org.apache.druid.java.util.common.parsers.ParseException;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
|
||||
public class KinesisInputFormatTest
|
||||
{
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
}
|
||||
|
||||
|
||||
private static final String KINESIS_APPROXIMATE_TIME_DATE = "2024-07-29";
|
||||
private static final long KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS = DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis();
|
||||
private static final String DATA_TIMSTAMP_DATE = "2024-07-30";
|
||||
private static final String PARTITION_KEY = "partition_key_1";
|
||||
|
||||
private static final byte[] SIMPLE_JSON_VALUE_BYTES = StringUtils.toUtf8(
|
||||
TestUtils.singleQuoteToStandardJson(
|
||||
"{"
|
||||
+ " 'timestamp': '" + DATA_TIMSTAMP_DATE + "',"
|
||||
+ " 'bar': null,"
|
||||
+ " 'foo': 'x',"
|
||||
+ " 'baz': 4,"
|
||||
+ " 'o': {'mg': 1}"
|
||||
+ "}"
|
||||
)
|
||||
);
|
||||
|
||||
private KinesisInputFormat format;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
format = new KinesisInputFormat(
|
||||
// Value Format
|
||||
new JsonInputFormat(
|
||||
new JSONPathSpec(
|
||||
true,
|
||||
ImmutableList.of(
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2")
|
||||
)
|
||||
),
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
),
|
||||
"kinesis.newts.partitionKey",
|
||||
"kinesis.newts.timestamp"
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSerde() throws JsonProcessingException
|
||||
{
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
KinesisInputFormat kif = new KinesisInputFormat(
|
||||
// Value Format
|
||||
new JsonInputFormat(
|
||||
new JSONPathSpec(
|
||||
true,
|
||||
ImmutableList.of(
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2")
|
||||
)
|
||||
),
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
),
|
||||
"kinesis.newts.partitionKey",
|
||||
"kinesis.newts.timestamp"
|
||||
);
|
||||
Assert.assertEquals(format, kif);
|
||||
|
||||
final byte[] formatBytes = mapper.writeValueAsBytes(format);
|
||||
final byte[] kifBytes = mapper.writeValueAsBytes(kif);
|
||||
Assert.assertArrayEquals(formatBytes, kifBytes);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimestampFromHeader() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("kinesis.newts.timestamp", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
// Payload verifications
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
|
||||
Assert.assertEquals(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRawSample() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec(KinesisInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING, "auto", DateTimes.EPOCH),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRowListPlusRawValues> iterator = reader.sample()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRowListPlusRawValues rawValues = iterator.next();
|
||||
Assert.assertEquals(1, rawValues.getInputRows().size());
|
||||
InputRow row = rawValues.getInputRows().get(0);
|
||||
// Payload verifications
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
|
||||
Assert.assertEquals(
|
||||
String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProcessesSampleTimestampFromHeader() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("kinesis.newts.timestamp", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRowListPlusRawValues> iterator = reader.sample()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRowListPlusRawValues rawValues = iterator.next();
|
||||
Assert.assertEquals(1, rawValues.getInputRows().size());
|
||||
InputRow row = rawValues.getInputRows().get(0);
|
||||
// Payload verifications
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
|
||||
Assert.assertEquals(DateTimes.of(String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS)), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithMultipleMixedRecordsTimestampFromHeader() throws IOException
|
||||
{
|
||||
final byte[][] values = new byte[5][];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = StringUtils.toUtf8(
|
||||
"{\n"
|
||||
+ " \"timestamp\": \"2024-07-2" + i + "\",\n"
|
||||
+ " \"bar\": null,\n"
|
||||
+ " \"foo\": \"x\",\n"
|
||||
+ " \"baz\": 4,\n"
|
||||
+ " \"index\": " + i + ",\n"
|
||||
+ " \"o\": {\n"
|
||||
+ " \"mg\": 1\n"
|
||||
+ " }\n"
|
||||
+ "}"
|
||||
);
|
||||
}
|
||||
|
||||
SettableByteEntity<KinesisRecordEntity> settableByteEntity = new SettableByteEntity<>();
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("kinesis.newts.timestamp", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo",
|
||||
"kinesis.newts.timestamp"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
settableByteEntity,
|
||||
null
|
||||
);
|
||||
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
KinesisRecordEntity inputEntity = makeInputEntity(values[i], DateTimes.of("2024-07-1" + i).getMillis());
|
||||
settableByteEntity.setEntity(inputEntity);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
|
||||
// Payload verification
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
Assert.assertEquals(DateTimes.of("2024-07-1" + i), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(DateTimes.of("2024-07-1" + i).getMillis()),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
Assert.assertEquals(String.valueOf(i), Iterables.getOnlyElement(row.getDimension("index")));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimestampFromData() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("timestamp", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
// Payload verifications
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
|
||||
Assert.assertEquals(DateTimes.of(DATA_TIMSTAMP_DATE), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithMultipleMixedRecordsTimestampFromData() throws IOException
|
||||
{
|
||||
final byte[][] values = new byte[5][];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = StringUtils.toUtf8(
|
||||
"{\n"
|
||||
+ " \"timestamp\": \"2024-07-2" + i + "\",\n"
|
||||
+ " \"bar\": null,\n"
|
||||
+ " \"foo\": \"x\",\n"
|
||||
+ " \"baz\": 4,\n"
|
||||
+ " \"index\": " + i + ",\n"
|
||||
+ " \"o\": {\n"
|
||||
+ " \"mg\": 1\n"
|
||||
+ " }\n"
|
||||
+ "}"
|
||||
);
|
||||
}
|
||||
|
||||
SettableByteEntity<KinesisRecordEntity> settableByteEntity = new SettableByteEntity<>();
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("timestamp", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo",
|
||||
"kinesis.newts.timestamp"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
settableByteEntity,
|
||||
null
|
||||
);
|
||||
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
KinesisRecordEntity inputEntity = makeInputEntity(values[i], KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
settableByteEntity.setEntity(inputEntity);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
|
||||
// Payload verification
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
Assert.assertEquals(DateTimes.of("2024-07-2" + i), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(DateTimes.of("2024-07-29").getMillis()),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
Assert.assertEquals(String.valueOf(i), Iterables.getOnlyElement(row.getDimension("index")));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMissingTimestampThrowsException() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity =
|
||||
makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("time", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo",
|
||||
"kinesis.newts.timestamp"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
while (iterator.hasNext()) {
|
||||
Throwable t = Assert.assertThrows(ParseException.class, iterator::next);
|
||||
Assert.assertTrue(
|
||||
t.getMessage().startsWith("Timestamp[null] is unparseable! Event: {")
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithSchemaDiscoveryKinesisTimestampExcluded() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity =
|
||||
makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("timestamp", "iso", null),
|
||||
DimensionsSpec.builder()
|
||||
.useSchemaDiscovery(true)
|
||||
.setDimensionExclusions(ImmutableList.of("kinesis.newts.timestamp"))
|
||||
.build(),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
List<String> expectedDimensions = Arrays.asList(
|
||||
"foo",
|
||||
"root_baz",
|
||||
"o",
|
||||
"bar",
|
||||
"path_omg",
|
||||
"jq_omg",
|
||||
"jq_omg2",
|
||||
"baz",
|
||||
"root_baz2",
|
||||
"path_omg2",
|
||||
"kinesis.newts.partitionKey"
|
||||
);
|
||||
Collections.sort(expectedDimensions);
|
||||
Collections.sort(row.getDimensions());
|
||||
Assert.assertEquals(
|
||||
expectedDimensions,
|
||||
row.getDimensions()
|
||||
);
|
||||
|
||||
// Payload verifications
|
||||
Assert.assertEquals(DateTimes.of(DATA_TIMSTAMP_DATE), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithSchemaDiscoveryTimestampFromHeader() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity =
|
||||
makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("kinesis.newts.timestamp", "iso", null),
|
||||
DimensionsSpec.builder()
|
||||
.useSchemaDiscovery(true)
|
||||
.build(),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
List<String> expectedDimensions = Arrays.asList(
|
||||
"foo",
|
||||
"timestamp",
|
||||
"root_baz",
|
||||
"o",
|
||||
"bar",
|
||||
"path_omg",
|
||||
"jq_omg",
|
||||
"jq_omg2",
|
||||
"baz",
|
||||
"root_baz2",
|
||||
"path_omg2",
|
||||
"kinesis.newts.partitionKey"
|
||||
);
|
||||
Collections.sort(expectedDimensions);
|
||||
Collections.sort(row.getDimensions());
|
||||
Assert.assertEquals(
|
||||
expectedDimensions,
|
||||
row.getDimensions()
|
||||
);
|
||||
|
||||
// Payload verifications
|
||||
Assert.assertEquals(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValueInCsvFormat() throws IOException
|
||||
{
|
||||
format = new KinesisInputFormat(
|
||||
// Value Format
|
||||
new CsvInputFormat(
|
||||
Arrays.asList("foo", "bar", "timestamp", "baz"),
|
||||
null,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
),
|
||||
"kinesis.newts.partitionKey",
|
||||
"kinesis.newts.timestamp"
|
||||
);
|
||||
|
||||
KinesisRecordEntity inputEntity =
|
||||
makeInputEntity(StringUtils.toUtf8("x,,2024-07-30,4"), KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("timestamp", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo",
|
||||
"kinesis.newts.timestamp",
|
||||
"kinesis.newts.partitionKey"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
Assert.assertEquals(
|
||||
Arrays.asList(
|
||||
"bar",
|
||||
"foo",
|
||||
"kinesis.newts.timestamp",
|
||||
"kinesis.newts.partitionKey"
|
||||
),
|
||||
row.getDimensions()
|
||||
);
|
||||
// Payload verifications
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
|
||||
Assert.assertEquals(DateTimes.of("2024-07-30"), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertTrue(row.getDimension("bar").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithPartialDeclarationSchemaDiscovery() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity =
|
||||
makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("timestamp", "iso", null),
|
||||
DimensionsSpec.builder().setDimensions(
|
||||
DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar"))
|
||||
).useSchemaDiscovery(true).build(),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
|
||||
List<String> expectedDimensions = Arrays.asList(
|
||||
"bar",
|
||||
"foo",
|
||||
"kinesis.newts.timestamp",
|
||||
"kinesis.newts.partitionKey",
|
||||
"root_baz",
|
||||
"o",
|
||||
"path_omg",
|
||||
"jq_omg",
|
||||
"jq_omg2",
|
||||
"baz",
|
||||
"root_baz2",
|
||||
"path_omg2"
|
||||
);
|
||||
Collections.sort(expectedDimensions);
|
||||
Collections.sort(row.getDimensions());
|
||||
Assert.assertEquals(
|
||||
expectedDimensions,
|
||||
row.getDimensions()
|
||||
);
|
||||
|
||||
// Payload verifications
|
||||
Assert.assertEquals(DateTimes.of(DATA_TIMSTAMP_DATE), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("ResultOfMethodCallIgnored")
|
||||
public void testValidInputFormatConstruction()
|
||||
{
|
||||
InputFormat valueFormat = new JsonInputFormat(
|
||||
new JSONPathSpec(
|
||||
true,
|
||||
ImmutableList.of(
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2")
|
||||
)
|
||||
),
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
);
|
||||
// null partitionKeyColumnName and null timestampColumnName is valid
|
||||
new KinesisInputFormat(valueFormat, null, null);
|
||||
|
||||
// non-null partitionKeyColumnName and null timestampColumnName is valid
|
||||
new KinesisInputFormat(valueFormat, "kinesis.partitionKey", null);
|
||||
|
||||
// null partitionKeyColumnName and non-null timestampColumnName is valid
|
||||
new KinesisInputFormat(valueFormat, null, "kinesis.timestamp");
|
||||
|
||||
// non-null partitionKeyColumnName and non-null timestampColumnName is valid
|
||||
new KinesisInputFormat(valueFormat, "kinesis.partitionKey", "kinesis.timestamp");
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("ResultOfMethodCallIgnored")
|
||||
public void testInvalidInputFormatConstruction()
|
||||
{
|
||||
// null value format is invalid
|
||||
Assert.assertThrows(
|
||||
"valueFormat must not be null",
|
||||
NullPointerException.class,
|
||||
() -> new KinesisInputFormat(null, null, null)
|
||||
);
|
||||
|
||||
InputFormat valueFormat = new JsonInputFormat(
|
||||
new JSONPathSpec(
|
||||
true,
|
||||
ImmutableList.of(
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2")
|
||||
)
|
||||
),
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
);
|
||||
|
||||
// partitionKeyColumnName == timestampColumnName is invalid
|
||||
Assert.assertThrows(
|
||||
"timestampColumnName and partitionKeyColumnName must be different",
|
||||
IllegalStateException.class,
|
||||
() -> new KinesisInputFormat(valueFormat, "kinesis.timestamp", "kinesis.timestamp")
|
||||
);
|
||||
}
|
||||
|
||||
private KinesisRecordEntity makeInputEntity(
|
||||
byte[] payload,
|
||||
long kinesisTimestampMillis)
|
||||
{
|
||||
return new KinesisRecordEntity(
|
||||
new Record().withData(ByteBuffer.wrap(payload))
|
||||
.withApproximateArrivalTimestamp(new Date(kinesisTimestampMillis))
|
||||
.withPartitionKey(PARTITION_KEY)
|
||||
);
|
||||
}
|
||||
|
||||
private SettableByteEntity<KinesisRecordEntity> newSettableByteEntity(KinesisRecordEntity kinesisRecordEntity)
|
||||
{
|
||||
SettableByteEntity<KinesisRecordEntity> settableByteEntity = new SettableByteEntity<>();
|
||||
settableByteEntity.setEntity(kinesisRecordEntity);
|
||||
return settableByteEntity;
|
||||
}
|
||||
}
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.indexing.kinesis;
|
||||
|
||||
import com.amazonaws.services.kinesis.model.Record;
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
@ -41,6 +42,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec;
|
|||
import org.apache.druid.data.input.impl.FloatDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.LongDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexer.TaskState;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexer.report.IngestionStatsAndErrors;
|
||||
|
@ -127,39 +129,39 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
private static final String SHARD_ID0 = "0";
|
||||
|
||||
private static final List<KinesisRecord> RECORDS = Arrays.asList(
|
||||
createRecord("1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "5", jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
|
||||
createRecord("1", "6", new ByteEntity(StringUtils.toUtf8("unparseable"))),
|
||||
createRecord("1", "7", new ByteEntity(StringUtils.toUtf8(""))),
|
||||
createRecord("1", "8", new ByteEntity(StringUtils.toUtf8("{}"))),
|
||||
createRecord("1", "9", jb("2013", "f", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "10", jb("2049", "f", "y", "notanumber", "20.0", "1.0")),
|
||||
createRecord("1", "11", jb("2049", "f", "y", "10", "notanumber", "1.0")),
|
||||
createRecord("1", "12", jb("2049", "f", "y", "10", "20.0", "notanumber")),
|
||||
createRecord("0", "0", jb("2012", "g", "y", "10", "20.0", "1.0")),
|
||||
createRecord("0", "1", jb("2011", "h", "y", "10", "20.0", "1.0"))
|
||||
createRecord("1", "0", kjb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "1", kjb("2009", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "2", kjb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "3", kjb("2011", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "4", kjb("2011", "e", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "5", kjb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
|
||||
createRecord("1", "6", new KinesisRecordEntity(new Record().withData(new ByteEntity(StringUtils.toUtf8("unparseable")).getBuffer()))),
|
||||
createRecord("1", "7", new KinesisRecordEntity(new Record().withData(new ByteEntity(StringUtils.toUtf8("")).getBuffer()))),
|
||||
createRecord("1", "8", new KinesisRecordEntity(new Record().withData(new ByteEntity(StringUtils.toUtf8("{}")).getBuffer()))),
|
||||
createRecord("1", "9", kjb("2013", "f", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "10", kjb("2049", "f", "y", "notanumber", "20.0", "1.0")),
|
||||
createRecord("1", "11", kjb("2049", "f", "y", "10", "notanumber", "1.0")),
|
||||
createRecord("1", "12", kjb("2049", "f", "y", "10", "20.0", "notanumber")),
|
||||
createRecord("0", "0", kjb("2012", "g", "y", "10", "20.0", "1.0")),
|
||||
createRecord("0", "1", kjb("2011", "h", "y", "10", "20.0", "1.0"))
|
||||
);
|
||||
|
||||
private static final List<KinesisRecord> SINGLE_PARTITION_RECORDS = Arrays.asList(
|
||||
createRecord("1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "5", jb("2012", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "6", jb("2013", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "7", jb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "8", jb("2011", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "9", jb("2011", "e", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "10", jb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "11", jb("2009", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "12", jb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "13", jb("2012", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "14", jb("2013", "e", "y", "10", "20.0", "1.0"))
|
||||
createRecord("1", "0", kjb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "1", kjb("2009", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "2", kjb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "3", kjb("2011", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "4", kjb("2011", "e", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "5", kjb("2012", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "6", kjb("2013", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "7", kjb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "8", kjb("2011", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "9", kjb("2011", "e", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "10", kjb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "11", kjb("2009", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "12", kjb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "13", kjb("2012", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "14", kjb("2013", "e", "y", "10", "20.0", "1.0"))
|
||||
);
|
||||
|
||||
private static KinesisRecordSupplier recordSupplier;
|
||||
|
@ -272,12 +274,12 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
record.getPartitionId(),
|
||||
record.getSequenceNumber(),
|
||||
record.getData().stream()
|
||||
.map(entity -> new ByteEntity(entity.getBuffer()))
|
||||
.map(entity -> new KinesisRecordEntity(new Record().withData(entity.getBuffer())))
|
||||
.collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
|
||||
private static List<OrderedPartitionableRecord<String, String, ByteEntity>> clone(
|
||||
private static List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> clone(
|
||||
List<KinesisRecord> records,
|
||||
int start,
|
||||
int end
|
||||
|
@ -289,14 +291,14 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
/**
|
||||
* Records can only be read once, hence we must use fresh records every time.
|
||||
*/
|
||||
private static List<OrderedPartitionableRecord<String, String, ByteEntity>> clone(
|
||||
private static List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> clone(
|
||||
List<KinesisRecord> records
|
||||
)
|
||||
{
|
||||
return records.stream().map(KinesisIndexTaskTest::clone).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private static KinesisRecord createRecord(String partitionId, String sequenceNumber, ByteEntity entity)
|
||||
private static KinesisRecord createRecord(String partitionId, String sequenceNumber, KinesisRecordEntity entity)
|
||||
{
|
||||
return new KinesisRecord(STREAM, partitionId, sequenceNumber, Collections.singletonList(entity));
|
||||
}
|
||||
|
@ -1697,7 +1699,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
maxRowsPerSegment = 2;
|
||||
maxRecordsPerPoll = 1;
|
||||
maxBytesPerPoll = 1_000_000;
|
||||
List<OrderedPartitionableRecord<String, String, ByteEntity>> records =
|
||||
List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> records =
|
||||
clone(SINGLE_PARTITION_RECORDS);
|
||||
|
||||
recordSupplier.assign(EasyMock.anyObject());
|
||||
|
@ -2148,7 +2150,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString());
|
||||
EasyMock.expectLastCall().anyTimes();
|
||||
|
||||
List<OrderedPartitionableRecord<String, String, ByteEntity>> eosRecord = ImmutableList.of(
|
||||
List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> eosRecord = ImmutableList.of(
|
||||
new OrderedPartitionableRecord<>(STREAM, SHARD_ID1, KinesisSequenceNumber.END_OF_SHARD_MARKER, null)
|
||||
);
|
||||
|
||||
|
@ -2454,6 +2456,18 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
return task.getRunner().getStatus() == SeekableStreamIndexTaskRunner.Status.READING;
|
||||
}
|
||||
|
||||
private static KinesisRecordEntity kjb(
|
||||
String timestamp,
|
||||
String dim1,
|
||||
String dim2,
|
||||
String dimLong,
|
||||
String dimFloat,
|
||||
String met1
|
||||
)
|
||||
{
|
||||
return new KinesisRecordEntity(new Record().withData(jb(timestamp, dim1, dim2, dimLong, dimFloat, met1).getBuffer()));
|
||||
}
|
||||
|
||||
@JsonTypeName("index_kinesis")
|
||||
private static class TestableKinesisIndexTask extends KinesisIndexTask
|
||||
{
|
||||
|
@ -2497,15 +2511,15 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
/**
|
||||
* Utility class to keep the test code more readable.
|
||||
*/
|
||||
private static class KinesisRecord extends OrderedPartitionableRecord<String, String, ByteEntity>
|
||||
private static class KinesisRecord extends OrderedPartitionableRecord<String, String, KinesisRecordEntity>
|
||||
{
|
||||
private final List<ByteEntity> data;
|
||||
private final List<KinesisRecordEntity> data;
|
||||
|
||||
public KinesisRecord(
|
||||
String stream,
|
||||
String partitionId,
|
||||
String sequenceNumber,
|
||||
List<ByteEntity> data
|
||||
List<KinesisRecordEntity> data
|
||||
)
|
||||
{
|
||||
super(stream, partitionId, sequenceNumber, data);
|
||||
|
@ -2514,7 +2528,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
|
||||
@Nonnull
|
||||
@Override
|
||||
public List<ByteEntity> getData()
|
||||
public List<KinesisRecordEntity> getData()
|
||||
{
|
||||
return data;
|
||||
}
|
||||
|
|
|
@ -39,6 +39,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
|
||||
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
@ -99,34 +100,26 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
new Record().withData(jb("2012", "g", "y", "10", "20.0", "1.0")).withSequenceNumber("8"),
|
||||
new Record().withData(jb("2011", "h", "y", "10", "20.0", "1.0")).withSequenceNumber("9")
|
||||
);
|
||||
private static final List<OrderedPartitionableRecord<String, String, ByteEntity>> ALL_RECORDS = ImmutableList.<OrderedPartitionableRecord<String, String, ByteEntity>>builder()
|
||||
.addAll(SHARD0_RECORDS.stream()
|
||||
.map(x -> new OrderedPartitionableRecord<>(
|
||||
STREAM,
|
||||
SHARD_ID0,
|
||||
x.getSequenceNumber(),
|
||||
Collections
|
||||
.singletonList(
|
||||
new ByteEntity(
|
||||
x.getData()))
|
||||
))
|
||||
.collect(
|
||||
Collectors
|
||||
.toList()))
|
||||
.addAll(SHARD1_RECORDS.stream()
|
||||
.map(x -> new OrderedPartitionableRecord<>(
|
||||
STREAM,
|
||||
SHARD_ID1,
|
||||
x.getSequenceNumber(),
|
||||
Collections
|
||||
.singletonList(
|
||||
new ByteEntity(
|
||||
x.getData()))
|
||||
))
|
||||
.collect(
|
||||
Collectors
|
||||
.toList()))
|
||||
.build();
|
||||
private static final List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> ALL_RECORDS = ImmutableList.<OrderedPartitionableRecord<String, String, KinesisRecordEntity>>builder()
|
||||
.addAll(SHARD0_RECORDS.stream()
|
||||
.map(x -> new OrderedPartitionableRecord<>(
|
||||
STREAM,
|
||||
SHARD_ID0,
|
||||
x.getSequenceNumber(),
|
||||
Collections.singletonList(new KinesisRecordEntity(new Record().withData(new ByteEntity(x.getData()).getBuffer())))
|
||||
))
|
||||
.collect(
|
||||
Collectors
|
||||
.toList()))
|
||||
.addAll(SHARD1_RECORDS.stream()
|
||||
.map(x -> new OrderedPartitionableRecord<>(
|
||||
STREAM,
|
||||
SHARD_ID1,
|
||||
x.getSequenceNumber(),
|
||||
Collections.singletonList(new KinesisRecordEntity(new Record().withData(new ByteEntity(x.getData()).getBuffer())))
|
||||
))
|
||||
.collect(Collectors.toList()))
|
||||
.build();
|
||||
|
||||
|
||||
private static ByteBuffer jb(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1)
|
||||
|
@ -316,7 +309,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
}
|
||||
|
||||
// filter out EOS markers
|
||||
private static List<OrderedPartitionableRecord<String, String, ByteEntity>> cleanRecords(List<OrderedPartitionableRecord<String, String, ByteEntity>> records)
|
||||
private static List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> cleanRecords(List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> records)
|
||||
{
|
||||
return records.stream()
|
||||
.filter(x -> !x.getSequenceNumber()
|
||||
|
@ -398,7 +391,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
List<OrderedPartitionableRecord<String, String, ByteEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
POLL_TIMEOUT_MILLIS));
|
||||
|
||||
verifyAll();
|
||||
|
@ -457,7 +450,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
}
|
||||
Assert.assertFalse(recordSupplier.isAnyFetchActive());
|
||||
|
||||
List<OrderedPartitionableRecord<String, String, ByteEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
POLL_TIMEOUT_MILLIS));
|
||||
|
||||
verifyAll();
|
||||
|
@ -531,7 +524,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
List<OrderedPartitionableRecord<String, String, ByteEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
POLL_TIMEOUT_MILLIS));
|
||||
|
||||
verifyAll();
|
||||
|
@ -687,7 +680,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
OrderedPartitionableRecord<String, String, ByteEntity> firstRecord = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0);
|
||||
OrderedPartitionableRecord<String, String, KinesisRecordEntity> firstRecord = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0);
|
||||
|
||||
Assert.assertEquals(
|
||||
ALL_RECORDS.get(7),
|
||||
|
@ -705,7 +698,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
}
|
||||
|
||||
|
||||
OrderedPartitionableRecord<String, String, ByteEntity> record2 = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0);
|
||||
OrderedPartitionableRecord<String, String, KinesisRecordEntity> record2 = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0);
|
||||
|
||||
Assert.assertEquals(ALL_RECORDS.get(9), record2);
|
||||
// only one partition in this test. second results come from getRecordsResult0, which has SHARD0_LAG_MILLIS
|
||||
|
@ -776,7 +769,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
List<OrderedPartitionableRecord<String, String, ByteEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
POLL_TIMEOUT_MILLIS));
|
||||
|
||||
verifyAll();
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.indexing.kinesis;
|
||||
|
||||
import com.amazonaws.services.kinesis.model.Record;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
|
@ -27,7 +28,6 @@ import org.apache.druid.client.indexing.SamplerResponse;
|
|||
import org.apache.druid.client.indexing.SamplerSpec;
|
||||
import org.apache.druid.common.aws.AWSCredentialsConfig;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.FloatDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
|
@ -37,6 +37,7 @@ import org.apache.druid.data.input.impl.LongDimensionSchema;
|
|||
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.StringInputRowParser;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorIOConfig;
|
||||
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec;
|
||||
import org.apache.druid.indexing.overlord.sampler.InputSourceSampler;
|
||||
|
@ -63,6 +64,7 @@ import org.junit.Assert;
|
|||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
|
@ -99,7 +101,7 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
|
|||
|
||||
private final KinesisRecordSupplier recordSupplier = mock(KinesisRecordSupplier.class);
|
||||
|
||||
private static List<OrderedPartitionableRecord<String, String, ByteEntity>> generateRecords(String stream)
|
||||
private static List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> generateRecords(String stream)
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new OrderedPartitionableRecord<>(stream, "1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
|
@ -115,9 +117,9 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
|
|||
stream,
|
||||
"1",
|
||||
"6",
|
||||
Collections.singletonList(new ByteEntity(StringUtils.toUtf8("unparseable")))
|
||||
Collections.singletonList(new KinesisRecordEntity(new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable")))))
|
||||
),
|
||||
new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(new ByteEntity(StringUtils.toUtf8("{}"))))
|
||||
new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(new KinesisRecordEntity(new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("{}"))))))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -428,19 +430,19 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
|
|||
Assert.assertFalse(it.hasNext());
|
||||
}
|
||||
|
||||
private static List<ByteEntity> jb(String ts, String dim1, String dim2, String dimLong, String dimFloat, String met1)
|
||||
private static List<KinesisRecordEntity> jb(String ts, String dim1, String dim2, String dimLong, String dimFloat, String met1)
|
||||
{
|
||||
try {
|
||||
return Collections.singletonList(new ByteEntity(new ObjectMapper().writeValueAsBytes(
|
||||
return Collections.singletonList(new KinesisRecordEntity(new Record().withData(ByteBuffer.wrap(new ObjectMapper().writeValueAsBytes(
|
||||
ImmutableMap.builder()
|
||||
.put("timestamp", ts)
|
||||
.put("dim1", dim1)
|
||||
.put("dim2", dim2)
|
||||
.put("dimLong", dimLong)
|
||||
.put("dimFloat", dimFloat)
|
||||
.put("met1", met1)
|
||||
.build()
|
||||
)));
|
||||
.put("timestamp", ts)
|
||||
.put("dim1", dim1)
|
||||
.put("dim2", dim2)
|
||||
.put("dimLong", dimLong)
|
||||
.put("dimFloat", dimFloat)
|
||||
.put("met1", met1)
|
||||
.build()
|
||||
)))));
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
|
|
|
@ -27,12 +27,12 @@ import com.google.common.collect.ImmutableSet;
|
|||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import org.apache.druid.data.input.InputFormat;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexer.TaskLocation;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TaskInfoProvider;
|
||||
|
@ -5656,7 +5656,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
}
|
||||
|
||||
@Override
|
||||
protected RecordSupplier<String, String, ByteEntity> setupRecordSupplier()
|
||||
protected RecordSupplier<String, String, KinesisRecordEntity> setupRecordSupplier()
|
||||
{
|
||||
return supervisorRecordSupplier;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue