1
0

[HUDI-1897] Deltastreamer source for AWS S3 (#3433)

- Added two sources for two stage pipeline. a. S3EventsSource that fetches events from SQS and ingests to a meta hoodie table. b. S3EventsHoodieIncrSource reads S3 events from this meta hoodie table, fetches actual objects from S3 and ingests to sink hoodie table. 
- Added selectors to assist in S3EventsSource. 

Co-authored-by: Satish M <84978833+satishmittal1111@users.noreply.github.com>
Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Sagar Sumit
2021-08-14 17:55:10 +05:30
committed by GitHub
parent 9056c68744
commit 5cc96e85c1
12 changed files with 1348 additions and 13 deletions

View File

@@ -0,0 +1,112 @@
/*
* 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.model.HoodieRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter;
import org.apache.hudi.utilities.testutils.sources.AbstractCloudObjectsSourceTestBase;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.Path;
import org.apache.spark.api.java.JavaRDD;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.List;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.Config.S3_SOURCE_QUEUE_REGION;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.Config.S3_SOURCE_QUEUE_URL;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.Config.S3_SOURCE_QUEUE_FS;
import static org.junit.jupiter.api.Assertions.assertEquals;
/**
* Basic tests for {@link S3EventsSource}.
*/
public class TestS3EventsSource extends AbstractCloudObjectsSourceTestBase {
@BeforeEach
public void setup() throws Exception {
super.setup();
this.dfsRoot = dfsBasePath + "/parquetFiles";
this.fileSuffix = ".parquet";
dfs.mkdirs(new Path(dfsRoot));
}
@AfterEach
public void teardown() throws Exception {
super.teardown();
}
/**
* Runs the test scenario of reading data from the source.
*
* @throws IOException
*/
@Test
public void testReadingFromSource() throws IOException {
SourceFormatAdapter sourceFormatAdapter = new SourceFormatAdapter(prepareCloudObjectSource());
// 1. Extract without any checkpoint => (no data available)
generateMessageInQueue(null);
assertEquals(
Option.empty(),
sourceFormatAdapter.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch());
// 2. Extract without any checkpoint => (adding new file)
generateMessageInQueue("1");
// Test fetching Avro format
InputBatch<JavaRDD<GenericRecord>> fetch1 =
sourceFormatAdapter.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE);
assertEquals(1, fetch1.getBatch().get().count());
// 3. Produce new data, extract new data
generateMessageInQueue("2");
// Test fetching Avro format
InputBatch<JavaRDD<GenericRecord>> fetch2 =
sourceFormatAdapter.fetchNewDataInAvroFormat(
Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE);
assertEquals(1, fetch2.getBatch().get().count());
GenericRecord s3 = (GenericRecord) fetch2.getBatch().get().rdd().first().get("s3");
GenericRecord s3Object = (GenericRecord) s3.get("object");
assertEquals("2.parquet", s3Object.get("key").toString());
}
@Override
public Source prepareCloudObjectSource() {
TypedProperties props = new TypedProperties();
props.setProperty(S3_SOURCE_QUEUE_URL, sqsUrl);
props.setProperty(S3_SOURCE_QUEUE_REGION, regionName);
props.setProperty(S3_SOURCE_QUEUE_FS, "hdfs");
S3EventsSource dfsSource = new S3EventsSource(props, jsc, sparkSession, null);
dfsSource.sqs = this.sqs;
return dfsSource;
}
@Override
public void writeNewDataToFile(List<HoodieRecord> records, Path path) throws IOException {
Helpers.saveParquetToDFS(Helpers.toGenericRecords(records), path);
}
}

View File

