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
Original file line number Diff line number Diff line change
Expand Up @@ -241,6 +241,10 @@ public SparkContext getOrCreateSparkContext(final SparkConf sparkConfig) {
return HELPER.getOrCreateSparkContext(sparkConfig);
}

public SparkContext createSparkContext(final SparkConf sparkConfig) {
return HELPER.getOrCreateSparkContext(sparkConfig, true);
}

public void retryAssertion(final Runnable assertion) {
retryAssertion(assertion, () -> {});
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
* Copyright 2008-present MongoDB, Inc.
*
* Licensed 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 com.mongodb.spark.sql.connector.utils;

import static java.lang.String.format;
import static org.junit.jupiter.api.Assertions.assertEquals;

import com.mongodb.spark.sql.connector.mongodb.MongoSparkConnectorTestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.spark.SparkConf;
import org.junit.jupiter.api.Test;

class SparkHadoopUtilsTest extends MongoSparkConnectorTestCase {
private static final String SPARK_HADOOP_FS_AZURE_ACCOUNT_AUTH_TYPE =
"spark.hadoop.fs.azure.account.auth.type";
private static final String FS_AZURE_ACCOUNT_AUTH_TYPE = "fs.azure.account.auth.type";
private static final String FS_AZURE_ACCOUNT_AUTH_TYPE_VALUE = "OAUTH";

@Test
void testDefaultHadoopConfiguration() {
SparkConf sparkConf = getSparkConf();
sparkConf.set(FS_AZURE_ACCOUNT_AUTH_TYPE, FS_AZURE_ACCOUNT_AUTH_TYPE_VALUE);

Configuration hadoopConfiguration = createSparkContext(sparkConf).hadoopConfiguration();
assertConfigValue(hadoopConfiguration, FS_AZURE_ACCOUNT_AUTH_TYPE, null);
}

@Test
void testSparkHadoopPrefix() {
SparkConf sparkConf = getSparkConf();
sparkConf.set(SPARK_HADOOP_FS_AZURE_ACCOUNT_AUTH_TYPE, FS_AZURE_ACCOUNT_AUTH_TYPE_VALUE);

Configuration hadoopConfiguration = createSparkContext(sparkConf).hadoopConfiguration();

assertConfigValue(
hadoopConfiguration, FS_AZURE_ACCOUNT_AUTH_TYPE, FS_AZURE_ACCOUNT_AUTH_TYPE_VALUE);
}

@Test
void testFSPrefix() {
SparkConf sparkConf = getSparkConf();
sparkConf.set(FS_AZURE_ACCOUNT_AUTH_TYPE, FS_AZURE_ACCOUNT_AUTH_TYPE_VALUE);
createSparkContext(sparkConf);

Configuration hadoopConfiguration = SparkHadoopUtils.createHadoopConfiguration();

assertConfigValue(
hadoopConfiguration, FS_AZURE_ACCOUNT_AUTH_TYPE, FS_AZURE_ACCOUNT_AUTH_TYPE_VALUE);
}

@Test
void testFSPrefixTakesPrecedence() {
SparkConf sparkConf = getSparkConf();
sparkConf.set(SPARK_HADOOP_FS_AZURE_ACCOUNT_AUTH_TYPE, "DEFAULT VALUE");
sparkConf.set(FS_AZURE_ACCOUNT_AUTH_TYPE, FS_AZURE_ACCOUNT_AUTH_TYPE_VALUE);
createSparkContext(sparkConf);

Configuration hadoopConfiguration = SparkHadoopUtils.createHadoopConfiguration();

assertConfigValue(
hadoopConfiguration, FS_AZURE_ACCOUNT_AUTH_TYPE, FS_AZURE_ACCOUNT_AUTH_TYPE_VALUE);
}

/**
* Assert that a hadoop configuration option has the expected value.
* @param hadoopConf configuration to query
* @param key key to look up
* @param expected expected value.
*/
private void assertConfigValue(
final Configuration hadoopConf, final String key, final String expected) {
assertEquals(expected, hadoopConf.get(key), format("Mismatch in expected value of %s", key));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import com.mongodb.spark.sql.connector.config.ReadConfig;
import com.mongodb.spark.sql.connector.schema.BsonDocumentToRowConverter;
import com.mongodb.spark.sql.connector.schema.InferSchema;
import org.apache.spark.SparkContext;
import org.apache.spark.sql.connector.read.InputPartition;
import org.apache.spark.sql.connector.read.streaming.ContinuousPartitionReaderFactory;
import org.apache.spark.sql.connector.read.streaming.ContinuousStream;
Expand Down Expand Up @@ -65,10 +64,8 @@ final class MongoContinuousStream implements ContinuousStream {
() ->
"Mongo Continuous streams require a schema to be explicitly defined, unless using publish full document only.");
this.schema = schema;
this.mongoOffsetStore = new MongoOffsetStore(
SparkContext.getOrCreate().hadoopConfiguration(),
checkpointLocation,
MongoOffset.getInitialOffset(readConfig));
this.mongoOffsetStore =
new MongoOffsetStore(checkpointLocation, MongoOffset.getInitialOffset(readConfig));
this.readConfig = readConfig;
this.bsonDocumentToRowConverter = new BsonDocumentToRowConverter(schema, readConfig);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import com.mongodb.spark.sql.connector.schema.BsonDocumentToRowConverter;
import com.mongodb.spark.sql.connector.schema.InferSchema;
import java.time.Instant;
import org.apache.spark.SparkContext;
import org.apache.spark.sql.connector.read.InputPartition;
import org.apache.spark.sql.connector.read.PartitionReaderFactory;
import org.apache.spark.sql.connector.read.streaming.MicroBatchStream;
Expand Down Expand Up @@ -68,10 +67,8 @@ final class MongoMicroBatchStream implements MicroBatchStream {
() ->
"Mongo micro batch streams require a schema to be explicitly defined, unless using publish full document only.");
this.schema = schema;
this.mongoOffsetStore = new MongoOffsetStore(
SparkContext.getOrCreate().hadoopConfiguration(),
checkpointLocation,
MongoOffset.getInitialOffset(readConfig));
this.mongoOffsetStore =
new MongoOffsetStore(checkpointLocation, MongoOffset.getInitialOffset(readConfig));
this.readConfig = readConfig;
this.bsonDocumentToRowConverter = new BsonDocumentToRowConverter(schema, readConfig);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,11 @@
import static java.lang.String.format;

import com.mongodb.spark.sql.connector.exceptions.ConfigException;
import com.mongodb.spark.sql.connector.utils.SparkHadoopUtils;
import java.io.IOException;
import java.net.URI;
import java.nio.charset.StandardCharsets;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
Expand All @@ -41,14 +41,13 @@ final class MongoOffsetStore {
/**
* Instantiates a new Mongo offset store.
*
* @param conf the conf
* @param checkpointLocation the checkpoint location for offsets
* @param offset the offset
*/
MongoOffsetStore(
final Configuration conf, final String checkpointLocation, final MongoOffset offset) {
MongoOffsetStore(final String checkpointLocation, final MongoOffset offset) {
try {
this.fs = FileSystem.get(URI.create(checkpointLocation), conf);
this.fs = FileSystem.get(
URI.create(checkpointLocation), SparkHadoopUtils.createHadoopConfiguration());
} catch (IOException e) {
throw new ConfigException(
format("Unable to initialize the MongoOffsetStore: %s", checkpointLocation), e);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* Copyright 2008-present MongoDB, Inc.
*
* Licensed 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 com.mongodb.spark.sql.connector.utils;

import java.util.Arrays;
import org.apache.hadoop.conf.Configuration;
import org.apache.spark.SparkConf;
import org.apache.spark.SparkContext;

/**
* An internal SparkHadoopUtils helper for creating Hadoop configuration
*/
public final class SparkHadoopUtils {

private static final String FILESYSTEM_CONFIGURATION_PREFIX = "fs.";

/**
* A helper that creates a Hadoop Configuration that includes all {@code fs.} prefixed configurations.
* <p>
* For cluster wide hadoop configurations users use the {@code spark.hadoop} prefix for configurations.
* However, existing azure documentation doesn't include the {@code spark.hadoop} prefix and just uses {@code fs.azure.}.
* This helper sets any filesystem configuration prefixed with {@code fs.} on top of the created
* {@code sparkContext.hadoopConfiguration()} configuration.
*
* @see <a href="https://learn.microsoft.com/en-us/azure/databricks/connect/storage/azure-storage">Azure-storage docs</a>
* @return the Hadoop Configuration
*/
public static Configuration createHadoopConfiguration() {
SparkContext sparkContext = SparkContext.getOrCreate();
Configuration hadoopConfiguration = sparkContext.hadoopConfiguration();

SparkConf sparkConf = sparkContext.getConf();
Arrays.stream(sparkConf.getAll())
.filter(kv -> kv._1.startsWith(FILESYSTEM_CONFIGURATION_PREFIX))
.forEach(kv -> hadoopConfiguration.set(kv._1, kv._2));

return hadoopConfiguration;
}

private SparkHadoopUtils() {}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,21 @@
/*
* Copyright 2008-present MongoDB, Inc.
*
* Licensed 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.
*
*/

@ApiStatus.Internal
package com.mongodb.spark.sql.connector.utils;

import org.jetbrains.annotations.ApiStatus;