Skip to content

Commit 1ff6035

Browse files
authored
Merge pull request #23 from tims/kafkaio
Kafka IO fixes
2 parents bb1b767 + 47b6d03 commit 1ff6035

File tree

6 files changed

+164
-189
lines changed

6 files changed

+164
-189
lines changed

ingestion/src/main/java/feast/ingestion/ImportJob.java

Lines changed: 2 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -18,8 +18,6 @@
1818
package feast.ingestion;
1919

2020
import com.google.api.services.bigquery.model.TableRow;
21-
import com.google.api.services.dataflow.DataflowScopes;
22-
import com.google.auth.oauth2.GoogleCredentials;
2321
import com.google.inject.Guice;
2422
import com.google.inject.Inject;
2523
import com.google.inject.Injector;
@@ -109,15 +107,11 @@ public static void main(String[] args) {
109107

110108
public static PipelineResult mainWithResult(String[] args) {
111109
log.info("Arguments: " + Arrays.toString(args));
112-
ImportJobOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(ImportJobOptions.class);
110+
ImportJobOptions options =
111+
PipelineOptionsFactory.fromArgs(args).withValidation().as(ImportJobOptions.class);
113112
if (options.getJobName().isEmpty()) {
114113
options.setJobName(generateName());
115114
}
116-
try {
117-
options.setGcpCredential(GoogleCredentials.getApplicationDefault().createScoped(DataflowScopes.all()));
118-
} catch (IOException e) {
119-
log.error("Exception while setting gcp credential manually : ", e.getMessage());
120-
}
121115
log.info("options: " + options.toString());
122116
ImportSpec importSpec = new ImportSpecSupplier(options).get();
123117
Injector injector =

ingestion/src/main/java/feast/ingestion/deserializer/FeatureRowDeserializer.java

Lines changed: 13 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -2,32 +2,29 @@
22

33
import com.google.protobuf.InvalidProtocolBufferException;
44
import feast.types.FeatureRowProto.FeatureRow;
5+
import java.util.Map;
56
import org.apache.kafka.common.errors.SerializationException;
67
import org.apache.kafka.common.serialization.Deserializer;
78

8-
import java.util.Map;
9-
109
/**
1110
* Deserializer for Kafka to deserialize Protocol Buffers messages
1211
*
1312
* @param <FeatureRow> Protobuf message type
1413
*/
1514
public class FeatureRowDeserializer implements Deserializer<FeatureRow> {
1615

17-
@Override
18-
public void configure(Map configs, boolean isKey) {
19-
}
16+
@Override
17+
public void configure(Map configs, boolean isKey) {}
2018

21-
@Override
22-
public FeatureRow deserialize(String topic, byte[] data) {
23-
try {
24-
return FeatureRow.parseFrom(data);
25-
} catch (InvalidProtocolBufferException e) {
26-
throw new SerializationException("Error deserializing FeatureRow from Protobuf message", e);
27-
}
19+
@Override
20+
public FeatureRow deserialize(String topic, byte[] data) {
21+
try {
22+
return FeatureRow.parseFrom(data);
23+
} catch (InvalidProtocolBufferException e) {
24+
throw new SerializationException("Error deserializing FeatureRow from Protobuf message", e);
2825
}
26+
}
2927

30-
@Override
31-
public void close() {
32-
}
33-
}
28+
@Override
29+
public void close() {}
30+
}
Lines changed: 15 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -1,33 +1,31 @@
11
package feast.ingestion.deserializer;
22

33
import com.google.protobuf.InvalidProtocolBufferException;
4-
import feast.types.FeatureRowProto.*;
4+
import feast.types.FeatureRowProto.FeatureRowKey;
5+
import java.util.Map;
56
import org.apache.kafka.common.errors.SerializationException;
67
import org.apache.kafka.common.serialization.Deserializer;
78

8-
import java.util.Map;
9-
109
/**
1110
* Deserializer for Kafka to deserialize Protocol Buffers messages
1211
*
1312
* @param <FeatureRowKey> Protobuf message type
1413
*/
1514
public class FeatureRowKeyDeserializer implements Deserializer<FeatureRowKey> {
1615

17-
@Override
18-
public void configure(Map configs, boolean isKey) {
19-
}
16+
@Override
17+
public void configure(Map configs, boolean isKey) {}
2018

21-
@Override
22-
public FeatureRowKey deserialize(String topic, byte[] data) {
23-
try {
24-
return FeatureRowKey.parseFrom(data);
25-
} catch (InvalidProtocolBufferException e) {
26-
throw new SerializationException("Error deserializing FeatureRowKey from Protobuf message", e);
27-
}
19+
@Override
20+
public FeatureRowKey deserialize(String topic, byte[] data) {
21+
try {
22+
return FeatureRowKey.parseFrom(data);
23+
} catch (InvalidProtocolBufferException e) {
24+
throw new SerializationException(
25+
"Error deserializing FeatureRowKey from Protobuf message", e);
2826
}
27+
}
2928

30-
@Override
31-
public void close() {
32-
}
33-
}
29+
@Override
30+
public void close() {}
31+
}

ingestion/src/main/java/feast/ingestion/options/ImportJobOptions.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,16 +19,14 @@
1919

2020
import com.google.auto.service.AutoService;
2121
import java.util.Collections;
22-
import org.apache.beam.runners.flink.FlinkPipelineOptions;
23-
import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
2422
import org.apache.beam.sdk.metrics.MetricsSink;
2523
import org.apache.beam.sdk.options.Default;
2624
import org.apache.beam.sdk.options.Description;
2725
import org.apache.beam.sdk.options.PipelineOptions;
2826
import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
2927
import org.apache.beam.sdk.options.Validation.Required;
3028

31-
public interface ImportJobOptions extends PipelineOptions, FlinkPipelineOptions, GcpOptions {
29+
public interface ImportJobOptions extends PipelineOptions {
3230
@Description("Import spec yaml file path")
3331
@Required(groups = {"importSpec"})
3432
String getImportSpecYamlFile();

ingestion/src/main/java/feast/ingestion/transform/FeatureRowKafkaIO.java

Lines changed: 56 additions & 59 deletions
Original file line numberDiff line numberDiff line change
@@ -17,84 +17,81 @@
1717

1818
package feast.ingestion.transform;
1919

20-
import com.google.common.base.Preconditions;
21-
import com.google.common.base.Strings;
20+
import static com.google.common.base.Preconditions.checkArgument;
21+
2222
import feast.ingestion.deserializer.FeatureRowDeserializer;
2323
import feast.ingestion.deserializer.FeatureRowKeyDeserializer;
24+
import feast.options.Options;
2425
import feast.options.OptionsParser;
2526
import feast.specs.ImportSpecProto.ImportSpec;
2627
import feast.types.FeatureRowProto.FeatureRow;
2728
import feast.types.FeatureRowProto.FeatureRowKey;
29+
import java.util.ArrayList;
30+
import java.util.Arrays;
31+
import java.util.List;
32+
import javax.validation.constraints.NotEmpty;
2833
import org.apache.beam.sdk.io.kafka.KafkaIO;
2934
import org.apache.beam.sdk.io.kafka.KafkaRecord;
3035
import org.apache.beam.sdk.transforms.DoFn;
3136
import org.apache.beam.sdk.transforms.ParDo;
3237
import org.apache.beam.sdk.values.PCollection;
3338
import org.apache.beam.sdk.values.PInput;
3439

35-
import java.util.ArrayList;
36-
import java.util.Arrays;
37-
import java.util.List;
38-
39-
import static com.google.common.base.Preconditions.checkArgument;
40-
4140
public class FeatureRowKafkaIO {
41+
static final String KAFKA_TYPE = "kafka";
4242

43-
static final String KAFKA_TYPE = "kafka";
43+
/**
44+
* Transform for reading {@link feast.types.FeatureRowProto.FeatureRow FeatureRow} proto messages
45+
* from kafka one or more kafka topics.
46+
*/
47+
public static Read read(ImportSpec importSpec) {
48+
return new Read(importSpec);
49+
}
4450

51+
public static class KafkaReadOptions implements Options {
52+
@NotEmpty public String server;
53+
@NotEmpty public String topics;
54+
}
4555

46-
/**
47-
* Transform for reading {@link feast.types.FeatureRowProto.FeatureRow FeatureRow}
48-
* proto messages from kafka one or more kafka topics.
49-
*
50-
*/
51-
public static Read read(ImportSpec importSpec) {
52-
return new Read(importSpec);
53-
}
56+
public static class Read extends FeatureIO.Read {
5457

55-
public static class Read extends FeatureIO.Read {
58+
private ImportSpec importSpec;
5659

57-
private ImportSpec importSpec;
58-
59-
private Read(ImportSpec importSpec) {
60-
this.importSpec = importSpec;
61-
}
62-
63-
@Override
64-
public PCollection<FeatureRow> expand(PInput input) {
65-
66-
checkArgument(importSpec.getType().equals(KAFKA_TYPE));
67-
68-
String bootstrapServer = importSpec.getOptionsMap().get("server");
69-
70-
Preconditions.checkArgument(
71-
!Strings.isNullOrEmpty(bootstrapServer), "kafka bootstrap server must be set");
72-
73-
String topics = importSpec.getOptionsMap().get("topics");
74-
75-
Preconditions.checkArgument(
76-
!Strings.isNullOrEmpty(topics), "kafka topic(s) must be set");
77-
78-
List<String> topicsList = new ArrayList<>(Arrays.asList(topics.split(",")));
79-
80-
KafkaIO.Read<FeatureRowKey, FeatureRow> kafkaIOReader = KafkaIO.<FeatureRowKey, FeatureRow>read()
81-
.withBootstrapServers(bootstrapServer)
82-
.withTopics(topicsList)
83-
.withKeyDeserializer(FeatureRowKeyDeserializer.class)
84-
.withValueDeserializer(FeatureRowDeserializer.class);
85-
86-
PCollection<KafkaRecord<FeatureRowKey, FeatureRow>> featureRowRecord = input.getPipeline().apply(kafkaIOReader);
60+
private Read(ImportSpec importSpec) {
61+
this.importSpec = importSpec;
62+
}
8763

88-
PCollection<FeatureRow> featureRow = featureRowRecord.apply(
89-
ParDo.of(
90-
new DoFn<KafkaRecord<FeatureRowKey, FeatureRow>, FeatureRow>() {
91-
@ProcessElement
92-
public void processElement(ProcessContext processContext) {
93-
KafkaRecord<FeatureRowKey, FeatureRow> record = processContext.element();
94-
processContext.output(record.getKV().getValue());
95-
}
96-
}));
97-
return featureRow;
98-
}
64+
@Override
65+
public PCollection<FeatureRow> expand(PInput input) {
66+
67+
checkArgument(importSpec.getType().equals(KAFKA_TYPE));
68+
69+
KafkaReadOptions options =
70+
OptionsParser.parse(importSpec.getOptionsMap(), KafkaReadOptions.class);
71+
72+
List<String> topicsList = new ArrayList<>(Arrays.asList(options.topics.split(",")));
73+
74+
KafkaIO.Read<FeatureRowKey, FeatureRow> kafkaIOReader =
75+
KafkaIO.<FeatureRowKey, FeatureRow>read()
76+
.withBootstrapServers(options.server)
77+
.withTopics(topicsList)
78+
.withKeyDeserializer(FeatureRowKeyDeserializer.class)
79+
.withValueDeserializer(FeatureRowDeserializer.class);
80+
81+
PCollection<KafkaRecord<FeatureRowKey, FeatureRow>> featureRowRecord =
82+
input.getPipeline().apply(kafkaIOReader);
83+
84+
PCollection<FeatureRow> featureRow =
85+
featureRowRecord.apply(
86+
ParDo.of(
87+
new DoFn<KafkaRecord<FeatureRowKey, FeatureRow>, FeatureRow>() {
88+
@ProcessElement
89+
public void processElement(ProcessContext processContext) {
90+
KafkaRecord<FeatureRowKey, FeatureRow> record = processContext.element();
91+
processContext.output(record.getKV().getValue());
92+
}
93+
}));
94+
return featureRow;
9995
}
96+
}
10097
}

0 commit comments

Comments
 (0)