@@ -0,0 +1,226 @@
/*
* 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.helpers;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.utilities.testutils.CloudObjectTestUtils;
import com.amazonaws.services.sqs.AmazonSQS;
import com.amazonaws.services.sqs.model.Message;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.json.JSONObject;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.MockitoAnnotations;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.Config.S3_SOURCE_QUEUE_REGION;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.Config.S3_SOURCE_QUEUE_URL;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.S3_FILE_PATH;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.S3_FILE_SIZE;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.S3_MODEL_EVENT_TIME;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.S3_PREFIX;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.SQS_ATTR_APPROX_MESSAGES;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.SQS_MODEL_EVENT_RECORDS;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.SQS_MODEL_MESSAGE;
import static org.apache.hudi.utilities.testutils.CloudObjectTestUtils.deleteMessagesInQueue;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class TestCloudObjectsSelector extends HoodieClientTestHarness {
static final String REGION_NAME = "us-east-1";
TypedProperties props;
String sqsUrl;
@Mock
AmazonSQS sqs;
@Mock
private CloudObjectsSelector cloudObjectsSelector;
@BeforeEach
void setUp() {
initSparkContexts();
initPath();
initFileSystem();
MockitoAnnotations.initMocks(this);
props = new TypedProperties();
sqsUrl = "test-queue";
props.setProperty(S3_SOURCE_QUEUE_URL, sqsUrl);
props.setProperty(S3_SOURCE_QUEUE_REGION, REGION_NAME);
}
@AfterEach
public void teardown() throws Exception {
Mockito.reset(cloudObjectsSelector);
cleanupResources();
}
@ParameterizedTest
@ValueSource(classes = {CloudObjectsSelector.class})
public void testSqsQueueAttributesShouldReturnsRequiredAttribute(Class<?> clazz) {
CloudObjectsSelector selector =
(CloudObjectsSelector) ReflectionUtils.loadClass(clazz.getName(), props);
// setup the queue attributes
CloudObjectTestUtils.setMessagesInQueue(sqs, null);
// test the return values
Map<String, String> queueAttributes = selector.getSqsQueueAttributes(sqs, sqsUrl);
assertEquals(1, queueAttributes.size());
// ApproximateNumberOfMessages is a required queue attribute for Cloud object selector
assertEquals("0", queueAttributes.get(SQS_ATTR_APPROX_MESSAGES));
}
@ParameterizedTest
@ValueSource(classes = {CloudObjectsSelector.class})
public void testFileAttributesFromRecordShouldReturnsExpectOutput(Class<?> clazz)
throws IOException {
CloudObjectsSelector selector =
(CloudObjectsSelector) ReflectionUtils.loadClass(clazz.getName(), props);
// setup s3 record
String bucket = "test-bucket";
String key = "test/year=test1/month=test2/day=test3/part-foo-bar.snappy.parquet";
String s3Records =
"{\n \"Type\" : \"Notification\",\n \"MessageId\" : \"1\",\n \"TopicArn\" : \"arn:aws:sns:foo:123:"
+ "foo-bar\",\n \"Subject\" : \"Amazon S3 Notification\",\n \"Message\" : \"{\\\"Records\\\":"
+ "[{\\\"eventVersion\\\":\\\"2.1\\\",\\\"eventSource\\\":\\\"aws:s3\\\",\\\"awsRegion\\\":\\\"us"
+ "-west-2\\\",\\\"eventTime\\\":\\\"2021-07-27T09:05:36.755Z\\\",\\\"eventName\\\":\\\"ObjectCreated"
+ ":Copy\\\",\\\"userIdentity\\\":{\\\"principalId\\\":\\\"AWS:test\\\"},\\\"requestParameters\\\":"
+ "{\\\"sourceIPAddress\\\":\\\"0.0.0.0\\\"},\\\"responseElements\\\":{\\\"x-amz-request-id\\\":\\\""
+ "test\\\",\\\"x-amz-id-2\\\":\\\"foobar\\\"},\\\"s3\\\":{\\\"s3SchemaVersion\\\":\\\"1.0\\\",\\\""
+ "configurationId\\\":\\\"foobar\\\",\\\"bucket\\\":{\\\"name\\\":\\\""
+ bucket
+ "\\\",\\\"ownerIdentity\\\":{\\\"principalId\\\":\\\"foo\\\"},\\\"arn\\\":\\\"arn:aws:s3:::foo\\\"}"
+ ",\\\"object\\\":{\\\"key\\\":\\\""
+ key
+ "\\\",\\\"size\\\":123,\\\"eTag\\\":\\\"test\\\",\\\"sequencer\\\":\\\"1\\\"}}}]}\"}";
JSONObject messageBody = new JSONObject(s3Records);
Map<String, Object> messageMap = new HashMap<>();
if (messageBody.has(SQS_MODEL_MESSAGE)) {
ObjectMapper mapper = new ObjectMapper();
messageMap =
(Map<String, Object>) mapper.readValue(messageBody.getString(SQS_MODEL_MESSAGE), Map.class);
}
List<Map<String, Object>> records = (List<Map<String, Object>>) messageMap.get(SQS_MODEL_EVENT_RECORDS);
// test the return values
Map<String, Object> fileAttributes =
selector.getFileAttributesFromRecord(new JSONObject(records.get(0)));
assertEquals(3, fileAttributes.size());
assertEquals(123L, (long) fileAttributes.get(S3_FILE_SIZE));
assertEquals(S3_PREFIX + bucket + "/" + key, fileAttributes.get(S3_FILE_PATH));
assertEquals(1627376736755L, (long) fileAttributes.get(S3_MODEL_EVENT_TIME));
}
@ParameterizedTest
@ValueSource(classes = {CloudObjectsSelector.class})
public void testCreateListPartitionsReturnsExpectedSetOfBatch(Class<?> clazz) {
CloudObjectsSelector selector =
(CloudObjectsSelector) ReflectionUtils.loadClass(clazz.getName(), props);
// setup lists
List<Message> testSingleList = new ArrayList<>();
testSingleList.add(new Message().addAttributesEntry("id", "1"));
testSingleList.add(new Message().addAttributesEntry("id", "2"));
testSingleList.add(new Message().addAttributesEntry("id", "3"));
testSingleList.add(new Message().addAttributesEntry("id", "4"));
testSingleList.add(new Message().addAttributesEntry("id", "5"));
List<Message> expectedFirstList = new ArrayList<>();
expectedFirstList.add(new Message().addAttributesEntry("id", "1"));
expectedFirstList.add(new Message().addAttributesEntry("id", "2"));
List<Message> expectedSecondList = new ArrayList<>();
expectedSecondList.add(new Message().addAttributesEntry("id", "3"));
expectedSecondList.add(new Message().addAttributesEntry("id", "4"));
List<Message> expectedFinalList = new ArrayList<>();
expectedFinalList.add(new Message().addAttributesEntry("id", "5"));
// test the return values
List<List<Message>> partitionedList = selector.createListPartitions(testSingleList, 2);
assertEquals(3, partitionedList.size());
assertEquals(expectedFirstList, partitionedList.get(0));
assertEquals(expectedSecondList, partitionedList.get(1));
assertEquals(expectedFinalList, partitionedList.get(2));
}
@ParameterizedTest
@ValueSource(classes = {CloudObjectsSelector.class})
public void testCreateListPartitionsReturnsEmptyIfBatchSizeIsZero(Class<?> clazz) {
CloudObjectsSelector selector =
(CloudObjectsSelector) ReflectionUtils.loadClass(clazz.getName(), props);
// setup lists
List<Message> testSingleList = new ArrayList<>();
testSingleList.add(new Message().addAttributesEntry("id", "1"));
testSingleList.add(new Message().addAttributesEntry("id", "2"));
// test the return values
List<List<Message>> partitionedList = selector.createListPartitions(testSingleList, 0);
assertEquals(0, partitionedList.size());
}
@ParameterizedTest
@ValueSource(classes = {CloudObjectsSelector.class})
public void testOnCommitDeleteProcessedMessages(Class<?> clazz) {
CloudObjectsSelector selector =
(CloudObjectsSelector) ReflectionUtils.loadClass(clazz.getName(), props);
// setup lists
List<Message> testSingleList = new ArrayList<>();
testSingleList.add(
new Message()
.addAttributesEntry("MessageId", "1")
.addAttributesEntry("ReceiptHandle", "1"));
testSingleList.add(
new Message()
.addAttributesEntry("MessageId", "2")
.addAttributesEntry("ReceiptHandle", "1"));
deleteMessagesInQueue(sqs);
// test the return values
selector.deleteProcessedMessages(sqs, sqsUrl, testSingleList);
}
}

View File

@@ -0,0 +1,105 @@
/*
* 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.helpers;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.utilities.testutils.CloudObjectTestUtils;
import com.amazonaws.services.sqs.AmazonSQS;
import com.amazonaws.services.sqs.model.Message;
import org.apache.hadoop.fs.Path;
import org.json.JSONObject;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.MockitoAnnotations;
import java.util.ArrayList;
import java.util.List;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.Config.S3_SOURCE_QUEUE_REGION;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.Config.S3_SOURCE_QUEUE_URL;
import static org.apache.hudi.utilities.sources.helpers.TestCloudObjectsSelector.REGION_NAME;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class TestS3EventsMetaSelector extends HoodieClientTestHarness {
TypedProperties props;
String sqsUrl;
@Mock
AmazonSQS sqs;
@Mock
private S3EventsMetaSelector s3EventsMetaSelector;
@BeforeEach
void setUp() {
initSparkContexts();
initPath();
initFileSystem();
MockitoAnnotations.initMocks(this);
props = new TypedProperties();
sqsUrl = "test-queue";
props.setProperty(S3_SOURCE_QUEUE_URL, sqsUrl);
props.setProperty(S3_SOURCE_QUEUE_REGION, REGION_NAME);
}
@AfterEach
public void teardown() throws Exception {
Mockito.reset(s3EventsMetaSelector);
cleanupResources();
}
@ParameterizedTest
@ValueSource(classes = {S3EventsMetaSelector.class})
public void testNextEventsFromQueueShouldReturnsEventsFromQueue(Class<?> clazz) {
S3EventsMetaSelector selector = (S3EventsMetaSelector) ReflectionUtils.loadClass(clazz.getName(), props);
// setup s3 record
String bucket = "test-bucket";
String key = "part-foo-bar.snappy.parquet";
Path path = new Path(bucket, key);
CloudObjectTestUtils.setMessagesInQueue(sqs, path);
List<Message> processed = new ArrayList<>();
// test the return values
Pair<List<String>, String> eventFromQueue =
selector.getNextEventsFromQueue(sqs, Option.empty(), processed);
assertEquals(1, eventFromQueue.getLeft().size());
assertEquals(1, processed.size());
assertEquals(
key,
new JSONObject(eventFromQueue.getLeft().get(0))
.getJSONObject("s3")
.getJSONObject("object")
.getString("key"));
assertEquals("1627376736755", eventFromQueue.getRight());
}
}

View File

@@ -0,0 +1,98 @@
/*
* 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.testutils;
import com.amazonaws.services.sqs.AmazonSQS;
import com.amazonaws.services.sqs.model.DeleteMessageBatchRequest;
import com.amazonaws.services.sqs.model.DeleteMessageBatchResult;
import com.amazonaws.services.sqs.model.GetQueueAttributesRequest;
import com.amazonaws.services.sqs.model.GetQueueAttributesResult;
import com.amazonaws.services.sqs.model.Message;
import com.amazonaws.services.sqs.model.ReceiveMessageRequest;
import com.amazonaws.services.sqs.model.ReceiveMessageResult;
import org.apache.hadoop.fs.Path;
import java.util.ArrayList;
import java.util.List;
import static org.apache.hudi.utilities.sources.helpers.CloudObjectsSelector.SQS_ATTR_APPROX_MESSAGES;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.when;
/**
* Utils Class for unit testing on CloudObject related sources.
*/
public class CloudObjectTestUtils {
/**
* Set a return value for mocked sqs instance. It will add a new messages (s3 Event) and set
* ApproximateNumberOfMessages attribute of the queue.
*
* @param sqs Mocked instance of AmazonSQS
* @param path Absolute Path of file in FileSystem
*/
public static void setMessagesInQueue(AmazonSQS sqs, Path path) {
ReceiveMessageResult receiveMessageResult = new ReceiveMessageResult();
String approximateNumberOfMessages = "0";
if (path != null) {
String body =
"{\n \"Type\" : \"Notification\",\n \"MessageId\" : \"1\",\n \"TopicArn\" : \"arn:aws:sns:foo:123:"
+ "foo-bar\",\n \"Subject\" : \"Amazon S3 Notification\",\n \"Message\" : \"{\\\"Records\\\":"
+ "[{\\\"eventVersion\\\":\\\"2.1\\\",\\\"eventSource\\\":\\\"aws:s3\\\",\\\"awsRegion\\\":\\\"us"
+ "-west-2\\\",\\\"eventTime\\\":\\\"2021-07-27T09:05:36.755Z\\\",\\\"eventName\\\":\\\"ObjectCreated"
+ ":Copy\\\",\\\"userIdentity\\\":{\\\"principalId\\\":\\\"AWS:test\\\"},\\\"requestParameters\\\":"
+ "{\\\"sourceIPAddress\\\":\\\"0.0.0.0\\\"},\\\"responseElements\\\":{\\\"x-amz-request-id\\\":\\\""
+ "test\\\",\\\"x-amz-id-2\\\":\\\"foobar\\\"},\\\"s3\\\":{\\\"s3SchemaVersion\\\":\\\"1.0\\\",\\\""
+ "configurationId\\\":\\\"foobar\\\",\\\"bucket\\\":{\\\"name\\\":\\\""
+ path.getParent().toString().replace("hdfs://", "")
+ "\\\",\\\"ownerIdentity\\\":{\\\"principalId\\\":\\\"foo\\\"},\\\"arn\\\":\\\"arn:aws:s3:::foo\\\"}"
+ ",\\\"object\\\":{\\\"key\\\":\\\""
+ path.getName()
+ "\\\",\\\"size\\\":123,\\\"eTag\\\":\\\"test\\\",\\\"sequencer\\\":\\\"1\\\"}}}]}\"}";
Message message = new Message();
message.setReceiptHandle("1");
message.setMessageId("1");
message.setBody(body);
List<Message> messages = new ArrayList<>();
messages.add(message);
receiveMessageResult.setMessages(messages);
approximateNumberOfMessages = "1";
}
when(sqs.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResult);
when(sqs.getQueueAttributes(any(GetQueueAttributesRequest.class)))
.thenReturn(
new GetQueueAttributesResult()
.addAttributesEntry(SQS_ATTR_APPROX_MESSAGES, approximateNumberOfMessages));
}
/**
* Mock the sqs.deleteMessageBatch() method from queue.
*
* @param sqs Mocked instance of AmazonSQS
*/
public static void deleteMessagesInQueue(AmazonSQS sqs) {
when(sqs.deleteMessageBatch(any(DeleteMessageBatchRequest.class)))
.thenReturn(new DeleteMessageBatchResult());
}
}

