Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 8 additions & 0 deletions hudi-utilities/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -402,6 +402,14 @@
<scope>test</scope>
</dependency>

<!-- AWS Services -->
<!-- https://mvnrepository.com/artifact/com.amazonaws/aws-java-sdk-sqs -->
<dependency>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-sqs</artifactId>
<version>${aws.sdk.version}</version>
Copy link
Member

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

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ack.

</dependency>

<!-- Hive - Test -->
<dependency>
<groupId>${hive.groupid}</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,45 +42,51 @@ public class HoodieIncrSource extends RowSource {

private static final Logger LOG = LogManager.getLogger(HoodieIncrSource.class);

protected static class Config {
static class Config {

/**
* {@value #HOODIE_SRC_BASE_PATH} is the base-path for the source Hoodie table.
*/
private static final String HOODIE_SRC_BASE_PATH = "hoodie.deltastreamer.source.hoodieincr.path";
static final String HOODIE_SRC_BASE_PATH = "hoodie.deltastreamer.source.hoodieincr.path";

/**
* {@value #NUM_INSTANTS_PER_FETCH} allows the max number of instants whose changes can be incrementally fetched.
*/
private static final String NUM_INSTANTS_PER_FETCH = "hoodie.deltastreamer.source.hoodieincr.num_instants";
private static final Integer DEFAULT_NUM_INSTANTS_PER_FETCH = 1;
static final String NUM_INSTANTS_PER_FETCH = "hoodie.deltastreamer.source.hoodieincr.num_instants";
static final Integer DEFAULT_NUM_INSTANTS_PER_FETCH = 1;

/**
* {@value #HOODIE_SRC_PARTITION_FIELDS} specifies partition fields that needs to be added to source table after
* parsing _hoodie_partition_path.
*/
private static final String HOODIE_SRC_PARTITION_FIELDS = "hoodie.deltastreamer.source.hoodieincr.partition.fields";
static final String HOODIE_SRC_PARTITION_FIELDS = "hoodie.deltastreamer.source.hoodieincr.partition.fields";

/**
* {@value #HOODIE_SRC_PARTITION_EXTRACTORCLASS} PartitionValueExtractor class to extract partition fields from
* _hoodie_partition_path.
*/
private static final String HOODIE_SRC_PARTITION_EXTRACTORCLASS =
static final String HOODIE_SRC_PARTITION_EXTRACTORCLASS =
"hoodie.deltastreamer.source.hoodieincr.partition.extractor.class";
private static final String DEFAULT_HOODIE_SRC_PARTITION_EXTRACTORCLASS =
static final String DEFAULT_HOODIE_SRC_PARTITION_EXTRACTORCLASS =
SlashEncodedDayPartitionValueExtractor.class.getCanonicalName();

/**
* {@value #READ_LATEST_INSTANT_ON_MISSING_CKPT} allows delta-streamer to incrementally fetch from latest committed
* instant when checkpoint is not provided.
*/
private static final String READ_LATEST_INSTANT_ON_MISSING_CKPT =
static final String READ_LATEST_INSTANT_ON_MISSING_CKPT =
"hoodie.deltastreamer.source.hoodieincr.read_latest_on_missing_ckpt";
private static final Boolean DEFAULT_READ_LATEST_INSTANT_ON_MISSING_CKPT = false;
static final Boolean DEFAULT_READ_LATEST_INSTANT_ON_MISSING_CKPT = false;

/**
* {@value #SOURCE_FILE_FORMAT} is passed to the reader while loading dataset. Default value is parquet.
*/
static final String SOURCE_FILE_FORMAT = "hoodie.deltastreamer.source.hoodieincr.file.format";
static final String DEFAULT_SOURCE_FILE_FORMAT = "parquet";
}

public HoodieIncrSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
SchemaProvider schemaProvider) {
SchemaProvider schemaProvider) {
super(props, sparkContext, sparkSession, schemaProvider);
}

Expand Down Expand Up @@ -123,10 +129,10 @@ public Pair<Option<Dataset<Row>>, String> fetchNextBatch(Option<String> lastCkpt

/*
* log.info("Partition Fields are : (" + partitionFields + "). Initial Source Schema :" + source.schema());
*
*
* StructType newSchema = new StructType(source.schema().fields()); for (String field : partitionFields) { newSchema
* = newSchema.add(field, DataTypes.StringType, true); }
*
*
* /** Validates if the commit time is sane and also generates Partition fields from _hoodie_partition_path if
* configured
*
Expand All @@ -139,7 +145,7 @@ public Pair<Option<Dataset<Row>>, String> fetchNextBatch(Option<String> lastCkpt
* "#partition-fields != #partition-values-extracted"); List<Object> rowObjs = new
* ArrayList<>(scala.collection.JavaConversions.seqAsJavaList(row.toSeq())); rowObjs.addAll(partitionVals); return
* RowFactory.create(rowObjs.toArray()); } return row; }, RowEncoder.apply(newSchema));
*
*
* log.info("Validated Source Schema :" + validated.schema());
*/

Expand Down
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());
}
}
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();
}
}
Loading