11package feast .ingestion ;
22
3- import static feast .specs .ImportJobSpecsProto .SourceSpec .SourceType .KAFKA ;
4-
53import com .google .cloud .bigquery .BigQueryOptions ;
64import com .google .common .collect .Sets ;
75import com .google .protobuf .util .JsonFormat ;
86import feast .core .FeatureSetProto .FeatureSetSpec ;
9- import feast .core .FeatureSetProto .FeatureSetSpec .Builder ;
7+ import feast .core .SourceProto .Source ;
8+ import feast .core .SourceProto .Source .SourceType ;
9+ import feast .core .StoreProto .Store ;
10+ import feast .core .StoreProto .Store .Subscription ;
1011import feast .ingestion .options .ImportJobPipelineOptions ;
1112import feast .ingestion .transform .ReadFeatureRow ;
1213import feast .ingestion .transform .ToFeatureRowExtended ;
13- import feast .ingestion .transform .WriteFeaturesTransform ;
1414import feast .ingestion .util .StorageUtil ;
15- import feast .specs .ImportJobSpecsProto .ImportJobSpecs ;
16- import feast .specs .ImportJobSpecsProto .SourceSpec .SourceType ;
17- import feast .specs .StorageSpecProto .StorageSpec ;
1815import java .io .IOException ;
1916import java .net .URISyntaxException ;
20- import java .util .ArrayList ;
2117import java .util .HashMap ;
2218import java .util .List ;
2319import java .util .Map ;
3228import org .apache .kafka .common .PartitionInfo ;
3329import org .apache .kafka .common .TopicPartition ;
3430
35- @ SuppressWarnings ("WeakerAccess" )
3631@ Slf4j
3732public class ImportJob {
38-
39-
4033 /**
41- * Create and run a Beam pipeline from command line arguments.
34+ * Create and run a Beam pipeline with PipelineOptions passed as a list of string arguments.
4235 *
4336 * <p>The arguments will be passed to Beam {@code PipelineOptionsFactory} to create {@code
4437 * ImportJobPipelineOptions}.
@@ -67,22 +60,38 @@ public static PipelineResult runPipeline(String[] args) throws IOException, URIS
6760 * @throws IOException if importJobSpecsUri is not accessible
6861 */
6962 public static PipelineResult runPipeline (ImportJobPipelineOptions pipelineOptions )
70- throws IOException , URISyntaxException {
63+ throws IOException {
7164 pipelineOptions =
7265 PipelineOptionsValidator .validate (ImportJobPipelineOptions .class , pipelineOptions );
7366 Pipeline pipeline = Pipeline .create (pipelineOptions );
7467
75- for (String featureSetSpecJson : pipelineOptions .getFeatureSetSpecJson ()) {
76- Builder builder = FeatureSetSpec .newBuilder ();
77- JsonFormat .parser ().merge (featureSetSpecJson , builder );
78- FeatureSetSpec featureSetSpec = builder .build ();
79-
80- // TODO: Setup storage, set Kafka consumer group to latest offset during pipeline setup
68+ for (String storeJson : pipelineOptions .getStoreJson ()) {
69+ Store .Builder storeBuilder = Store .newBuilder ();
70+ JsonFormat .parser ().merge (storeJson , storeBuilder );
71+ Store store = storeBuilder .build ();
72+
73+ for (Subscription subscription : store .getSubscriptionsList ()) {
74+ // TODO: handle version ranges and keyword (e.g. latest) in subscription
75+
76+ for (String featureSetSpecJson : pipelineOptions .getFeatureSetSpecJson ()) {
77+ FeatureSetSpec .Builder featureSetSpecBuilder = FeatureSetSpec .newBuilder ();
78+ JsonFormat .parser ().merge (featureSetSpecJson , featureSetSpecBuilder );
79+ FeatureSetSpec featureSetSpec = featureSetSpecBuilder .build ();
80+
81+ if (subscription .getName ().equalsIgnoreCase (featureSetSpec .getName ())
82+ && subscription
83+ .getVersion ()
84+ .equalsIgnoreCase (String .valueOf (featureSetSpec .getVersion ()))) {
85+ setupSource (featureSetSpec .getSource ());
86+ setupStore (store , featureSetSpec );
87+ }
8188
82- pipeline
83- .apply ("Read FeatureRow" , new ReadFeatureRow (featureSetSpec ))
84- .apply ("Create FeatureRowExtended from FeatureRow" , new ToFeatureRowExtended ());
85- //.apply("Write FeatureRowExtended", new WriteFeaturesTransform(featureSetSpec));
89+ pipeline
90+ .apply ("Read FeatureRow" , new ReadFeatureRow (featureSetSpec ))
91+ .apply ("Create FeatureRowExtended from FeatureRow" , new ToFeatureRowExtended ());
92+ // .apply("Write FeatureRowExtended", new WriteFeaturesTransform(featureSetSpec));
93+ }
94+ }
8695 }
8796
8897 return pipeline .run ();
@@ -98,87 +107,81 @@ public static PipelineResult runPipeline(ImportJobPipelineOptions pipelineOption
98107 * <p>For example, when using BigQuery as the storage backend, this method ensures that, given a
99108 * list of features, the corresponding BigQuery dataset and table are created.
100109 *
101- * @param importJobSpecs import job specification, refer to {@code ImportJobSpecs .proto}
110+ * @param store Store specification, refer to {@code feast.core.Store .proto}
102111 */
103- private static void setupStorage ( ImportJobSpecs importJobSpecs ) {
104- StorageSpec sinkStorageSpec = importJobSpecs . getSinkStorageSpec ();
105- String storageSpecType = sinkStorageSpec . getType ();
106-
107- switch ( storageSpecType ) {
108- case " BIGQUERY" :
112+ private static void setupStore ( Store store , FeatureSetSpec featureSetSpec ) {
113+ switch ( store . getType ()) {
114+ case REDIS :
115+ StorageUtil . checkRedisConnection ( store . getRedisConfig ());
116+ break ;
117+ case BIGQUERY :
109118 StorageUtil .setupBigQuery (
110- importJobSpecs . getSinkStorageSpec () ,
111- importJobSpecs . getEntitySpec (),
112- importJobSpecs . getFeatureSpecsList (),
119+ featureSetSpec ,
120+ store . getBigqueryConfig (). getProjectId (),
121+ store . getBigqueryConfig (). getDatasetId (),
113122 BigQueryOptions .getDefaultInstance ().getService ());
114123 break ;
115- case "REDIS" :
116- StorageUtil .checkRedisConnection (sinkStorageSpec );
117- break ;
118124 default :
119- throw new IllegalArgumentException (
120- String .format (
121- "Unsupported type of sinkStorageSpec: \" %s\" . Only REDIS and BIGQUERY are supported in Feast 0.2" ,
122- storageSpecType ));
125+ throw new UnsupportedOperationException (
126+ String .format ("Store type: %s not implemented yet" , store .getType ()));
123127 }
124128 }
125129
126130 /**
127- * Manually sets the consumer group offset for this job's consumer group to the offset at the time
128- * at which we provision the ingestion job.
131+ * TODO: Update documentation
132+ *
133+ * <p>Manually sets the consumer group offset for this job's consumer group to the offset at the
134+ * time at which we provision the ingestion job.
129135 *
130136 * <p>This is necessary because the setup time for certain runners (e.g. Dataflow) might cause the
131137 * worker to miss the messages that were emitted into the stream prior to the workers being ready.
132- *
133- * @param importJobSpecs import job specification, refer to {@code ImportJobSpecs.proto}
134138 */
135- private static void setupConsumerGroupOffset (ImportJobSpecs importJobSpecs ) {
136- SourceType sourceType = importJobSpecs .getSourceSpec ().getType ();
137- switch (sourceType ) {
138- case KAFKA :
139- String consumerGroupId = String .format ("feast-import-job-%s" , importJobSpecs .getJobId ());
140- Properties consumerProperties = new Properties ();
141- consumerProperties .setProperty ("group.id" , consumerGroupId );
142- consumerProperties .setProperty (
143- "bootstrap.servers" ,
144- importJobSpecs .getSourceSpec ().getOptionsOrThrow ("bootstrapServers" ));
145- consumerProperties .setProperty (
146- "key.deserializer" , "org.apache.kafka.common.serialization.ByteArrayDeserializer" );
147- consumerProperties .setProperty (
148- "value.deserializer" , "org.apache.kafka.common.serialization.ByteArrayDeserializer" );
149- KafkaConsumer kafkaConsumer = new KafkaConsumer (consumerProperties );
150-
151- String [] topics = importJobSpecs .getSourceSpec ().getOptionsOrThrow ("topics" ).split ("," );
152- long timestamp = System .currentTimeMillis ();
153- Map <TopicPartition , Long > timestampsToSearch = new HashMap <>();
154- for (String topic : topics ) {
155- List <PartitionInfo > partitionInfos = kafkaConsumer .partitionsFor (topic );
156- for (PartitionInfo partitionInfo : partitionInfos ) {
157- TopicPartition topicPartition = new TopicPartition (topic , partitionInfo .partition ());
158- timestampsToSearch .put (topicPartition , timestamp );
159- }
160- }
161- Map <TopicPartition , OffsetAndTimestamp > offsets =
162- kafkaConsumer .offsetsForTimes (timestampsToSearch );
163-
164- kafkaConsumer .assign (offsets .keySet ());
165- kafkaConsumer .poll (1000 );
166- kafkaConsumer .commitSync ();
167-
168- offsets .forEach (
169- (topicPartition , offset ) -> {
170- if (offset != null ) {
171- kafkaConsumer .seek (topicPartition , offset .offset ());
172- } else {
173- kafkaConsumer .seekToBeginning (Sets .newHashSet (topicPartition ));
174- }
175- });
176- return ;
177- default :
178- throw new IllegalArgumentException (
179- String .format (
180- "Unsupported type of sourceSpec: \" %s\" . Only KAFKA is supported in Feast 0.2" ,
181- sourceType ));
139+ private static void setupSource (Source source ) {
140+ if (!source .getType ().equals (SourceType .KAFKA )) {
141+ throw new UnsupportedOperationException (
142+ String .format ("Source type: %s not implemented yet" , source .getType ()));
182143 }
144+
145+ if (!source .getOptions ().containsKey ("consumerGroupId" )) {
146+ log .warn (
147+ "consumerGroupId is not provided in the source options. Import job will not be able to resume correctly from existing checkpoint." );
148+ return ;
149+ }
150+
151+ Properties consumerProperties = new Properties ();
152+ consumerProperties .setProperty ("group.id" , source .getOptionsOrThrow ("consumerGroupId" ));
153+ consumerProperties .setProperty (
154+ "bootstrap.servers" , source .getOptionsOrThrow ("bootstrapServers" ));
155+ consumerProperties .setProperty (
156+ "key.deserializer" , "org.apache.kafka.common.serialization.ByteArrayDeserializer" );
157+ consumerProperties .setProperty (
158+ "value.deserializer" , "org.apache.kafka.common.serialization.ByteArrayDeserializer" );
159+ KafkaConsumer kafkaConsumer = new KafkaConsumer (consumerProperties );
160+
161+ String [] topics = source .getOptionsOrThrow ("topics" ).split ("," );
162+ long timestamp = System .currentTimeMillis ();
163+ Map <TopicPartition , Long > timestampsToSearch = new HashMap <>();
164+ for (String topic : topics ) {
165+ List <PartitionInfo > partitionInfos = kafkaConsumer .partitionsFor (topic );
166+ for (PartitionInfo partitionInfo : partitionInfos ) {
167+ TopicPartition topicPartition = new TopicPartition (topic , partitionInfo .partition ());
168+ timestampsToSearch .put (topicPartition , timestamp );
169+ }
170+ }
171+ Map <TopicPartition , OffsetAndTimestamp > offsets =
172+ kafkaConsumer .offsetsForTimes (timestampsToSearch );
173+
174+ kafkaConsumer .assign (offsets .keySet ());
175+ kafkaConsumer .poll (1000 );
176+ kafkaConsumer .commitSync ();
177+
178+ offsets .forEach (
179+ (topicPartition , offset ) -> {
180+ if (offset != null ) {
181+ kafkaConsumer .seek (topicPartition , offset .offset ());
182+ } else {
183+ kafkaConsumer .seekToBeginning (Sets .newHashSet (topicPartition ));
184+ }
185+ });
183186 }
184187}
0 commit comments