View File

@@ -0,0 +1,114 @@
/*
* 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.testutils.sources;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
import org.apache.hudi.utilities.sources.Source;
import org.apache.hudi.utilities.testutils.CloudObjectTestUtils;
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
import com.amazonaws.services.sqs.AmazonSQS;
import org.apache.hadoop.fs.Path;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
import java.io.IOException;
import java.util.List;
/**
* An abstract test base for {@link Source} using CloudObjects as the file system.
*/
public abstract class AbstractCloudObjectsSourceTestBase extends UtilitiesTestBase {
protected FilebasedSchemaProvider schemaProvider;
protected String dfsRoot;
protected String fileSuffix;
protected HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
protected boolean useFlattenedSchema = false;
protected String sqsUrl = "test-queue";
protected String regionName = "us-east-1";
@Mock
protected AmazonSQS sqs;
@BeforeAll
public static void initClass() throws Exception {
UtilitiesTestBase.initClass();
}
@AfterAll
public static void cleanupClass() {
UtilitiesTestBase.cleanupClass();
}
@BeforeEach
public void setup() throws Exception {
super.setup();
schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS(), jsc);
MockitoAnnotations.initMocks(this);
}
@AfterEach
public void teardown() throws Exception {
super.teardown();
}
/**
* Prepares the specific {@link Source} to test, by passing in necessary configurations.
*
* @return A {@link Source} using DFS as the file system.
*/
protected abstract Source prepareCloudObjectSource();
/**
* Writes test data, i.e., a {@link List} of {@link HoodieRecord}, to a file on DFS.
*
* @param records Test data.
* @param path The path in {@link Path} of the file to write.
*/
protected abstract void writeNewDataToFile(List<HoodieRecord> records, Path path)
throws IOException;
/**
* Generates a batch of test data and writes the data to a file.
*
* @param filename The name of the file.
* @param instantTime The commit time.
* @param n The number of records to generate.
* @return The file path.
*/
protected Path generateOneFile(String filename, String instantTime, int n) throws IOException {
Path path = new Path(dfsRoot, filename + fileSuffix);
writeNewDataToFile(dataGenerator.generateInserts(instantTime, n, useFlattenedSchema), path);
return path;
}
public void generateMessageInQueue(String filename) {
Path path = null;
if (filename != null) {
path = new Path(dfsRoot, filename + fileSuffix);
}
CloudObjectTestUtils.setMessagesInQueue(sqs, path);
}
}