-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-1897] Deltastreamer source for AWS S3 #3433
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
dec655b
[HUDI-1897] Deltastreamer source for AWS S3
data-storyteller bd9b7dd
Handle spark read failure and address minor comments
codope ee8fbce
Add prefix constants for props
codope 5ffdc82
Minor tweaks, making file existence check configurable, turned off by…
vinothchandar File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
137 changes: 137 additions & 0 deletions
137
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,137 @@ | ||
| /* | ||
| * 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.hudi.utilities.sources; | ||
|
|
||
| import org.apache.hudi.DataSourceReadOptions; | ||
| import org.apache.hudi.DataSourceUtils; | ||
| import org.apache.hudi.common.config.TypedProperties; | ||
| import org.apache.hudi.common.fs.FSUtils; | ||
| import org.apache.hudi.common.util.Option; | ||
| import org.apache.hudi.common.util.collection.Pair; | ||
| import org.apache.hudi.utilities.schema.SchemaProvider; | ||
| import org.apache.hudi.utilities.sources.helpers.IncrSourceHelper; | ||
|
|
||
| import org.apache.hadoop.fs.FileSystem; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.log4j.LogManager; | ||
| import org.apache.log4j.Logger; | ||
| import org.apache.spark.api.java.JavaSparkContext; | ||
| import org.apache.spark.sql.DataFrameReader; | ||
| import org.apache.spark.sql.Dataset; | ||
| import org.apache.spark.sql.Row; | ||
| import org.apache.spark.sql.SparkSession; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.ArrayList; | ||
| import java.util.Collections; | ||
| import java.util.List; | ||
|
|
||
| import static org.apache.hudi.utilities.sources.HoodieIncrSource.Config.DEFAULT_NUM_INSTANTS_PER_FETCH; | ||
| import static org.apache.hudi.utilities.sources.HoodieIncrSource.Config.DEFAULT_READ_LATEST_INSTANT_ON_MISSING_CKPT; | ||
| import static org.apache.hudi.utilities.sources.HoodieIncrSource.Config.DEFAULT_SOURCE_FILE_FORMAT; | ||
| import static org.apache.hudi.utilities.sources.HoodieIncrSource.Config.HOODIE_SRC_BASE_PATH; | ||
| import static org.apache.hudi.utilities.sources.HoodieIncrSource.Config.NUM_INSTANTS_PER_FETCH; | ||
| import static org.apache.hudi.utilities.sources.HoodieIncrSource.Config.READ_LATEST_INSTANT_ON_MISSING_CKPT; | ||
| import static org.apache.hudi.utilities.sources.HoodieIncrSource.Config.SOURCE_FILE_FORMAT; | ||
| import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.S3_PREFIX; | ||
|
|
||
| /** | ||
| * This source will use the S3 events meta information from hoodie table generate by {@link S3EventsSource}. | ||
| */ | ||
| public class S3EventsHoodieIncrSource extends HoodieIncrSource { | ||
|
|
||
| private static final Logger LOG = LogManager.getLogger(S3EventsHoodieIncrSource.class); | ||
|
|
||
| static class Config { | ||
| // control whether we do existence check for files before consuming them | ||
| static final String ENABLE_EXISTS_CHECK = "hoodie.deltastreamer.source.s3incr.check.file.exists"; | ||
| static final Boolean DEFAULT_ENABLE_EXISTS_CHECK = false; | ||
| } | ||
|
|
||
| public S3EventsHoodieIncrSource( | ||
| TypedProperties props, | ||
| JavaSparkContext sparkContext, | ||
| SparkSession sparkSession, | ||
| SchemaProvider schemaProvider) { | ||
| super(props, sparkContext, sparkSession, schemaProvider); | ||
| } | ||
|
|
||
| @Override | ||
| public Pair<Option<Dataset<Row>>, String> fetchNextBatch(Option<String> lastCkptStr, long sourceLimit) { | ||
| DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(HOODIE_SRC_BASE_PATH)); | ||
| String srcPath = props.getString(HOODIE_SRC_BASE_PATH); | ||
| int numInstantsPerFetch = props.getInteger(NUM_INSTANTS_PER_FETCH, DEFAULT_NUM_INSTANTS_PER_FETCH); | ||
| boolean readLatestOnMissingCkpt = props.getBoolean( | ||
| READ_LATEST_INSTANT_ON_MISSING_CKPT, DEFAULT_READ_LATEST_INSTANT_ON_MISSING_CKPT); | ||
|
|
||
| // Use begin Instant if set and non-empty | ||
| Option<String> beginInstant = | ||
| lastCkptStr.isPresent() | ||
| ? lastCkptStr.get().isEmpty() ? Option.empty() : lastCkptStr | ||
| : Option.empty(); | ||
|
|
||
| Pair<String, String> instantEndpts = | ||
| IncrSourceHelper.calculateBeginAndEndInstants( | ||
| sparkContext, srcPath, numInstantsPerFetch, beginInstant, readLatestOnMissingCkpt); | ||
|
|
||
| if (instantEndpts.getKey().equals(instantEndpts.getValue())) { | ||
| LOG.warn("Already caught up. Begin Checkpoint was :" + instantEndpts.getKey()); | ||
| return Pair.of(Option.empty(), instantEndpts.getKey()); | ||
| } | ||
|
|
||
| // Do incremental pull. Set end instant if available. | ||
| DataFrameReader metaReader = sparkSession.read().format("org.apache.hudi") | ||
| .option(DataSourceReadOptions.QUERY_TYPE().key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL()) | ||
| .option(DataSourceReadOptions.BEGIN_INSTANTTIME().key(), instantEndpts.getLeft()) | ||
| .option(DataSourceReadOptions.END_INSTANTTIME().key(), instantEndpts.getRight()); | ||
| Dataset<Row> source = metaReader.load(srcPath); | ||
| // Extract distinct file keys from s3 meta hoodie table | ||
| final List<Row> cloudMetaDf = source | ||
| .filter("s3.object.size > 0") | ||
| .select("s3.bucket.name", "s3.object.key") | ||
| .distinct() | ||
| .collectAsList(); | ||
| // Create S3 paths | ||
| final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, Config.DEFAULT_ENABLE_EXISTS_CHECK); | ||
| List<String> cloudFiles = new ArrayList<>(); | ||
| for (Row row : cloudMetaDf) { | ||
| // construct file path, row index 0 refers to bucket and 1 refers to key | ||
| String bucket = row.getString(0); | ||
| String filePath = S3_PREFIX + bucket + "/" + row.getString(1); | ||
| if (checkExists) { | ||
| FileSystem fs = FSUtils.getFs(S3_PREFIX + bucket, sparkSession.sparkContext().hadoopConfiguration()); | ||
| try { | ||
| if (fs.exists(new Path(filePath))) { | ||
| cloudFiles.add(filePath); | ||
| } | ||
| } catch (IOException e) { | ||
| LOG.error(String.format("Error while checking path exists for %s ", filePath), e); | ||
| } | ||
| } else { | ||
| cloudFiles.add(filePath); | ||
| } | ||
| } | ||
| String fileFormat = props.getString(SOURCE_FILE_FORMAT, DEFAULT_SOURCE_FILE_FORMAT); | ||
| Option<Dataset<Row>> dataset = Option.empty(); | ||
| if (!cloudFiles.isEmpty()) { | ||
| dataset = Option.of(sparkSession.read().format(fileFormat).load(cloudFiles.toArray(new String[0]))); | ||
| } | ||
| return Pair.of(dataset, instantEndpts.getRight()); | ||
| } | ||
| } |
87 changes: 87 additions & 0 deletions
87
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsSource.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,87 @@ | ||
| /* | ||
| * 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.hudi.utilities.sources; | ||
|
|
||
| import org.apache.hudi.common.config.TypedProperties; | ||
| import org.apache.hudi.common.util.Option; | ||
| import org.apache.hudi.common.util.collection.Pair; | ||
| import org.apache.hudi.utilities.schema.SchemaProvider; | ||
| import org.apache.hudi.utilities.sources.helpers.S3EventsMetaSelector; | ||
|
|
||
| import com.amazonaws.services.sqs.AmazonSQS; | ||
| import com.amazonaws.services.sqs.model.Message; | ||
| import org.apache.spark.api.java.JavaSparkContext; | ||
| import org.apache.spark.sql.Dataset; | ||
| import org.apache.spark.sql.Encoders; | ||
| import org.apache.spark.sql.Row; | ||
| import org.apache.spark.sql.SparkSession; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
|
|
||
| /** | ||
| * This source provides capability to create the hudi table for S3 events metadata (eg. S3 | ||
| * put events data). It will use the SQS for receiving the object key events. This can be useful | ||
| * to check S3 files activity over time. The hudi table created by this source is consumed by | ||
| * {@link S3EventsHoodieIncrSource} to apply changes to the hudi table corresponding to user data. | ||
| */ | ||
| public class S3EventsSource extends RowSource { | ||
|
|
||
| private final S3EventsMetaSelector pathSelector; | ||
| private final List<Message> processedMessages = new ArrayList<>(); | ||
| AmazonSQS sqs; | ||
|
|
||
| public S3EventsSource( | ||
| TypedProperties props, | ||
| JavaSparkContext sparkContext, | ||
| SparkSession sparkSession, | ||
| SchemaProvider schemaProvider) { | ||
| super(props, sparkContext, sparkSession, schemaProvider); | ||
| this.pathSelector = S3EventsMetaSelector.createSourceSelector(props); | ||
| this.sqs = this.pathSelector.createAmazonSqsClient(); | ||
| } | ||
|
|
||
| /** | ||
| * Fetches next events from the queue. | ||
| * | ||
| * @param lastCkptStr The last checkpoint instant string, empty if first run. | ||
| * @param sourceLimit Limit on the size of data to fetch. For {@link S3EventsSource}, | ||
| * {@link org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.Config#S3_SOURCE_QUEUE_MAX_MESSAGES_PER_BATCH} is used. | ||
| * @return A pair of dataset of event records and the next checkpoint instant string | ||
| */ | ||
| @Override | ||
| public Pair<Option<Dataset<Row>>, String> fetchNextBatch(Option<String> lastCkptStr, long sourceLimit) { | ||
| Pair<List<String>, String> selectPathsWithLatestSqsMessage = | ||
| pathSelector.getNextEventsFromQueue(sqs, lastCkptStr, processedMessages); | ||
| if (selectPathsWithLatestSqsMessage.getLeft().isEmpty()) { | ||
| return Pair.of(Option.empty(), selectPathsWithLatestSqsMessage.getRight()); | ||
| } else { | ||
| Dataset<String> eventRecords = sparkSession.createDataset(selectPathsWithLatestSqsMessage.getLeft(), Encoders.STRING()); | ||
| return Pair.of( | ||
| Option.of(sparkSession.read().json(eventRecords)), | ||
| selectPathsWithLatestSqsMessage.getRight()); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void onCommit(String lastCkptStr) { | ||
| pathSelector.deleteProcessedMessages(sqs, pathSelector.queueUrl, processedMessages); | ||
| processedMessages.clear(); | ||
| } | ||
| } |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we are not bundling this. So we should ensure to document --jars to add this for to work during runtime cc @codope
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ack.