diff --git a/shims/spark321/src/main/java/org/apache/spark/util/sketch/BloomFilter.java b/shims/spark321/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
new file mode 100644
index 000000000..2a6e270a9
--- /dev/null
+++ b/shims/spark321/src/main/java/org/apache/spark/util/sketch/BloomFilter.java
@@ -0,0 +1,253 @@
+/*
+ * 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.spark.util.sketch;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * A Bloom filter is a space-efficient probabilistic data structure that offers an approximate
+ * containment test with one-sided error: if it claims that an item is contained in it, this
+ * might be in error, but if it claims that an item is not contained in it, then this is
+ * definitely true. Currently supported data types include:
+ *
+ * - {@link Byte}
+ * - {@link Short}
+ * - {@link Integer}
+ * - {@link Long}
+ * - {@link String}
+ *
+ * The false positive probability ({@code FPP}) of a Bloom filter is defined as the probability that
+ * {@linkplain #mightContain(Object)} will erroneously return {@code true} for an object that has
+ * not actually been put in the {@code BloomFilter}.
+ *
+ * The implementation is largely based on the {@code BloomFilter} class from Guava.
+ */
+public abstract class BloomFilter {
+
+ public enum Version {
+ /**
+ * {@code BloomFilter} binary format version 1. All values written in big-endian order:
+ *
+ * - Version number, always 1 (32 bit)
+ * - Number of hash functions (32 bit)
+ * - Total number of words of the underlying bit array (32 bit)
+ * - The words/longs (numWords * 64 bit)
+ *
+ */
+ V1(1);
+
+ private final int versionNumber;
+
+ Version(int versionNumber) {
+ this.versionNumber = versionNumber;
+ }
+
+ int getVersionNumber() {
+ return versionNumber;
+ }
+ }
+
+ /**
+ * Returns the probability that {@linkplain #mightContain(Object)} erroneously return {@code true}
+ * for an object that has not actually been put in the {@code BloomFilter}.
+ *
+ * Ideally, this number should be close to the {@code fpp} parameter passed in
+ * {@linkplain #create(long, double)}, or smaller. If it is significantly higher, it is usually
+ * the case that too many items (more than expected) have been put in the {@code BloomFilter},
+ * degenerating it.
+ */
+ public abstract double expectedFpp();
+
+ /**
+ * Returns the number of bits in the underlying bit array.
+ */
+ public abstract long bitSize();
+
+ /**
+ * Puts an item into this {@code BloomFilter}. Ensures that subsequent invocations of
+ * {@linkplain #mightContain(Object)} with the same item will always return {@code true}.
+ *
+ * @return true if the bloom filter's bits changed as a result of this operation. If the bits
+ * changed, this is definitely the first time {@code object} has been added to the
+ * filter. If the bits haven't changed, this might be the first time {@code object}
+ * has been added to the filter. Note that {@code put(t)} always returns the
+ * opposite result to what {@code mightContain(t)} would have returned at the time
+ * it is called.
+ */
+ public abstract boolean put(Object item);
+
+ /**
+ * A specialized variant of {@link #put(Object)} that only supports {@code String} items.
+ */
+ public abstract boolean putString(String item);
+
+ /**
+ * A specialized variant of {@link #put(Object)} that only supports {@code long} items.
+ */
+ public abstract boolean putLong(long item);
+
+ /**
+ * A specialized variant of {@link #put(Object)} that only supports byte array items.
+ */
+ public abstract boolean putBinary(byte[] item);
+
+ /**
+ * Determines whether a given bloom filter is compatible with this bloom filter. For two
+ * bloom filters to be compatible, they must have the same bit size.
+ *
+ * @param other The bloom filter to check for compatibility.
+ */
+ public abstract boolean isCompatible(BloomFilter other);
+
+ /**
+ * Combines this bloom filter with another bloom filter by performing a bitwise OR of the
+ * underlying data. The mutations happen to this instance. Callers must ensure the
+ * bloom filters are appropriately sized to avoid saturating them.
+ *
+ * @param other The bloom filter to combine this bloom filter with. It is not mutated.
+ * @throws IncompatibleMergeException if {@code isCompatible(other) == false}
+ */
+ public abstract BloomFilter mergeInPlace(BloomFilter other) throws IncompatibleMergeException;
+
+ /**
+ * Combines this bloom filter with another bloom filter by performing a bitwise AND of the
+ * underlying data. The mutations happen to this instance. Callers must ensure the
+ * bloom filters are appropriately sized to avoid saturating them.
+ *
+ * @param other The bloom filter to combine this bloom filter with. It is not mutated.
+ * @throws IncompatibleMergeException if {@code isCompatible(other) == false}
+ */
+ public abstract BloomFilter intersectInPlace(BloomFilter other) throws IncompatibleMergeException;
+
+ /**
+ * Returns {@code true} if the element might have been put in this Bloom filter,
+ * {@code false} if this is definitely not the case.
+ */
+ public abstract boolean mightContain(Object item);
+
+ /**
+ * A specialized variant of {@link #mightContain(Object)} that only tests {@code String} items.
+ */
+ public abstract boolean mightContainString(String item);
+
+ /**
+ * A specialized variant of {@link #mightContain(Object)} that only tests {@code long} items.
+ */
+ public abstract boolean mightContainLong(long item);
+
+ /**
+ * A specialized variant of {@link #mightContain(Object)} that only tests byte array items.
+ */
+ public abstract boolean mightContainBinary(byte[] item);
+
+ /**
+ * Writes out this {@link BloomFilter} to an output stream in binary format. It is the caller's
+ * responsibility to close the stream.
+ */
+ public abstract void writeTo(OutputStream out) throws IOException;
+
+ /**
+ * @return the number of set bits in this {@link BloomFilter}.
+ */
+ public long cardinality() {
+ throw new UnsupportedOperationException("Not implemented");
+ }
+
+ /**
+ * Reads in a {@link BloomFilter} from an input stream. It is the caller's responsibility to close
+ * the stream.
+ */
+ public static BloomFilter readFrom(InputStream in) throws IOException {
+ return BloomFilterImpl.readFrom(in);
+ }
+
+ /**
+ * Computes the optimal k (number of hashes per item inserted in Bloom filter), given the
+ * expected insertions and total number of bits in the Bloom filter.
+ *
+ * See http://en.wikipedia.org/wiki/File:Bloom_filter_fp_probability.svg for the formula.
+ *
+ * @param n expected insertions (must be positive)
+ * @param m total number of bits in Bloom filter (must be positive)
+ */
+ private static int optimalNumOfHashFunctions(long n, long m) {
+ // (m / n) * log(2), but avoid truncation due to division!
+ return Math.max(1, (int) Math.round((double) m / n * Math.log(2)));
+ }
+
+ /**
+ * Computes m (total bits of Bloom filter) which is expected to achieve, for the specified
+ * expected insertions, the required false positive probability.
+ *
+ * See http://en.wikipedia.org/wiki/Bloom_filter#Probability_of_false_positives for the formula.
+ *
+ * @param n expected insertions (must be positive)
+ * @param p false positive rate (must be 0 < p < 1)
+ */
+ private static long optimalNumOfBits(long n, double p) {
+ return (long) (-n * Math.log(p) / (Math.log(2) * Math.log(2)));
+ }
+
+ static final double DEFAULT_FPP = 0.03;
+
+ /**
+ * Creates a {@link BloomFilter} with the expected number of insertions and a default expected
+ * false positive probability of 3%.
+ *
+ * Note that overflowing a {@code BloomFilter} with significantly more elements than specified,
+ * will result in its saturation, and a sharp deterioration of its false positive probability.
+ */
+ public static BloomFilter create(long expectedNumItems) {
+ return create(expectedNumItems, DEFAULT_FPP);
+ }
+
+ /**
+ * Creates a {@link BloomFilter} with the expected number of insertions and expected false
+ * positive probability.
+ *
+ * Note that overflowing a {@code BloomFilter} with significantly more elements than specified,
+ * will result in its saturation, and a sharp deterioration of its false positive probability.
+ */
+ public static BloomFilter create(long expectedNumItems, double fpp) {
+ if (fpp <= 0D || fpp >= 1D) {
+ throw new IllegalArgumentException(
+ "False positive probability must be within range (0.0, 1.0)"
+ );
+ }
+
+ return create(expectedNumItems, optimalNumOfBits(expectedNumItems, fpp));
+ }
+
+ /**
+ * Creates a {@link BloomFilter} with given {@code expectedNumItems} and {@code numBits}, it will
+ * pick an optimal {@code numHashFunctions} which can minimize {@code fpp} for the bloom filter.
+ */
+ public static BloomFilter create(long expectedNumItems, long numBits) {
+ if (expectedNumItems <= 0) {
+ throw new IllegalArgumentException("Expected insertions must be positive");
+ }
+
+ if (numBits <= 0) {
+ throw new IllegalArgumentException("Number of bits must be positive");
+ }
+
+ return new BloomFilterImpl(optimalNumOfHashFunctions(expectedNumItems, numBits), numBits);
+ }
+}
diff --git a/shims/spark321/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java b/shims/spark321/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java
new file mode 100644
index 000000000..b053de8bd
--- /dev/null
+++ b/shims/spark321/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java
@@ -0,0 +1,276 @@
+/*
+ * 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.spark.util.sketch;
+
+import java.io.*;
+
+class BloomFilterImpl extends BloomFilter implements Serializable {
+
+ private int numHashFunctions;
+
+ private BitArray bits;
+
+ BloomFilterImpl(int numHashFunctions, long numBits) {
+ this(new BitArray(numBits), numHashFunctions);
+ }
+
+ private BloomFilterImpl(BitArray bits, int numHashFunctions) {
+ this.bits = bits;
+ this.numHashFunctions = numHashFunctions;
+ }
+
+ private BloomFilterImpl() {}
+
+ @Override
+ public boolean equals(Object other) {
+ if (other == this) {
+ return true;
+ }
+
+ if (other == null || !(other instanceof BloomFilterImpl)) {
+ return false;
+ }
+
+ BloomFilterImpl that = (BloomFilterImpl) other;
+
+ return this.numHashFunctions == that.numHashFunctions && this.bits.equals(that.bits);
+ }
+
+ @Override
+ public int hashCode() {
+ return bits.hashCode() * 31 + numHashFunctions;
+ }
+
+ @Override
+ public double expectedFpp() {
+ return Math.pow((double) bits.cardinality() / bits.bitSize(), numHashFunctions);
+ }
+
+ @Override
+ public long bitSize() {
+ return bits.bitSize();
+ }
+
+ @Override
+ public boolean put(Object item) {
+ if (item instanceof String) {
+ return putString((String) item);
+ } else if (item instanceof byte[]) {
+ return putBinary((byte[]) item);
+ } else {
+ return putLong(Utils.integralToLong(item));
+ }
+ }
+
+ @Override
+ public boolean putString(String item) {
+ return putBinary(Utils.getBytesFromUTF8String(item));
+ }
+
+ @Override
+ public boolean putBinary(byte[] item) {
+ int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, 0);
+ int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1);
+
+ long bitSize = bits.bitSize();
+ boolean bitsChanged = false;
+ for (int i = 1; i <= numHashFunctions; i++) {
+ int combinedHash = h1 + (i * h2);
+ // Flip all the bits if it's negative (guaranteed positive number)
+ if (combinedHash < 0) {
+ combinedHash = ~combinedHash;
+ }
+ bitsChanged |= bits.set(combinedHash % bitSize);
+ }
+ return bitsChanged;
+ }
+
+ @Override
+ public boolean mightContainString(String item) {
+ return mightContainBinary(Utils.getBytesFromUTF8String(item));
+ }
+
+ @Override
+ public boolean mightContainBinary(byte[] item) {
+ int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, 0);
+ int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1);
+
+ long bitSize = bits.bitSize();
+ for (int i = 1; i <= numHashFunctions; i++) {
+ int combinedHash = h1 + (i * h2);
+ // Flip all the bits if it's negative (guaranteed positive number)
+ if (combinedHash < 0) {
+ combinedHash = ~combinedHash;
+ }
+ if (!bits.get(combinedHash % bitSize)) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ @Override
+ public boolean putLong(long item) {
+ // Here we first hash the input long element into 2 int hash values, h1 and h2, then produce n
+ // hash values by `h1 + i * h2` with 1 <= i <= numHashFunctions.
+ // Note that `CountMinSketch` use a different strategy, it hash the input long element with
+ // every i to produce n hash values.
+ // TODO: the strategy of `CountMinSketch` looks more advanced, should we follow it here?
+ int h1 = Murmur3_x86_32.hashLong(item, 0);
+ int h2 = Murmur3_x86_32.hashLong(item, h1);
+
+ long bitSize = bits.bitSize();
+ boolean bitsChanged = false;
+ for (int i = 1; i <= numHashFunctions; i++) {
+ int combinedHash = h1 + (i * h2);
+ // Flip all the bits if it's negative (guaranteed positive number)
+ if (combinedHash < 0) {
+ combinedHash = ~combinedHash;
+ }
+ bitsChanged |= bits.set(combinedHash % bitSize);
+ }
+ return bitsChanged;
+ }
+
+ @Override
+ public boolean mightContainLong(long item) {
+ int h1 = Murmur3_x86_32.hashLong(item, 0);
+ int h2 = Murmur3_x86_32.hashLong(item, h1);
+
+ long bitSize = bits.bitSize();
+ for (int i = 1; i <= numHashFunctions; i++) {
+ int combinedHash = h1 + (i * h2);
+ // Flip all the bits if it's negative (guaranteed positive number)
+ if (combinedHash < 0) {
+ combinedHash = ~combinedHash;
+ }
+ if (!bits.get(combinedHash % bitSize)) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ @Override
+ public boolean mightContain(Object item) {
+ if (item instanceof String) {
+ return mightContainString((String) item);
+ } else if (item instanceof byte[]) {
+ return mightContainBinary((byte[]) item);
+ } else {
+ return mightContainLong(Utils.integralToLong(item));
+ }
+ }
+
+ @Override
+ public boolean isCompatible(BloomFilter other) {
+ if (other == null) {
+ return false;
+ }
+
+ if (!(other instanceof BloomFilterImpl)) {
+ return false;
+ }
+
+ BloomFilterImpl that = (BloomFilterImpl) other;
+ return this.bitSize() == that.bitSize() && this.numHashFunctions == that.numHashFunctions;
+ }
+
+ @Override
+ public BloomFilter mergeInPlace(BloomFilter other) throws IncompatibleMergeException {
+ BloomFilterImpl otherImplInstance = checkCompatibilityForMerge(other);
+
+ this.bits.putAll(otherImplInstance.bits);
+ return this;
+ }
+
+ @Override
+ public BloomFilter intersectInPlace(BloomFilter other) throws IncompatibleMergeException {
+ BloomFilterImpl otherImplInstance = checkCompatibilityForMerge(other);
+
+ this.bits.and(otherImplInstance.bits);
+ return this;
+ }
+
+ @Override
+ public long cardinality() {
+ return this.bits.cardinality();
+ }
+
+ private BloomFilterImpl checkCompatibilityForMerge(BloomFilter other)
+ throws IncompatibleMergeException {
+ // Duplicates the logic of `isCompatible` here to provide better error message.
+ if (other == null) {
+ throw new IncompatibleMergeException("Cannot merge null bloom filter");
+ }
+
+ if (!(other instanceof BloomFilterImpl)) {
+ throw new IncompatibleMergeException(
+ "Cannot merge bloom filter of class " + other.getClass().getName()
+ );
+ }
+
+ BloomFilterImpl that = (BloomFilterImpl) other;
+
+ if (this.bitSize() != that.bitSize()) {
+ throw new IncompatibleMergeException("Cannot merge bloom filters with different bit size");
+ }
+
+ if (this.numHashFunctions != that.numHashFunctions) {
+ throw new IncompatibleMergeException(
+ "Cannot merge bloom filters with different number of hash functions"
+ );
+ }
+ return that;
+ }
+
+ @Override
+ public void writeTo(OutputStream out) throws IOException {
+ DataOutputStream dos = new DataOutputStream(out);
+
+ dos.writeInt(Version.V1.getVersionNumber());
+ dos.writeInt(numHashFunctions);
+ bits.writeTo(dos);
+ }
+
+ private void readFrom0(InputStream in) throws IOException {
+ DataInputStream dis = new DataInputStream(in);
+
+ int version = dis.readInt();
+ if (version != Version.V1.getVersionNumber()) {
+ throw new IOException("Unexpected Bloom filter version number (" + version + ")");
+ }
+
+ this.numHashFunctions = dis.readInt();
+ this.bits = BitArray.readFrom(dis);
+ }
+
+ public static BloomFilterImpl readFrom(InputStream in) throws IOException {
+ BloomFilterImpl filter = new BloomFilterImpl();
+ filter.readFrom0(in);
+ return filter;
+ }
+
+ private void writeObject(ObjectOutputStream out) throws IOException {
+ writeTo(out);
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException {
+ readFrom0(in);
+ }
+}
diff --git a/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala b/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala
new file mode 100644
index 000000000..9a1cf637e
--- /dev/null
+++ b/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/expressions/BloomFilterMightContain.scala
@@ -0,0 +1,100 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.io.ByteArrayInputStream
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
+import org.apache.spark.sql.catalyst.trees.TreePattern.OUTER_REFERENCE
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal scalar function that returns the membership check result (either true or false)
+ * for values of `valueExpression` in the Bloom filter represented by `bloomFilterExpression`.
+ * Not that since the function is "might contain", always returning true regardless is not
+ * wrong.
+ * Note that this expression requires that `bloomFilterExpression` is either a constant value or
+ * an uncorrelated scalar subquery. This is sufficient for the Bloom filter join rewrite.
+ *
+ * @param bloomFilterExpression the Binary data of Bloom filter.
+ * @param valueExpression the Long value to be tested for the membership of `bloomFilterExpression`.
+ */
+case class BloomFilterMightContain(
+ bloomFilterExpression: Expression,
+ valueExpression: Expression) extends BinaryExpression {
+
+ override def nullable: Boolean = true
+ override def left: Expression = bloomFilterExpression
+ override def right: Expression = valueExpression
+ override def prettyName: String = "might_contain"
+ override def dataType: DataType = BooleanType
+
+ override def checkInputDataTypes(): TypeCheckResult = {
+ val typeCheckResult = (left.dataType, right.dataType) match {
+ case (BinaryType, NullType) | (NullType, LongType) | (NullType, NullType) |
+ (BinaryType, LongType) => TypeCheckResult.TypeCheckSuccess
+ case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+ s"been ${BinaryType.simpleString} followed by a value with ${LongType.simpleString}, " +
+ s"but it's [${left.dataType.catalogString}, ${right.dataType.catalogString}].")
+ }
+ if (typeCheckResult.isFailure) {
+ return typeCheckResult
+ }
+ bloomFilterExpression match {
+ case e : Expression if e.foldable => TypeCheckResult.TypeCheckSuccess
+ case subquery : PlanExpression[_] if !subquery.containsPattern(OUTER_REFERENCE) =>
+ TypeCheckResult.TypeCheckSuccess
+ case _ =>
+ TypeCheckResult.TypeCheckFailure(s"The Bloom filter binary input to $prettyName " +
+ s"should be either a constant value or a scalar subquery expression")
+ }
+ }
+
+ override protected def withNewChildrenInternal(
+ newBloomFilterExpression: Expression,
+ newValueExpression: Expression): BloomFilterMightContain =
+ copy(bloomFilterExpression = newBloomFilterExpression,
+ valueExpression = newValueExpression)
+
+ // The bloom filter created from `bloomFilterExpression`.
+ @transient private var bloomFilter: BloomFilter = _
+
+ override def nullSafeEval(bloomFilterBytes: Any, value: Any): Any = {
+ if (bloomFilter == null) {
+ bloomFilter = deserialize(bloomFilterBytes.asInstanceOf[Array[Byte]])
+ }
+ bloomFilter.mightContainLong(value.asInstanceOf[Long])
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val thisObj = ctx.addReferenceObj("thisObj", this)
+ nullSafeCodeGen(ctx, ev, (bloomFilterBytes, value) => {
+ s"\n${ev.value} = (Boolean) $thisObj.nullSafeEval($bloomFilterBytes, $value);\n"
+ })
+ }
+
+ final def deserialize(bytes: Array[Byte]): BloomFilter = {
+ val in = new ByteArrayInputStream(bytes)
+ val bloomFilter = BloomFilter.readFrom(in)
+ in.close()
+ bloomFilter
+ }
+
+}
diff --git a/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala b/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
new file mode 100644
index 000000000..86d3d62e1
--- /dev/null
+++ b/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/BloomFilterAggregate.scala
@@ -0,0 +1,196 @@
+/*
+ * 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.spark.sql.catalyst.expressions.aggregate
+
+import java.io.ByteArrayInputStream
+import java.io.ByteArrayOutputStream
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.types._
+import org.apache.spark.util.sketch.BloomFilter
+
+/**
+ * An internal aggregate function that creates a Bloom filter from input values.
+ *
+ * @param child Child expression of Long values for creating a Bloom filter.
+ * @param estimatedNumItemsExpression The number of estimated distinct items (optional).
+ * @param numBitsExpression The number of bits to use (optional).
+ */
+case class BloomFilterAggregate(
+ child: Expression,
+ estimatedNumItemsExpression: Expression,
+ numBitsExpression: Expression,
+ override val mutableAggBufferOffset: Int,
+ override val inputAggBufferOffset: Int)
+ extends TypedImperativeAggregate[BloomFilter] with TernaryLike[Expression] {
+
+ def this(child: Expression, estimatedNumItemsExpression: Expression,
+ numBitsExpression: Expression) = {
+ this(child, estimatedNumItemsExpression, numBitsExpression, 0, 0)
+ }
+
+ def this(child: Expression, estimatedNumItemsExpression: Expression) = {
+ this(child, estimatedNumItemsExpression,
+ // 1 byte per item.
+ Multiply(estimatedNumItemsExpression, Literal(8L)))
+ }
+
+ def this(child: Expression) = {
+ this(child, Literal(BloomFilterAggregate.DEFAULT_EXPECTED_NUM_ITEMS),
+ Literal(BloomFilterAggregate.DEFAULT_NUM_BITS))
+ }
+
+ override def checkInputDataTypes(): TypeCheckResult = {
+ val typeCheckResult = (first.dataType, second.dataType, third.dataType) match {
+ case (_, NullType, _) | (_, _, NullType) =>
+ TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as size arguments")
+ case (LongType, LongType, LongType) => TypeCheckResult.TypeCheckSuccess
+ case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " +
+ s"been a ${LongType.simpleString} value followed with two ${LongType.simpleString} size " +
+ s"arguments, but it's [${first.dataType.catalogString}, " +
+ s"${second.dataType.catalogString}, ${third.dataType.catalogString}]")
+ }
+ if (typeCheckResult.isFailure) {
+ return typeCheckResult
+ }
+ if (!estimatedNumItemsExpression.foldable) {
+ TypeCheckFailure("The estimated number of items provided must be a constant literal")
+ } else if (estimatedNumItems <= 0L) {
+ TypeCheckFailure("The estimated number of items must be a positive value " +
+ s" (current value = $estimatedNumItems)")
+ } else if (!numBitsExpression.foldable) {
+ TypeCheckFailure("The number of bits provided must be a constant literal")
+ } else if (numBits <= 0L) {
+ TypeCheckFailure("The number of bits must be a positive value " +
+ s" (current value = $numBits)")
+ } else {
+ require(estimatedNumItems <= BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)
+ require(numBits <= BloomFilterAggregate.MAX_NUM_BITS)
+ TypeCheckSuccess
+ }
+ }
+ override def nullable: Boolean = true
+
+ override def dataType: DataType = BinaryType
+
+ override def prettyName: String = "bloom_filter_agg"
+
+ // Mark as lazy so that `estimatedNumItems` is not evaluated during tree transformation.
+ private lazy val estimatedNumItems: Long =
+ Math.min(estimatedNumItemsExpression.eval().asInstanceOf[Number].longValue,
+ BloomFilterAggregate.MAX_ALLOWED_NUM_ITEMS)
+
+ // Mark as lazy so that `numBits` is not evaluated during tree transformation.
+ private lazy val numBits: Long =
+ Math.min(numBitsExpression.eval().asInstanceOf[Number].longValue,
+ BloomFilterAggregate.MAX_NUM_BITS)
+
+ override def first: Expression = child
+
+ override def second: Expression = estimatedNumItemsExpression
+
+ override def third: Expression = numBitsExpression
+
+ override protected def withNewChildrenInternal(newChild: Expression,
+ newEstimatedNumItemsExpression: Expression, newNumBitsExpression: Expression)
+ : BloomFilterAggregate = {
+ copy(child = newChild, estimatedNumItemsExpression = newEstimatedNumItemsExpression,
+ numBitsExpression = newNumBitsExpression)
+ }
+
+ override def createAggregationBuffer(): BloomFilter = {
+ BloomFilter.create(estimatedNumItems, numBits)
+ }
+
+ override def update(buffer: BloomFilter, inputRow: InternalRow): BloomFilter = {
+ val value = child.eval(inputRow)
+ // Ignore null values.
+ if (value == null) {
+ return buffer
+ }
+ buffer.putLong(value.asInstanceOf[Long])
+ buffer
+ }
+
+ override def merge(buffer: BloomFilter, other: BloomFilter): BloomFilter = {
+ buffer.mergeInPlace(other)
+ }
+
+ override def eval(buffer: BloomFilter): Any = {
+ if (buffer.cardinality() == 0) {
+ // There's no set bit in the Bloom filter and hence no not-null value is processed.
+ return null
+ }
+ serialize(buffer)
+ }
+
+ override def withNewMutableAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+ copy(mutableAggBufferOffset = newOffset)
+
+ override def withNewInputAggBufferOffset(newOffset: Int): BloomFilterAggregate =
+ copy(inputAggBufferOffset = newOffset)
+
+ override def serialize(obj: BloomFilter): Array[Byte] = {
+ BloomFilterAggregate.serde.serialize(obj)
+ }
+
+ override def deserialize(bytes: Array[Byte]): BloomFilter = {
+ BloomFilterAggregate.serde.deserialize(bytes)
+ }
+}
+
+object BloomFilterAggregate {
+
+ val DEFAULT_EXPECTED_NUM_ITEMS: Long = 1000000L // Default 1M distinct items
+
+ val MAX_ALLOWED_NUM_ITEMS: Long = 4000000L // At most 4M distinct items
+
+ val DEFAULT_NUM_BITS: Long = 8388608 // Default 1MB
+
+ val MAX_NUM_BITS: Long = 67108864 // At most 8MB
+
+ /**
+ * Serializer/Deserializer for class [[BloomFilter]]
+ *
+ * This class is thread safe.
+ */
+ class BloomFilterSerDe {
+
+ final def serialize(obj: BloomFilter): Array[Byte] = {
+ val size = obj.bitSize()/8
+ require(size <= Integer.MAX_VALUE, s"actual number of bits is too large $size")
+ val out = new ByteArrayOutputStream(size.intValue())
+ obj.writeTo(out)
+ out.close()
+ out.toByteArray
+ }
+
+ final def deserialize(bytes: Array[Byte]): BloomFilter = {
+ val in = new ByteArrayInputStream(bytes)
+ val bloomFilter = BloomFilter.readFrom(in)
+ in.close()
+ bloomFilter
+ }
+ }
+
+ val serde: BloomFilterSerDe = new BloomFilterSerDe
+}
diff --git a/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
new file mode 100644
index 000000000..c42843764
--- /dev/null
+++ b/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
@@ -0,0 +1,1950 @@
+/*
+ * 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.spark.sql.catalyst.expressions.objects
+
+import java.lang.reflect.{Method, Modifier}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{Builder, WrappedArray}
+import scala.reflect.ClassTag
+import scala.util.{Properties, Try}
+
+import org.apache.commons.lang3.reflect.MethodUtils
+
+import org.apache.spark.{SparkConf, SparkEnv}
+import org.apache.spark.serializer._
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.{InternalRow, ScalaReflection}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.trees.TernaryLike
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, _}
+import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+/**
+ * Common base class for [[StaticInvoke]], [[Invoke]], and [[NewInstance]].
+ */
+trait InvokeLike extends Expression with NonSQLExpression with ImplicitCastInputTypes {
+
+ def arguments: Seq[Expression]
+
+ def propagateNull: Boolean
+
+ protected lazy val needNullCheck: Boolean = needNullCheckForIndex.contains(true)
+ protected lazy val needNullCheckForIndex: Array[Boolean] =
+ arguments.map(a => a.nullable && (propagateNull ||
+ ScalaReflection.dataTypeJavaClass(a.dataType).isPrimitive)).toArray
+ protected lazy val evaluatedArgs: Array[Object] = new Array[Object](arguments.length)
+ private lazy val boxingFn: Any => Any =
+ ScalaReflection.typeBoxedJavaMapping
+ .get(dataType)
+ .map(cls => v => cls.cast(v))
+ .getOrElse(identity)
+
+
+ /**
+ * Prepares codes for arguments.
+ *
+ * - generate codes for argument.
+ * - use ctx.splitExpressions() to not exceed 64kb JVM limit while preparing arguments.
+ * - avoid some of nullability checking which are not needed because the expression is not
+ * nullable.
+ * - when needNullCheck == true, short circuit if we found one of arguments is null because
+ * preparing rest of arguments can be skipped in the case.
+ *
+ * @param ctx a [[CodegenContext]]
+ * @return (code to prepare arguments, argument string, result of argument null check)
+ */
+ def prepareArguments(ctx: CodegenContext): (String, String, ExprValue) = {
+
+ val resultIsNull = if (needNullCheck) {
+ val resultIsNull = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "resultIsNull")
+ JavaCode.isNullGlobal(resultIsNull)
+ } else {
+ FalseLiteral
+ }
+ val argValues = arguments.map { e =>
+ val argValue = ctx.addMutableState(CodeGenerator.javaType(e.dataType), "argValue")
+ argValue
+ }
+
+ val argCodes = if (needNullCheck) {
+ val reset = s"$resultIsNull = false;"
+ val argCodes = arguments.zipWithIndex.map { case (e, i) =>
+ val expr = e.genCode(ctx)
+ val updateResultIsNull = if (needNullCheckForIndex(i)) {
+ s"$resultIsNull = ${expr.isNull};"
+ } else {
+ ""
+ }
+ s"""
+ if (!$resultIsNull) {
+ ${expr.code}
+ $updateResultIsNull
+ ${argValues(i)} = ${expr.value};
+ }
+ """
+ }
+ reset +: argCodes
+ } else {
+ arguments.zipWithIndex.map { case (e, i) =>
+ val expr = e.genCode(ctx)
+ s"""
+ ${expr.code}
+ ${argValues(i)} = ${expr.value};
+ """
+ }
+ }
+ val argCode = ctx.splitExpressionsWithCurrentInputs(argCodes)
+
+ (argCode, argValues.mkString(", "), resultIsNull)
+ }
+
+ /**
+ * Evaluate each argument with a given row, invoke a method with a given object and arguments,
+ * and cast a return value if the return type can be mapped to a Java Boxed type
+ *
+ * @param obj the object for the method to be called. If null, perform s static method call
+ * @param method the method object to be called
+ * @param input the row used for evaluating arguments
+ * @return the return object of a method call
+ */
+ def invoke(obj: Any, method: Method, input: InternalRow): Any = {
+ var i = 0
+ val len = arguments.length
+ var resultNull = false
+ while (i < len) {
+ val result = arguments(i).eval(input).asInstanceOf[Object]
+ evaluatedArgs(i) = result
+ resultNull = resultNull || (result == null && needNullCheckForIndex(i))
+ i += 1
+ }
+ if (needNullCheck && resultNull) {
+ // return null if one of arguments is null
+ null
+ } else {
+ val ret = try {
+ method.invoke(obj, evaluatedArgs: _*)
+ } catch {
+ // Re-throw the original exception.
+ case e: java.lang.reflect.InvocationTargetException if e.getCause != null =>
+ throw e.getCause
+ }
+ boxingFn(ret)
+ }
+ }
+
+ final def findMethod(cls: Class[_], functionName: String, argClasses: Seq[Class[_]]): Method = {
+ val method = MethodUtils.getMatchingAccessibleMethod(cls, functionName, argClasses: _*)
+ if (method == null) {
+ throw QueryExecutionErrors.methodNotDeclaredError(functionName)
+ } else {
+ method
+ }
+ }
+}
+
+/**
+ * Common trait for [[DecodeUsingSerializer]] and [[EncodeUsingSerializer]]
+ */
+trait SerializerSupport {
+ /**
+ * If true, Kryo serialization is used, otherwise the Java one is used
+ */
+ val kryo: Boolean
+
+ /**
+ * The serializer instance to be used for serialization/deserialization in interpreted execution
+ */
+ lazy val serializerInstance: SerializerInstance = SerializerSupport.newSerializer(kryo)
+
+ /**
+ * Adds a immutable state to the generated class containing a reference to the serializer.
+ * @return a string containing the name of the variable referencing the serializer
+ */
+ def addImmutableSerializerIfNeeded(ctx: CodegenContext): String = {
+ val (serializerInstance, serializerInstanceClass) = {
+ if (kryo) {
+ ("kryoSerializer",
+ classOf[KryoSerializerInstance].getName)
+ } else {
+ ("javaSerializer",
+ classOf[JavaSerializerInstance].getName)
+ }
+ }
+ val newSerializerMethod = s"${classOf[SerializerSupport].getName}$$.MODULE$$.newSerializer"
+ // Code to initialize the serializer
+ ctx.addImmutableStateIfNotExists(serializerInstanceClass, serializerInstance, v =>
+ s"""
+ |$v = ($serializerInstanceClass) $newSerializerMethod($kryo);
+ """.stripMargin)
+ serializerInstance
+ }
+}
+
+object SerializerSupport {
+ /**
+ * It creates a new `SerializerInstance` which is either a `KryoSerializerInstance` (is
+ * `useKryo` is set to `true`) or a `JavaSerializerInstance`.
+ */
+ def newSerializer(useKryo: Boolean): SerializerInstance = {
+ // try conf from env, otherwise create a new one
+ val conf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf)
+ val s = if (useKryo) {
+ new KryoSerializer(conf)
+ } else {
+ new JavaSerializer(conf)
+ }
+ s.newInstance()
+ }
+}
+
+/**
+ * Invokes a static function, returning the result. By default, any of the arguments being null
+ * will result in returning null instead of calling the function.
+ *
+ * @param staticObject The target of the static call. This can either be the object itself
+ * (methods defined on scala objects), or the class object
+ * (static methods defined in java).
+ * @param dataType The expected return type of the function call
+ * @param functionName The name of the method to call.
+ * @param arguments An optional list of expressions to pass as arguments to the function.
+ * @param inputTypes A list of data types specifying the input types for the method to be invoked.
+ * If enabled, it must have the same length as [[arguments]]. In case an input
+ * type differs from the actual argument type, Spark will try to perform
+ * type coercion and insert cast whenever necessary before invoking the method.
+ * The above is disabled if this is empty.
+ * @param propagateNull When true, and any of the arguments is null, null will be returned instead
+ * of calling the function. Also note: when this is false but any of the
+ * arguments is of primitive type and is null, null also will be returned
+ * without invoking the function.
+ * @param returnNullable When false, indicating the invoked method will always return
+ * non-null value.
+ * @param isDeterministic Whether the method invocation is deterministic or not. If false, Spark
+ * will not apply certain optimizations such as constant folding.
+ */
+case class StaticInvoke(
+ staticObject: Class[_],
+ dataType: DataType,
+ functionName: String,
+ arguments: Seq[Expression] = Nil,
+ inputTypes: Seq[AbstractDataType] = Nil,
+ propagateNull: Boolean = true,
+ returnNullable: Boolean = true,
+ isDeterministic: Boolean = true) extends InvokeLike {
+
+ val objectName = staticObject.getName.stripSuffix("$")
+ val cls = if (staticObject.getName == objectName) {
+ staticObject
+ } else {
+ Utils.classForName(objectName)
+ }
+
+ override def nullable: Boolean = needNullCheck || returnNullable
+ override def children: Seq[Expression] = arguments
+ override lazy val deterministic: Boolean = isDeterministic && arguments.forall(_.deterministic)
+
+ lazy val argClasses = ScalaReflection.expressionJavaClasses(arguments)
+ @transient lazy val method = findMethod(cls, functionName, argClasses)
+
+ override def eval(input: InternalRow): Any = {
+ invoke(null, method, input)
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val javaType = CodeGenerator.javaType(dataType)
+
+ val (argCode, argString, resultIsNull) = prepareArguments(ctx)
+
+ val callFunc = s"$objectName.$functionName($argString)"
+
+ val prepareIsNull = if (nullable) {
+ s"boolean ${ev.isNull} = $resultIsNull;"
+ } else {
+ ev.isNull = FalseLiteral
+ ""
+ }
+
+ val evaluate = if (returnNullable && !method.getReturnType.isPrimitive) {
+ if (CodeGenerator.defaultValue(dataType) == "null") {
+ s"""
+ ${ev.value} = $callFunc;
+ ${ev.isNull} = ${ev.value} == null;
+ """
+ } else {
+ val boxedResult = ctx.freshName("boxedResult")
+ s"""
+ ${CodeGenerator.boxedType(dataType)} $boxedResult = $callFunc;
+ ${ev.isNull} = $boxedResult == null;
+ if (!${ev.isNull}) {
+ ${ev.value} = $boxedResult;
+ }
+ """
+ }
+ } else {
+ s"${ev.value} = $callFunc;"
+ }
+
+ val code = code"""
+ $argCode
+ $prepareIsNull
+ $javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
+ if (!$resultIsNull) {
+ $evaluate
+ }
+ """
+ ev.copy(code = code)
+ }
+
+ override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Expression =
+ copy(arguments = newChildren)
+}
+
+/**
+ * Calls the specified function on an object, optionally passing arguments. If the `targetObject`
+ * expression evaluates to null then null will be returned.
+ *
+ * In some cases, due to erasure, the schema may expect a primitive type when in fact the method
+ * is returning java.lang.Object. In this case, we will generate code that attempts to unbox the
+ * value automatically.
+ *
+ * @param targetObject An expression that will return the object to call the method on.
+ * @param functionName The name of the method to call.
+ * @param dataType The expected return type of the function.
+ * @param arguments An optional list of expressions, whose evaluation will be passed to the
+ * function.
+ * @param methodInputTypes A list of data types specifying the input types for the method to be
+ * invoked. If enabled, it must have the same length as [[arguments]]. In
+ * case an input type differs from the actual argument type, Spark will
+ * try to perform type coercion and insert cast whenever necessary before
+ * invoking the method. The type coercion is disabled if this is empty.
+ * @param propagateNull When true, and any of the arguments is null, null will be returned instead
+ * of calling the function. Also note: when this is false but any of the
+ * arguments is of primitive type and is null, null also will be returned
+ * without invoking the function.
+ * @param returnNullable When false, indicating the invoked method will always return
+ * non-null value.
+ * @param isDeterministic Whether the method invocation is deterministic or not. If false, Spark
+ * will not apply certain optimizations such as constant folding.
+ */
+case class Invoke(
+ targetObject: Expression,
+ functionName: String,
+ dataType: DataType,
+ arguments: Seq[Expression] = Nil,
+ methodInputTypes: Seq[AbstractDataType] = Nil,
+ propagateNull: Boolean = true,
+ returnNullable : Boolean = true,
+ isDeterministic: Boolean = true) extends InvokeLike {
+
+ lazy val argClasses = ScalaReflection.expressionJavaClasses(arguments)
+
+ final override val nodePatterns: Seq[TreePattern] = Seq(INVOKE)
+
+ override def nullable: Boolean = targetObject.nullable || needNullCheck || returnNullable
+ override def children: Seq[Expression] = targetObject +: arguments
+ override lazy val deterministic: Boolean = isDeterministic && arguments.forall(_.deterministic)
+ override def inputTypes: Seq[AbstractDataType] =
+ if (methodInputTypes.nonEmpty) {
+ Seq(targetObject.dataType) ++ methodInputTypes
+ } else {
+ Nil
+ }
+
+ private lazy val encodedFunctionName = ScalaReflection.encodeFieldNameToIdentifier(functionName)
+
+ @transient lazy val method = targetObject.dataType match {
+ case ObjectType(cls) =>
+ Some(findMethod(cls, encodedFunctionName, argClasses))
+ case _ => None
+ }
+
+ override def eval(input: InternalRow): Any = {
+ val obj = targetObject.eval(input)
+ if (obj == null) {
+ // return null if obj is null
+ null
+ } else {
+ val invokeMethod = if (method.isDefined) {
+ method.get
+ } else {
+ obj.getClass.getMethod(functionName, argClasses: _*)
+ }
+ invoke(obj, invokeMethod, input)
+ }
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val javaType = CodeGenerator.javaType(dataType)
+ val obj = targetObject.genCode(ctx)
+
+ val (argCode, argString, resultIsNull) = prepareArguments(ctx)
+
+ val returnPrimitive = method.isDefined && method.get.getReturnType.isPrimitive
+ val needTryCatch = method.isDefined && method.get.getExceptionTypes.nonEmpty
+
+ def getFuncResult(resultVal: String, funcCall: String): String = if (needTryCatch) {
+ s"""
+ try {
+ $resultVal = $funcCall;
+ } catch (Exception e) {
+ org.apache.spark.unsafe.Platform.throwException(e);
+ }
+ """
+ } else {
+ s"$resultVal = $funcCall;"
+ }
+
+ val evaluate = if (returnPrimitive) {
+ getFuncResult(ev.value, s"${obj.value}.$encodedFunctionName($argString)")
+ } else {
+ val funcResult = ctx.freshName("funcResult")
+ // If the function can return null, we do an extra check to make sure our null bit is still
+ // set correctly.
+ val assignResult = if (!returnNullable) {
+ s"${ev.value} = (${CodeGenerator.boxedType(javaType)}) $funcResult;"
+ } else {
+ s"""
+ if ($funcResult != null) {
+ ${ev.value} = (${CodeGenerator.boxedType(javaType)}) $funcResult;
+ } else {
+ ${ev.isNull} = true;
+ }
+ """
+ }
+ s"""
+ Object $funcResult = null;
+ ${getFuncResult(funcResult, s"${obj.value}.$encodedFunctionName($argString)")}
+ $assignResult
+ """
+ }
+
+ val mainEvalCode =
+ code"""
+ |$argCode
+ |${ev.isNull} = $resultIsNull;
+ |if (!${ev.isNull}) {
+ | $evaluate
+ |}
+ |""".stripMargin
+
+ val evalWithNullCheck = if (targetObject.nullable) {
+ code"""
+ |if (!${obj.isNull}) {
+ | $mainEvalCode
+ |}
+ |""".stripMargin
+ } else {
+ mainEvalCode
+ }
+
+ val code = obj.code + code"""
+ boolean ${ev.isNull} = true;
+ $javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
+ $evalWithNullCheck
+ """
+ ev.copy(code = code)
+ }
+
+ override def toString: String = s"$targetObject.$functionName"
+
+ override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): Invoke =
+ copy(targetObject = newChildren.head, arguments = newChildren.tail)
+}
+
+object NewInstance {
+ def apply(
+ cls: Class[_],
+ arguments: Seq[Expression],
+ dataType: DataType,
+ propagateNull: Boolean = true): NewInstance =
+ new NewInstance(cls, arguments, inputTypes = Nil, propagateNull, dataType, None)
+}
+
+/**
+ * Constructs a new instance of the given class, using the result of evaluating the specified
+ * expressions as arguments.
+ *
+ * @param cls The class to construct.
+ * @param arguments A list of expression to use as arguments to the constructor.
+ * @param inputTypes A list of data types specifying the input types for the method to be invoked.
+ * If enabled, it must have the same length as [[arguments]]. In case an input
+ * type differs from the actual argument type, Spark will try to perform
+ * type coercion and insert cast whenever necessary before invoking the method.
+ * The above is disabled if this is empty.
+ * @param propagateNull When true, if any of the arguments is null, then null will be returned
+ * instead of trying to construct the object. Also note: when this is false
+ * but any of the arguments is of primitive type and is null, null also will
+ * be returned without constructing the object.
+ * @param dataType The type of object being constructed, as a Spark SQL datatype. This allows you
+ * to manually specify the type when the object in question is a valid internal
+ * representation (i.e. ArrayData) instead of an object.
+ * @param outerPointer If the object being constructed is an inner class, the outerPointer for the
+ * containing class must be specified. This parameter is defined as an optional
+ * function, which allows us to get the outer pointer lazily,and it's useful if
+ * the inner class is defined in REPL.
+ */
+case class NewInstance(
+ cls: Class[_],
+ arguments: Seq[Expression],
+ inputTypes: Seq[AbstractDataType],
+ propagateNull: Boolean,
+ dataType: DataType,
+ outerPointer: Option[() => AnyRef]) extends InvokeLike {
+ private val className = cls.getName
+
+ override def nullable: Boolean = needNullCheck
+
+ override def children: Seq[Expression] = arguments
+
+ final override val nodePatterns: Seq[TreePattern] = Seq(NEW_INSTANCE)
+
+ override lazy val resolved: Boolean = {
+ // If the class to construct is an inner class, we need to get its outer pointer, or this
+ // expression should be regarded as unresolved.
+ // Note that static inner classes (e.g., inner classes within Scala objects) don't need
+ // outer pointer registration.
+ val needOuterPointer =
+ outerPointer.isEmpty && Utils.isMemberClass(cls) && !Modifier.isStatic(cls.getModifiers)
+ childrenResolved && !needOuterPointer
+ }
+
+ @transient private lazy val constructor: (Seq[AnyRef]) => Any = {
+ val paramTypes = ScalaReflection.expressionJavaClasses(arguments)
+ val getConstructor = (paramClazz: Seq[Class[_]]) => {
+ ScalaReflection.findConstructor(cls, paramClazz).getOrElse {
+ throw QueryExecutionErrors.constructorNotFoundError(cls.toString)
+ }
+ }
+ outerPointer.map { p =>
+ val outerObj = p()
+ val c = getConstructor(outerObj.getClass +: paramTypes)
+ (args: Seq[AnyRef]) => {
+ c(outerObj +: args)
+ }
+ }.getOrElse {
+ val c = getConstructor(paramTypes)
+ (args: Seq[AnyRef]) => {
+ c(args)
+ }
+ }
+ }
+
+ override def eval(input: InternalRow): Any = {
+ val argValues = arguments.map(_.eval(input))
+ constructor(argValues.map(_.asInstanceOf[AnyRef]))
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val javaType = CodeGenerator.javaType(dataType)
+
+ val (argCode, argString, resultIsNull) = prepareArguments(ctx)
+
+ val outer = outerPointer.map(func => Literal.fromObject(func()).genCode(ctx))
+
+ ev.isNull = resultIsNull
+
+ val constructorCall = cls.getConstructors.size match {
+ // If there are no constructors, the `new` method will fail. In
+ // this case we can try to call the apply method constructor
+ // that might be defined on the companion object.
+ case 0 => s"$className$$.MODULE$$.apply($argString)"
+ case _ => outer.map { gen =>
+ s"${gen.value}.new ${Utils.getSimpleName(cls)}($argString)"
+ }.getOrElse {
+ s"new $className($argString)"
+ }
+ }
+
+ val code = code"""
+ $argCode
+ ${outer.map(_.code).getOrElse("")}
+ final $javaType ${ev.value} = ${ev.isNull} ?
+ ${CodeGenerator.defaultValue(dataType)} : $constructorCall;
+ """
+ ev.copy(code = code)
+ }
+
+ override def toString: String = s"newInstance($cls)"
+
+ override protected def withNewChildrenInternal(newChildren: IndexedSeq[Expression]): NewInstance =
+ copy(arguments = newChildren)
+}
+
+/**
+ * Given an expression that returns on object of type `Option[_]`, this expression unwraps the
+ * option into the specified Spark SQL datatype. In the case of `None`, the nullbit is set instead.
+ *
+ * @param dataType The expected unwrapped option type.
+ * @param child An expression that returns an `Option`
+ */
+case class UnwrapOption(
+ dataType: DataType,
+ child: Expression) extends UnaryExpression with NonSQLExpression with ExpectsInputTypes {
+
+ override def nullable: Boolean = true
+
+ override def inputTypes: Seq[AbstractDataType] = ObjectType :: Nil
+
+ override def eval(input: InternalRow): Any = {
+ val inputObject = child.eval(input)
+ if (inputObject == null) {
+ null
+ } else {
+ inputObject.asInstanceOf[Option[_]].orNull
+ }
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val javaType = CodeGenerator.javaType(dataType)
+ val inputObject = child.genCode(ctx)
+
+ val code = inputObject.code + code"""
+ final boolean ${ev.isNull} = ${inputObject.isNull} || ${inputObject.value}.isEmpty();
+ $javaType ${ev.value} = ${ev.isNull} ? ${CodeGenerator.defaultValue(dataType)} :
+ (${CodeGenerator.boxedType(javaType)}) ${inputObject.value}.get();
+ """
+ ev.copy(code = code)
+ }
+
+ override protected def withNewChildInternal(newChild: Expression): UnwrapOption =
+ copy(child = newChild)
+}
+
+/**
+ * Converts the result of evaluating `child` into an option, checking both the isNull bit and
+ * (in the case of reference types) equality with null.
+ *
+ * @param child The expression to evaluate and wrap.
+ * @param optType The type of this option.
+ */
+case class WrapOption(child: Expression, optType: DataType)
+ extends UnaryExpression with NonSQLExpression with ExpectsInputTypes {
+
+ override def dataType: DataType = ObjectType(classOf[Option[_]])
+
+ override def nullable: Boolean = false
+
+ override def inputTypes: Seq[AbstractDataType] = optType :: Nil
+
+ override def eval(input: InternalRow): Any = Option(child.eval(input))
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val inputObject = child.genCode(ctx)
+
+ val code = inputObject.code + code"""
+ scala.Option ${ev.value} =
+ ${inputObject.isNull} ?
+ scala.Option$$.MODULE$$.apply(null) : new scala.Some(${inputObject.value});
+ """
+ ev.copy(code = code, isNull = FalseLiteral)
+ }
+
+ override protected def withNewChildInternal(newChild: Expression): WrapOption =
+ copy(child = newChild)
+}
+
+object LambdaVariable {
+ private val curId = new java.util.concurrent.atomic.AtomicLong()
+
+ // Returns the codegen-ed `LambdaVariable` and add it to mutable states, so that it can be
+ // accessed anywhere in the generated code.
+ def prepareLambdaVariable(ctx: CodegenContext, variable: LambdaVariable): ExprCode = {
+ val variableCode = variable.genCode(ctx)
+ assert(variableCode.code.isEmpty)
+
+ ctx.addMutableState(
+ CodeGenerator.javaType(variable.dataType),
+ variableCode.value,
+ forceInline = true,
+ useFreshName = false)
+
+ if (variable.nullable) {
+ ctx.addMutableState(
+ CodeGenerator.JAVA_BOOLEAN,
+ variableCode.isNull,
+ forceInline = true,
+ useFreshName = false)
+ }
+
+ variableCode
+ }
+}
+
+/**
+ * A placeholder for the loop variable used in [[MapObjects]]. This should never be constructed
+ * manually, but will instead be passed into the provided lambda function.
+ */
+// TODO: Merge this and `NamedLambdaVariable`.
+case class LambdaVariable(
+ name: String,
+ dataType: DataType,
+ nullable: Boolean,
+ id: Long = LambdaVariable.curId.incrementAndGet) extends LeafExpression with NonSQLExpression {
+
+ private val accessor: (InternalRow, Int) => Any = InternalRow.getAccessor(dataType, nullable)
+
+ final override val nodePatterns: Seq[TreePattern] = Seq(LAMBDA_VARIABLE)
+
+ // Interpreted execution of `LambdaVariable` always get the 0-index element from input row.
+ override def eval(input: InternalRow): Any = {
+ assert(input.numFields == 1,
+ "The input row of interpreted LambdaVariable should have only 1 field.")
+ accessor(input, 0)
+ }
+
+ override def genCode(ctx: CodegenContext): ExprCode = {
+ // If `LambdaVariable` IDs are reassigned by the `ReassignLambdaVariableID` rule, the IDs will
+ // all be negative.
+ val suffix = "lambda_variable_" + math.abs(id)
+ val isNull = if (nullable) {
+ JavaCode.isNullVariable(s"isNull_${name}_$suffix")
+ } else {
+ FalseLiteral
+ }
+ val value = JavaCode.variable(s"value_${name}_$suffix", dataType)
+ ExprCode(isNull, value)
+ }
+
+ // This won't be called as `genCode` is overrided, just overriding it to make
+ // `LambdaVariable` non-abstract.
+ override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = ev
+}
+
+/**
+ * When constructing [[MapObjects]], the element type must be given, which may not be available
+ * before analysis. This class acts like a placeholder for [[MapObjects]], and will be replaced by
+ * [[MapObjects]] during analysis after the input data is resolved.
+ * Note that, ideally we should not serialize and send unresolved expressions to executors, but
+ * users may accidentally do this(e.g. mistakenly reference an encoder instance when implementing
+ * Aggregator). Here we mark `function` as transient because it may reference scala Type, which is
+ * not serializable. Then even users mistakenly reference unresolved expression and serialize it,
+ * it's just a performance issue(more network traffic), and will not fail.
+ */
+case class UnresolvedMapObjects(
+ @transient function: Expression => Expression,
+ child: Expression,
+ customCollectionCls: Option[Class[_]] = None) extends UnaryExpression with Unevaluable {
+ override lazy val resolved = false
+
+ override def dataType: DataType = customCollectionCls.map(ObjectType.apply).getOrElse {
+ throw QueryExecutionErrors.customCollectionClsNotResolvedError
+ }
+
+ override protected def withNewChildInternal(newChild: Expression): UnresolvedMapObjects =
+ copy(child = newChild)
+}
+
+object MapObjects {
+ /**
+ * Construct an instance of MapObjects case class.
+ *
+ * @param function The function applied on the collection elements.
+ * @param inputData An expression that when evaluated returns a collection object.
+ * @param elementType The data type of elements in the collection.
+ * @param elementNullable When false, indicating elements in the collection are always
+ * non-null value.
+ * @param customCollectionCls Class of the resulting collection (returning ObjectType)
+ * or None (returning ArrayType)
+ */
+ def apply(
+ function: Expression => Expression,
+ inputData: Expression,
+ elementType: DataType,
+ elementNullable: Boolean = true,
+ customCollectionCls: Option[Class[_]] = None): MapObjects = {
+ // UnresolvedMapObjects does not serialize its 'function' field.
+ // If an array expression or array Encoder is not correctly resolved before
+ // serialization, this exception condition may occur.
+ require(function != null,
+ "MapObjects applied with a null function. " +
+ "Likely cause is failure to resolve an array expression or encoder. " +
+ "(See UnresolvedMapObjects)")
+ val loopVar = LambdaVariable("MapObject", elementType, elementNullable)
+ MapObjects(loopVar, function(loopVar), inputData, customCollectionCls)
+ }
+}
+
+/**
+ * Applies the given expression to every element of a collection of items, returning the result
+ * as an ArrayType or ObjectType. This is similar to a typical map operation, but where the lambda
+ * function is expressed using catalyst expressions.
+ *
+ * The type of the result is determined as follows:
+ * - ArrayType - when customCollectionCls is None
+ * - ObjectType(collection) - when customCollectionCls contains a collection class
+ *
+ * The following collection ObjectTypes are currently supported on input:
+ * Seq, Array, ArrayData, java.util.List
+ *
+ * @param loopVar the [[LambdaVariable]] expression representing the loop variable that used to
+ * iterate the collection, and used as input for the `lambdaFunction`.
+ * @param lambdaFunction A function that take the `loopVar` as input, and used as lambda function
+ * to handle collection elements.
+ * @param inputData An expression that when evaluated returns a collection object.
+ * @param customCollectionCls Class of the resulting collection (returning ObjectType)
+ * or None (returning ArrayType)
+ */
+case class MapObjects private(
+ loopVar: LambdaVariable,
+ lambdaFunction: Expression,
+ inputData: Expression,
+ customCollectionCls: Option[Class[_]]) extends Expression with NonSQLExpression
+ with TernaryLike[Expression] {
+
+ override def nullable: Boolean = inputData.nullable
+
+ override def first: Expression = loopVar
+ override def second: Expression = lambdaFunction
+ override def third: Expression = inputData
+
+ final override val nodePatterns: Seq[TreePattern] = Seq(MAP_OBJECTS)
+
+ // The data with UserDefinedType are actually stored with the data type of its sqlType.
+ // When we want to apply MapObjects on it, we have to use it.
+ lazy private val inputDataType = inputData.dataType match {
+ case u: UserDefinedType[_] => u.sqlType
+ case _ => inputData.dataType
+ }
+
+ private def executeFuncOnCollection(inputCollection: Seq[_]): Iterator[_] = {
+ val row = new GenericInternalRow(1)
+ inputCollection.toIterator.map { element =>
+ row.update(0, element)
+ lambdaFunction.eval(row)
+ }
+ }
+
+ private lazy val convertToSeq: Any => Seq[_] = inputDataType match {
+ case ObjectType(cls) if classOf[scala.collection.Seq[_]].isAssignableFrom(cls) =>
+ _.asInstanceOf[scala.collection.Seq[_]].toSeq
+ case ObjectType(cls) if cls.isArray =>
+ _.asInstanceOf[Array[_]].toSeq
+ case ObjectType(cls) if classOf[java.util.List[_]].isAssignableFrom(cls) =>
+ _.asInstanceOf[java.util.List[_]].asScala.toSeq
+ case ObjectType(cls) if cls == classOf[Object] =>
+ (inputCollection) => {
+ if (inputCollection.getClass.isArray) {
+ inputCollection.asInstanceOf[Array[_]].toSeq
+ } else {
+ inputCollection.asInstanceOf[Seq[_]]
+ }
+ }
+ case ArrayType(et, _) =>
+ _.asInstanceOf[ArrayData].toSeq[Any](et)
+ }
+
+ private lazy val mapElements: Seq[_] => Any = customCollectionCls match {
+ case Some(cls) if classOf[WrappedArray[_]].isAssignableFrom(cls) =>
+ // Scala WrappedArray
+ inputCollection => WrappedArray.make(executeFuncOnCollection(inputCollection).toArray)
+ case Some(cls) if classOf[scala.collection.Seq[_]].isAssignableFrom(cls) =>
+ // Scala sequence
+ executeFuncOnCollection(_).toSeq
+ case Some(cls) if classOf[scala.collection.Set[_]].isAssignableFrom(cls) =>
+ // Scala set
+ executeFuncOnCollection(_).toSet
+ case Some(cls) if classOf[java.util.List[_]].isAssignableFrom(cls) =>
+ // Java list
+ if (cls == classOf[java.util.List[_]] || cls == classOf[java.util.AbstractList[_]] ||
+ cls == classOf[java.util.AbstractSequentialList[_]]) {
+ // Specifying non concrete implementations of `java.util.List`
+ executeFuncOnCollection(_).toSeq.asJava
+ } else {
+ val constructors = cls.getConstructors()
+ val intParamConstructor = constructors.find { constructor =>
+ constructor.getParameterCount == 1 && constructor.getParameterTypes()(0) == classOf[Int]
+ }
+ val noParamConstructor = constructors.find { constructor =>
+ constructor.getParameterCount == 0
+ }
+
+ val constructor = intParamConstructor.map { intConstructor =>
+ (len: Int) => intConstructor.newInstance(len.asInstanceOf[Object])
+ }.getOrElse {
+ (_: Int) => noParamConstructor.get.newInstance()
+ }
+
+ // Specifying concrete implementations of `java.util.List`
+ (inputs) => {
+ val results = executeFuncOnCollection(inputs)
+ val builder = constructor(inputs.length).asInstanceOf[java.util.List[Any]]
+ results.foreach(builder.add(_))
+ builder
+ }
+ }
+ case None =>
+ // array
+ x => new GenericArrayData(executeFuncOnCollection(x).toArray)
+ case Some(cls) =>
+ throw QueryExecutionErrors.classUnsupportedByMapObjectsError(cls)
+ }
+
+ override def eval(input: InternalRow): Any = {
+ val inputCollection = inputData.eval(input)
+
+ if (inputCollection == null) {
+ return null
+ }
+ mapElements(convertToSeq(inputCollection))
+ }
+
+ override def dataType: DataType =
+ customCollectionCls.map(ObjectType.apply).getOrElse(
+ ArrayType(lambdaFunction.dataType, containsNull = lambdaFunction.nullable))
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val elementJavaType = CodeGenerator.javaType(loopVar.dataType)
+ val loopVarCode = LambdaVariable.prepareLambdaVariable(ctx, loopVar)
+ val genInputData = inputData.genCode(ctx)
+ val genFunction = lambdaFunction.genCode(ctx)
+ val dataLength = ctx.freshName("dataLength")
+ val convertedArray = ctx.freshName("convertedArray")
+ val loopIndex = ctx.freshName("loopIndex")
+
+ val convertedType = CodeGenerator.boxedType(lambdaFunction.dataType)
+
+ // Because of the way Java defines nested arrays, we have to handle the syntax specially.
+ // Specifically, we have to insert the [$dataLength] in between the type and any extra nested
+ // array declarations (i.e. new String[1][]).
+ val arrayConstructor = if (convertedType contains "[]") {
+ val rawType = convertedType.takeWhile(_ != '[')
+ val arrayPart = convertedType.reverse.takeWhile(c => c == '[' || c == ']').reverse
+ s"new $rawType[$dataLength]$arrayPart"
+ } else {
+ s"new $convertedType[$dataLength]"
+ }
+
+ // In RowEncoder, we use `Object` to represent Array or Seq, so we need to determine the type
+ // of input collection at runtime for this case.
+ val seq = ctx.freshName("seq")
+ val array = ctx.freshName("array")
+ val determineCollectionType = inputData.dataType match {
+ case ObjectType(cls) if cls == classOf[Object] =>
+ val seqClass = classOf[scala.collection.Seq[_]].getName
+ s"""
+ $seqClass $seq = null;
+ $elementJavaType[] $array = null;
+ if (${genInputData.value}.getClass().isArray()) {
+ $array = ($elementJavaType[]) ${genInputData.value};
+ } else {
+ $seq = ($seqClass) ${genInputData.value};
+ }
+ """
+ case _ => ""
+ }
+
+ // `MapObjects` generates a while loop to traverse the elements of the input collection. We
+ // need to take care of Seq and List because they may have O(n) complexity for indexed accessing
+ // like `list.get(1)`. Here we use Iterator to traverse Seq and List.
+ val (getLength, prepareLoop, getLoopVar) = inputDataType match {
+ case ObjectType(cls) if classOf[scala.collection.Seq[_]].isAssignableFrom(cls) =>
+ val it = ctx.freshName("it")
+ (
+ s"${genInputData.value}.size()",
+ s"scala.collection.Iterator $it = ${genInputData.value}.toIterator();",
+ s"$it.next()"
+ )
+ case ObjectType(cls) if cls.isArray =>
+ (
+ s"${genInputData.value}.length",
+ "",
+ s"${genInputData.value}[$loopIndex]"
+ )
+ case ObjectType(cls) if classOf[java.util.List[_]].isAssignableFrom(cls) =>
+ val it = ctx.freshName("it")
+ (
+ s"${genInputData.value}.size()",
+ s"java.util.Iterator $it = ${genInputData.value}.iterator();",
+ s"$it.next()"
+ )
+ case ArrayType(et, _) =>
+ (
+ s"${genInputData.value}.numElements()",
+ "",
+ CodeGenerator.getValue(genInputData.value, et, loopIndex)
+ )
+ case ObjectType(cls) if cls == classOf[Object] =>
+ val it = ctx.freshName("it")
+ (
+ s"$seq == null ? $array.length : $seq.size()",
+ s"scala.collection.Iterator $it = $seq == null ? null : $seq.toIterator();",
+ s"$it == null ? $array[$loopIndex] : $it.next()"
+ )
+ }
+
+ // Make a copy of the data if it's unsafe-backed
+ def makeCopyIfInstanceOf(clazz: Class[_ <: Any], value: String) =
+ s"$value instanceof ${clazz.getSimpleName}? ${value}.copy() : $value"
+ val genFunctionValue: String = lambdaFunction.dataType match {
+ case StructType(_) => makeCopyIfInstanceOf(classOf[UnsafeRow], genFunction.value)
+ case ArrayType(_, _) => makeCopyIfInstanceOf(classOf[UnsafeArrayData], genFunction.value)
+ case MapType(_, _, _) => makeCopyIfInstanceOf(classOf[UnsafeMapData], genFunction.value)
+ case _ => genFunction.value
+ }
+
+ val loopNullCheck = if (loopVar.nullable) {
+ inputDataType match {
+ case _: ArrayType => s"${loopVarCode.isNull} = ${genInputData.value}.isNullAt($loopIndex);"
+ case _ => s"${loopVarCode.isNull} = ${loopVarCode.value} == null;"
+ }
+ } else {
+ ""
+ }
+
+ val (initCollection, addElement, getResult): (String, String => String, String) =
+ customCollectionCls match {
+ case Some(cls) if classOf[WrappedArray[_]].isAssignableFrom(cls) =>
+ def doCodeGenForScala212 = {
+ // WrappedArray in Scala 2.12
+ val getBuilder = s"${cls.getName}$$.MODULE$$.newBuilder()"
+ val builder = ctx.freshName("collectionBuilder")
+ (
+ s"""
+ ${classOf[Builder[_, _]].getName} $builder = $getBuilder;
+ $builder.sizeHint($dataLength);
+ """,
+ (genValue: String) => s"$builder.$$plus$$eq($genValue);",
+ s"(${cls.getName}) ${classOf[WrappedArray[_]].getName}$$." +
+ s"MODULE$$.make(((${classOf[IndexedSeq[_]].getName})$builder" +
+ s".result()).toArray(scala.reflect.ClassTag$$.MODULE$$.Object()));"
+ )
+ }
+
+ def doCodeGenForScala213 = {
+ // In Scala 2.13, WrappedArray is mutable.ArraySeq and newBuilder method need
+ // a ClassTag type construction parameter
+ val getBuilder = s"${cls.getName}$$.MODULE$$.newBuilder(" +
+ s"scala.reflect.ClassTag$$.MODULE$$.Object())"
+ val builder = ctx.freshName("collectionBuilder")
+ (
+ s"""
+ ${classOf[Builder[_, _]].getName} $builder = $getBuilder;
+ $builder.sizeHint($dataLength);
+ """,
+ (genValue: String) => s"$builder.$$plus$$eq($genValue);",
+ s"(${cls.getName})$builder.result();"
+ )
+ }
+
+ val scalaVersion = Properties.versionNumberString
+ if (scalaVersion.startsWith("2.12")) {
+ doCodeGenForScala212
+ } else {
+ doCodeGenForScala213
+ }
+ case Some(cls) if classOf[scala.collection.Seq[_]].isAssignableFrom(cls) ||
+ classOf[scala.collection.Set[_]].isAssignableFrom(cls) =>
+ // Scala sequence or set
+ val getBuilder = s"${cls.getName}$$.MODULE$$.newBuilder()"
+ val builder = ctx.freshName("collectionBuilder")
+ (
+ s"""
+ ${classOf[Builder[_, _]].getName} $builder = $getBuilder;
+ $builder.sizeHint($dataLength);
+ """,
+ (genValue: String) => s"$builder.$$plus$$eq($genValue);",
+ s"(${cls.getName}) $builder.result();"
+ )
+ case Some(cls) if classOf[java.util.List[_]].isAssignableFrom(cls) =>
+ // Java list
+ val builder = ctx.freshName("collectionBuilder")
+ (
+ if (cls == classOf[java.util.List[_]] || cls == classOf[java.util.AbstractList[_]] ||
+ cls == classOf[java.util.AbstractSequentialList[_]]) {
+ s"${cls.getName} $builder = new java.util.ArrayList($dataLength);"
+ } else {
+ val param = Try(cls.getConstructor(Integer.TYPE)).map(_ => dataLength).getOrElse("")
+ s"${cls.getName} $builder = new ${cls.getName}($param);"
+ },
+ (genValue: String) => s"$builder.add($genValue);",
+ s"$builder;"
+ )
+ case _ =>
+ // array
+ (
+ s"""
+ $convertedType[] $convertedArray = null;
+ $convertedArray = $arrayConstructor;
+ """,
+ (genValue: String) => s"$convertedArray[$loopIndex] = $genValue;",
+ s"new ${classOf[GenericArrayData].getName}($convertedArray);"
+ )
+ }
+
+ val code = genInputData.code + code"""
+ ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
+
+ if (!${genInputData.isNull}) {
+ $determineCollectionType
+ int $dataLength = $getLength;
+ $initCollection
+
+ int $loopIndex = 0;
+ $prepareLoop
+ while ($loopIndex < $dataLength) {
+ ${loopVarCode.value} = ($elementJavaType) ($getLoopVar);
+ $loopNullCheck
+
+ ${genFunction.code}
+ if (${genFunction.isNull}) {
+ ${addElement("null")}
+ } else {
+ ${addElement(genFunctionValue)}
+ }
+
+ $loopIndex += 1;
+ }
+
+ ${ev.value} = $getResult
+ }
+ """
+ ev.copy(code = code, isNull = genInputData.isNull)
+ }
+
+ override protected def withNewChildrenInternal(
+ newFirst: Expression, newSecond: Expression, newThird: Expression): Expression =
+ copy(
+ loopVar = newFirst.asInstanceOf[LambdaVariable],
+ lambdaFunction = newSecond,
+ inputData = newThird)
+}
+
+/**
+ * Similar to [[UnresolvedMapObjects]], this is a placeholder of [[CatalystToExternalMap]].
+ *
+ * @param child An expression that when evaluated returns a map object.
+ * @param keyFunction The function applied on the key collection elements.
+ * @param valueFunction The function applied on the value collection elements.
+ * @param collClass The type of the resulting collection.
+ */
+case class UnresolvedCatalystToExternalMap(
+ child: Expression,
+ @transient keyFunction: Expression => Expression,
+ @transient valueFunction: Expression => Expression,
+ collClass: Class[_]) extends UnaryExpression with Unevaluable {
+
+ override lazy val resolved = false
+
+ override def dataType: DataType = ObjectType(collClass)
+
+ override protected def withNewChildInternal(
+ newChild: Expression): UnresolvedCatalystToExternalMap = copy(child = newChild)
+}
+
+object CatalystToExternalMap {
+ def apply(u: UnresolvedCatalystToExternalMap): CatalystToExternalMap = {
+ val mapType = u.child.dataType.asInstanceOf[MapType]
+ val keyLoopVar = LambdaVariable(
+ "CatalystToExternalMap_key", mapType.keyType, nullable = false)
+ val valueLoopVar = LambdaVariable(
+ "CatalystToExternalMap_value", mapType.valueType, mapType.valueContainsNull)
+ CatalystToExternalMap(
+ keyLoopVar, u.keyFunction(keyLoopVar),
+ valueLoopVar, u.valueFunction(valueLoopVar),
+ u.child, u.collClass)
+ }
+}
+
+/**
+ * Expression used to convert a Catalyst Map to an external Scala Map.
+ * The collection is constructed using the associated builder, obtained by calling `newBuilder`
+ * on the collection's companion object.
+ *
+ * @param keyLoopVar the [[LambdaVariable]] expression representing the loop variable that is used
+ * when iterating over the key collection, and which is used as input for the
+ * `keyLambdaFunction`.
+ * @param keyLambdaFunction A function that takes the `keyLoopVar` as input, and is used as
+ * a lambda function to handle collection elements.
+ * @param valueLoopVar the [[LambdaVariable]] expression representing the loop variable that is used
+ * when iterating over the value collection, and which is used as input for the
+ * `valueLambdaFunction`.
+ * @param valueLambdaFunction A function that takes the `valueLoopVar` as input, and is used as
+ * a lambda function to handle collection elements.
+ * @param inputData An expression that when evaluated returns a map object.
+ * @param collClass The type of the resulting collection.
+ */
+case class CatalystToExternalMap private(
+ keyLoopVar: LambdaVariable,
+ keyLambdaFunction: Expression,
+ valueLoopVar: LambdaVariable,
+ valueLambdaFunction: Expression,
+ inputData: Expression,
+ collClass: Class[_]) extends Expression with NonSQLExpression {
+
+ override def nullable: Boolean = inputData.nullable
+
+ override def children: Seq[Expression] = Seq(
+ keyLoopVar, keyLambdaFunction, valueLoopVar, valueLambdaFunction, inputData)
+
+ private lazy val inputMapType = inputData.dataType.asInstanceOf[MapType]
+
+ private lazy val (newMapBuilderMethod, moduleField) = {
+ val clazz = Utils.classForName(collClass.getCanonicalName + "$")
+ (clazz.getMethod("newBuilder"), clazz.getField("MODULE$").get(null))
+ }
+
+ private def newMapBuilder(): Builder[AnyRef, AnyRef] = {
+ newMapBuilderMethod.invoke(moduleField).asInstanceOf[Builder[AnyRef, AnyRef]]
+ }
+
+ override def eval(input: InternalRow): Any = {
+ val result = inputData.eval(input).asInstanceOf[MapData]
+ if (result != null) {
+ val builder = newMapBuilder()
+ builder.sizeHint(result.numElements())
+ val keyArray = result.keyArray()
+ val valueArray = result.valueArray()
+ val row = new GenericInternalRow(1)
+ var i = 0
+ while (i < result.numElements()) {
+ row.update(0, keyArray.get(i, inputMapType.keyType))
+ val key = keyLambdaFunction.eval(row)
+ row.update(0, valueArray.get(i, inputMapType.valueType))
+ val value = valueLambdaFunction.eval(row)
+ builder += Tuple2(key, value)
+ i += 1
+ }
+ builder.result()
+ } else {
+ null
+ }
+ }
+
+ override def dataType: DataType = ObjectType(collClass)
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val keyCode = LambdaVariable.prepareLambdaVariable(ctx, keyLoopVar)
+ val valueCode = LambdaVariable.prepareLambdaVariable(ctx, valueLoopVar)
+ val keyElementJavaType = CodeGenerator.javaType(keyLoopVar.dataType)
+ val genKeyFunction = keyLambdaFunction.genCode(ctx)
+ val valueElementJavaType = CodeGenerator.javaType(valueLoopVar.dataType)
+ val genValueFunction = valueLambdaFunction.genCode(ctx)
+ val genInputData = inputData.genCode(ctx)
+ val dataLength = ctx.freshName("dataLength")
+ val loopIndex = ctx.freshName("loopIndex")
+ val tupleLoopValue = ctx.freshName("tupleLoopValue")
+ val builderValue = ctx.freshName("builderValue")
+
+ val keyArray = ctx.freshName("keyArray")
+ val valueArray = ctx.freshName("valueArray")
+ val getKeyLoopVar = CodeGenerator.getValue(keyArray, keyLoopVar.dataType, loopIndex)
+ val getValueLoopVar = CodeGenerator.getValue(valueArray, valueLoopVar.dataType, loopIndex)
+
+ // Make a copy of the data if it's unsafe-backed
+ def makeCopyIfInstanceOf(clazz: Class[_ <: Any], value: String) =
+ s"$value instanceof ${clazz.getSimpleName}? $value.copy() : $value"
+ def genFunctionValue(lambdaFunction: Expression, genFunction: ExprCode) =
+ lambdaFunction.dataType match {
+ case StructType(_) => makeCopyIfInstanceOf(classOf[UnsafeRow], genFunction.value)
+ case ArrayType(_, _) => makeCopyIfInstanceOf(classOf[UnsafeArrayData], genFunction.value)
+ case MapType(_, _, _) => makeCopyIfInstanceOf(classOf[UnsafeMapData], genFunction.value)
+ case _ => genFunction.value
+ }
+ val genKeyFunctionValue = genFunctionValue(keyLambdaFunction, genKeyFunction)
+ val genValueFunctionValue = genFunctionValue(valueLambdaFunction, genValueFunction)
+
+ val valueLoopNullCheck = if (valueLoopVar.nullable) {
+ s"${valueCode.isNull} = $valueArray.isNullAt($loopIndex);"
+ } else {
+ ""
+ }
+
+ val builderClass = classOf[Builder[_, _]].getName
+ val constructBuilder = s"""
+ $builderClass $builderValue = ${collClass.getName}$$.MODULE$$.newBuilder();
+ $builderValue.sizeHint($dataLength);
+ """
+
+ val tupleClass = classOf[(_, _)].getName
+ val appendToBuilder = s"""
+ $tupleClass $tupleLoopValue;
+
+ if (${genValueFunction.isNull}) {
+ $tupleLoopValue = new $tupleClass($genKeyFunctionValue, null);
+ } else {
+ $tupleLoopValue = new $tupleClass($genKeyFunctionValue, $genValueFunctionValue);
+ }
+
+ $builderValue.$$plus$$eq($tupleLoopValue);
+ """
+ val getBuilderResult = s"${ev.value} = (${collClass.getName}) $builderValue.result();"
+
+ val code = genInputData.code + code"""
+ ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
+
+ if (!${genInputData.isNull}) {
+ int $dataLength = ${genInputData.value}.numElements();
+ $constructBuilder
+ ArrayData $keyArray = ${genInputData.value}.keyArray();
+ ArrayData $valueArray = ${genInputData.value}.valueArray();
+
+ int $loopIndex = 0;
+ while ($loopIndex < $dataLength) {
+ ${keyCode.value} = ($keyElementJavaType) ($getKeyLoopVar);
+ ${valueCode.value} = ($valueElementJavaType) ($getValueLoopVar);
+ $valueLoopNullCheck
+
+ ${genKeyFunction.code}
+ ${genValueFunction.code}
+
+ $appendToBuilder
+
+ $loopIndex += 1;
+ }
+
+ $getBuilderResult
+ }
+ """
+ ev.copy(code = code, isNull = genInputData.isNull)
+ }
+
+ override protected def withNewChildrenInternal(
+ newChildren: IndexedSeq[Expression]): CatalystToExternalMap =
+ copy(
+ keyLoopVar = newChildren(0).asInstanceOf[LambdaVariable],
+ keyLambdaFunction = newChildren(1),
+ valueLoopVar = newChildren(2).asInstanceOf[LambdaVariable],
+ valueLambdaFunction = newChildren(3),
+ inputData = newChildren(4))
+}
+
+object ExternalMapToCatalyst {
+ def apply(
+ inputMap: Expression,
+ keyType: DataType,
+ keyConverter: Expression => Expression,
+ keyNullable: Boolean,
+ valueType: DataType,
+ valueConverter: Expression => Expression,
+ valueNullable: Boolean): ExternalMapToCatalyst = {
+ val keyLoopVar = LambdaVariable("ExternalMapToCatalyst_key", keyType, keyNullable)
+ val valueLoopVar = LambdaVariable("ExternalMapToCatalyst_value", valueType, valueNullable)
+ ExternalMapToCatalyst(
+ keyLoopVar,
+ keyConverter(keyLoopVar),
+ valueLoopVar,
+ valueConverter(valueLoopVar),
+ inputMap)
+ }
+}
+
+/**
+ * Converts a Scala/Java map object into catalyst format, by applying the key/value converter when
+ * iterate the map.
+ *
+ * @param keyLoopVar the [[LambdaVariable]] expression representing the loop variable that is used
+ * when iterating over the key collection, and which is used as input for the
+ * `keyConverter`.
+ * @param keyConverter A function that take the `key` as input, and converts it to catalyst format.
+ * @param valueLoopVar the [[LambdaVariable]] expression representing the loop variable that is used
+ * when iterating over the value collection, and which is used as input for the
+ * `valueConverter`.
+ * @param valueConverter A function that take the `value` as input, and converts it to catalyst
+ * format.
+ * @param inputData An expression that when evaluated returns the input map object.
+ */
+case class ExternalMapToCatalyst private(
+ keyLoopVar: LambdaVariable,
+ keyConverter: Expression,
+ valueLoopVar: LambdaVariable,
+ valueConverter: Expression,
+ inputData: Expression)
+ extends Expression with NonSQLExpression {
+
+ override def foldable: Boolean = false
+
+ override def nullable: Boolean = inputData.nullable
+
+ override def children: Seq[Expression] = Seq(
+ keyLoopVar, keyConverter, valueLoopVar, valueConverter, inputData)
+
+ override def dataType: MapType = MapType(
+ keyConverter.dataType, valueConverter.dataType, valueContainsNull = valueConverter.nullable)
+
+ private lazy val mapCatalystConverter: Any => (Array[Any], Array[Any]) = {
+ val rowBuffer = InternalRow.fromSeq(Array[Any](1))
+ def rowWrapper(data: Any): InternalRow = {
+ rowBuffer.update(0, data)
+ rowBuffer
+ }
+
+ inputData.dataType match {
+ case ObjectType(cls) if classOf[java.util.Map[_, _]].isAssignableFrom(cls) =>
+ (input: Any) => {
+ val data = input.asInstanceOf[java.util.Map[Any, Any]]
+ val keys = new Array[Any](data.size)
+ val values = new Array[Any](data.size)
+ val iter = data.entrySet().iterator()
+ var i = 0
+ while (iter.hasNext) {
+ val entry = iter.next()
+ val (key, value) = (entry.getKey, entry.getValue)
+ keys(i) = if (key != null) {
+ keyConverter.eval(rowWrapper(key))
+ } else {
+ throw QueryExecutionErrors.nullAsMapKeyNotAllowedError
+ }
+ values(i) = if (value != null) {
+ valueConverter.eval(rowWrapper(value))
+ } else {
+ null
+ }
+ i += 1
+ }
+ (keys, values)
+ }
+
+ case ObjectType(cls) if classOf[scala.collection.Map[_, _]].isAssignableFrom(cls) =>
+ (input: Any) => {
+ val data = input.asInstanceOf[scala.collection.Map[Any, Any]]
+ val keys = new Array[Any](data.size)
+ val values = new Array[Any](data.size)
+ var i = 0
+ for ((key, value) <- data) {
+ keys(i) = if (key != null) {
+ keyConverter.eval(rowWrapper(key))
+ } else {
+ throw QueryExecutionErrors.nullAsMapKeyNotAllowedError
+ }
+ values(i) = if (value != null) {
+ valueConverter.eval(rowWrapper(value))
+ } else {
+ null
+ }
+ i += 1
+ }
+ (keys, values)
+ }
+ }
+ }
+
+ override def eval(input: InternalRow): Any = {
+ val result = inputData.eval(input)
+ if (result != null) {
+ val (keys, values) = mapCatalystConverter(result)
+ new ArrayBasedMapData(new GenericArrayData(keys), new GenericArrayData(values))
+ } else {
+ null
+ }
+ }
+
+ override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val inputMap = inputData.genCode(ctx)
+ val genKeyConverter = keyConverter.genCode(ctx)
+ val genValueConverter = valueConverter.genCode(ctx)
+ val length = ctx.freshName("length")
+ val index = ctx.freshName("index")
+ val convertedKeys = ctx.freshName("convertedKeys")
+ val convertedValues = ctx.freshName("convertedValues")
+ val entry = ctx.freshName("entry")
+ val entries = ctx.freshName("entries")
+
+ val keyJavaType = CodeGenerator.javaType(keyLoopVar.dataType)
+ val valueJavaType = CodeGenerator.javaType(valueLoopVar.dataType)
+ val keyCode = LambdaVariable.prepareLambdaVariable(ctx, keyLoopVar)
+ val valueCode = LambdaVariable.prepareLambdaVariable(ctx, valueLoopVar)
+
+ val (defineEntries, defineKeyValue) = inputData.dataType match {
+ case ObjectType(cls) if classOf[java.util.Map[_, _]].isAssignableFrom(cls) =>
+ val javaIteratorCls = classOf[java.util.Iterator[_]].getName
+ val javaMapEntryCls = classOf[java.util.Map.Entry[_, _]].getName
+
+ val defineEntries =
+ s"final $javaIteratorCls $entries = ${inputMap.value}.entrySet().iterator();"
+
+ val defineKeyValue =
+ s"""
+ final $javaMapEntryCls $entry = ($javaMapEntryCls) $entries.next();
+ ${keyCode.value} = (${CodeGenerator.boxedType(keyJavaType)}) $entry.getKey();
+ ${valueCode.value} = (${CodeGenerator.boxedType(valueJavaType)}) $entry.getValue();
+ """
+
+ defineEntries -> defineKeyValue
+
+ case ObjectType(cls) if classOf[scala.collection.Map[_, _]].isAssignableFrom(cls) =>
+ val scalaIteratorCls = classOf[Iterator[_]].getName
+ val scalaMapEntryCls = classOf[Tuple2[_, _]].getName
+
+ val defineEntries = s"final $scalaIteratorCls $entries = ${inputMap.value}.iterator();"
+
+ val defineKeyValue =
+ s"""
+ final $scalaMapEntryCls $entry = ($scalaMapEntryCls) $entries.next();
+ ${keyCode.value} = (${CodeGenerator.boxedType(keyJavaType)}) $entry._1();
+ ${valueCode.value} = (${CodeGenerator.boxedType(valueJavaType)}) $entry._2();
+ """
+
+ defineEntries -> defineKeyValue
+ }
+
+ val keyNullCheck = if (keyLoopVar.nullable) {
+ s"${keyCode.isNull} = ${keyCode.value} == null;"
+ } else {
+ ""
+ }
+
+ val valueNullCheck = if (valueLoopVar.nullable) {
+ s"${valueCode.isNull} = ${valueCode.value} == null;"
+ } else {
+ ""
+ }
+
+ val arrayCls = classOf[GenericArrayData].getName
+ val mapCls = classOf[ArrayBasedMapData].getName
+ val convertedKeyType = CodeGenerator.boxedType(keyConverter.dataType)
+ val convertedValueType = CodeGenerator.boxedType(valueConverter.dataType)
+ val code = inputMap.code +
+ code"""
+ ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
+ if (!${inputMap.isNull}) {
+ final int $length = ${inputMap.value}.size();
+ final Object[] $convertedKeys = new Object[$length];
+ final Object[] $convertedValues = new Object[$length];
+ int $index = 0;
+ $defineEntries
+ while($entries.hasNext()) {
+ $defineKeyValue
+ $keyNullCheck
+ $valueNullCheck
+
+ ${genKeyConverter.code}
+ if (${genKeyConverter.isNull}) {
+ throw QueryExecutionErrors.nullAsMapKeyNotAllowedError();
+ } else {
+ $convertedKeys[$index] = ($convertedKeyType) ${genKeyConverter.value};
+ }
+
+ ${genValueConverter.code}
+ if (${genValueConverter.isNull}) {
+ $convertedValues[$index] = null;
+ } else {
+ $convertedValues[$index] = ($convertedValueType) ${genValueConverter.value};
+ }
+
+ $index++;
+ }
+
+ ${ev.value} = new $mapCls(new $arrayCls($convertedKeys), new $arrayCls($convertedValues));
+ }
+ """
+ ev.copy(code = code, isNull = inputMap.isNull)
+ }
+
+ override protected def withNewChildrenInternal(
+ newChildren: IndexedSeq[Expression]): ExternalMapToCatalyst =
+ copy(
+ keyLoopVar = newChildren(0).asInstanceOf[LambdaVariable],
+ keyConverter = newChildren(1),
+ valueLoopVar = newChildren(2).asInstanceOf[LambdaVariable],
+ valueConverter = newChildren(3),
+ inputData = newChildren(4))
+}
+
+/**
+ * Constructs a new external row, using the result of evaluating the specified expressions
+ * as content.
+ *
+ * @param children A list of expression to use as content of the external row.
+ */
+case class CreateExternalRow(children: Seq[Expression], schema: StructType)
+ extends Expression with NonSQLExpression {
+
+ override def dataType: DataType = ObjectType(classOf[Row])
+
+ override def nullable: Boolean = false
+
+ override def eval(input: InternalRow): Any = {
+ val values = children.map(_.eval(input)).toArray
+ new GenericRowWithSchema(values, schema)
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val rowClass = classOf[GenericRowWithSchema].getName
+ val values = ctx.freshName("values")
+
+ val childrenCodes = children.zipWithIndex.map { case (e, i) =>
+ val eval = e.genCode(ctx)
+ s"""
+ |${eval.code}
+ |if (${eval.isNull}) {
+ | $values[$i] = null;
+ |} else {
+ | $values[$i] = ${eval.value};
+ |}
+ """.stripMargin
+ }
+
+ val childrenCode = ctx.splitExpressionsWithCurrentInputs(
+ expressions = childrenCodes,
+ funcName = "createExternalRow",
+ extraArguments = "Object[]" -> values :: Nil)
+ val schemaField = ctx.addReferenceObj("schema", schema)
+
+ val code =
+ code"""
+ |Object[] $values = new Object[${children.size}];
+ |$childrenCode
+ |final ${classOf[Row].getName} ${ev.value} = new $rowClass($values, $schemaField);
+ """.stripMargin
+ ev.copy(code = code, isNull = FalseLiteral)
+ }
+
+ override protected def withNewChildrenInternal(
+ newChildren: IndexedSeq[Expression]): CreateExternalRow = copy(children = newChildren)
+}
+
+/**
+ * Serializes an input object using a generic serializer (Kryo or Java).
+ *
+ * @param kryo if true, use Kryo. Otherwise, use Java.
+ */
+case class EncodeUsingSerializer(child: Expression, kryo: Boolean)
+ extends UnaryExpression with NonSQLExpression with SerializerSupport {
+
+ override def nullSafeEval(input: Any): Any = {
+ serializerInstance.serialize(input).array()
+ }
+
+ override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val serializer = addImmutableSerializerIfNeeded(ctx)
+ // Code to serialize.
+ val input = child.genCode(ctx)
+ val javaType = CodeGenerator.javaType(dataType)
+ val serialize = s"$serializer.serialize(${input.value}, null).array()"
+
+ val code = input.code + code"""
+ final $javaType ${ev.value} =
+ ${input.isNull} ? ${CodeGenerator.defaultValue(dataType)} : $serialize;
+ """
+ ev.copy(code = code, isNull = input.isNull)
+ }
+
+ override def dataType: DataType = BinaryType
+
+ override protected def withNewChildInternal(newChild: Expression): EncodeUsingSerializer =
+ copy(child = newChild)
+}
+
+/**
+ * Serializes an input object using a generic serializer (Kryo or Java). Note that the ClassTag
+ * is not an implicit parameter because TreeNode cannot copy implicit parameters.
+ *
+ * @param kryo if true, use Kryo. Otherwise, use Java.
+ */
+case class DecodeUsingSerializer[T](child: Expression, tag: ClassTag[T], kryo: Boolean)
+ extends UnaryExpression with NonSQLExpression with SerializerSupport {
+
+ override def nullSafeEval(input: Any): Any = {
+ val inputBytes = java.nio.ByteBuffer.wrap(input.asInstanceOf[Array[Byte]])
+ serializerInstance.deserialize(inputBytes)
+ }
+
+ override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val serializer = addImmutableSerializerIfNeeded(ctx)
+ // Code to deserialize.
+ val input = child.genCode(ctx)
+ val javaType = CodeGenerator.javaType(dataType)
+ val deserialize =
+ s"($javaType) $serializer.deserialize(java.nio.ByteBuffer.wrap(${input.value}), null)"
+
+ val code = input.code + code"""
+ final $javaType ${ev.value} =
+ ${input.isNull} ? ${CodeGenerator.defaultValue(dataType)} : $deserialize;
+ """
+ ev.copy(code = code, isNull = input.isNull)
+ }
+
+ override def dataType: DataType = ObjectType(tag.runtimeClass)
+
+ override protected def withNewChildInternal(newChild: Expression): DecodeUsingSerializer[T] =
+ copy(child = newChild)
+}
+
+/**
+ * Initialize a Java Bean instance by setting its field values via setters.
+ */
+case class InitializeJavaBean(beanInstance: Expression, setters: Map[String, Expression])
+ extends Expression with NonSQLExpression {
+
+ override def nullable: Boolean = beanInstance.nullable
+ override def children: Seq[Expression] = beanInstance +: setters.values.toSeq
+ override def dataType: DataType = beanInstance.dataType
+
+ private lazy val resolvedSetters = {
+ assert(beanInstance.dataType.isInstanceOf[ObjectType])
+
+ val ObjectType(beanClass) = beanInstance.dataType
+ setters.map {
+ case (name, expr) =>
+ // Looking for known type mapping.
+ // But also looking for general `Object`-type parameter for generic methods.
+ val paramTypes = ScalaReflection.expressionJavaClasses(Seq(expr)) ++ Seq(classOf[Object])
+ val methods = paramTypes.flatMap { fieldClass =>
+ try {
+ Some(beanClass.getDeclaredMethod(name, fieldClass))
+ } catch {
+ case e: NoSuchMethodException => None
+ }
+ }
+ if (methods.isEmpty) {
+ throw QueryExecutionErrors.methodNotDeclaredError(name)
+ }
+ methods.head -> expr
+ }
+ }
+
+ override def eval(input: InternalRow): Any = {
+ val instance = beanInstance.eval(input)
+ if (instance != null) {
+ val bean = instance.asInstanceOf[Object]
+ resolvedSetters.foreach {
+ case (setter, expr) =>
+ val paramVal = expr.eval(input)
+ // We don't call setter if input value is null.
+ if (paramVal != null) {
+ setter.invoke(bean, paramVal.asInstanceOf[AnyRef])
+ }
+ }
+ }
+ instance
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val instanceGen = beanInstance.genCode(ctx)
+
+ val javaBeanInstance = ctx.freshName("javaBean")
+ val beanInstanceJavaType = CodeGenerator.javaType(beanInstance.dataType)
+
+ val initialize = setters.map {
+ case (setterMethod, fieldValue) =>
+ val fieldGen = fieldValue.genCode(ctx)
+ s"""
+ |${fieldGen.code}
+ |if (!${fieldGen.isNull}) {
+ | $javaBeanInstance.$setterMethod(${fieldGen.value});
+ |}
+ """.stripMargin
+ }
+ val initializeCode = ctx.splitExpressionsWithCurrentInputs(
+ expressions = initialize.toSeq,
+ funcName = "initializeJavaBean",
+ extraArguments = beanInstanceJavaType -> javaBeanInstance :: Nil)
+
+ val code = instanceGen.code +
+ code"""
+ |$beanInstanceJavaType $javaBeanInstance = ${instanceGen.value};
+ |if (!${instanceGen.isNull}) {
+ | $initializeCode
+ |}
+ """.stripMargin
+ ev.copy(code = code, isNull = instanceGen.isNull, value = instanceGen.value)
+ }
+
+ override protected def withNewChildrenInternal(
+ newChildren: IndexedSeq[Expression]): InitializeJavaBean =
+ super.legacyWithNewChildren(newChildren).asInstanceOf[InitializeJavaBean]
+}
+
+/**
+ * Asserts that input values of a non-nullable child expression are not null.
+ *
+ * Note that there are cases where `child.nullable == true`, while we still need to add this
+ * assertion. Consider a nullable column `s` whose data type is a struct containing a non-nullable
+ * `Int` field named `i`. Expression `s.i` is nullable because `s` can be null. However, for all
+ * non-null `s`, `s.i` can't be null.
+ */
+case class AssertNotNull(child: Expression, walkedTypePath: Seq[String] = Nil)
+ extends UnaryExpression with NonSQLExpression {
+
+ override def dataType: DataType = child.dataType
+ override def foldable: Boolean = false
+ override def nullable: Boolean = false
+
+ final override val nodePatterns: Seq[TreePattern] = Seq(NULL_CHECK)
+
+ override def flatArguments: Iterator[Any] = Iterator(child)
+
+ private val errMsg = "Null value appeared in non-nullable field:" +
+ walkedTypePath.mkString("\n", "\n", "\n") +
+ "If the schema is inferred from a Scala tuple/case class, or a Java bean, " +
+ "please try to use scala.Option[_] or other nullable types " +
+ "(e.g. java.lang.Integer instead of int/scala.Int)."
+
+ override def eval(input: InternalRow): Any = {
+ val result = child.eval(input)
+ if (result == null) {
+ throw new NullPointerException(errMsg)
+ }
+ result
+ }
+
+ override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val childGen = child.genCode(ctx)
+
+ // Use unnamed reference that doesn't create a local field here to reduce the number of fields
+ // because errMsgField is used only when the value is null.
+ val errMsgField = ctx.addReferenceObj("errMsg", errMsg)
+
+ val code = childGen.code + code"""
+ if (${childGen.isNull}) {
+ throw new NullPointerException($errMsgField);
+ }
+ """
+ ev.copy(code = code, isNull = FalseLiteral, value = childGen.value)
+ }
+
+ override protected def withNewChildInternal(newChild: Expression): AssertNotNull =
+ copy(child = newChild)
+}
+
+/**
+ * Returns the value of field at index `index` from the external row `child`.
+ * This class can be viewed as [[GetStructField]] for [[Row]]s instead of [[InternalRow]]s.
+ *
+ * Note that the input row and the field we try to get are both guaranteed to be not null, if they
+ * are null, a runtime exception will be thrown.
+ */
+case class GetExternalRowField(
+ child: Expression,
+ index: Int,
+ fieldName: String) extends UnaryExpression with NonSQLExpression {
+
+ override def nullable: Boolean = false
+
+ override def dataType: DataType = ObjectType(classOf[Object])
+
+ private val errMsg = QueryExecutionErrors.fieldCannotBeNullMsg(index, fieldName)
+
+ override def eval(input: InternalRow): Any = {
+ val inputRow = child.eval(input).asInstanceOf[Row]
+ if (inputRow == null) {
+ throw QueryExecutionErrors.inputExternalRowCannotBeNullError
+ }
+ if (inputRow.isNullAt(index)) {
+ throw QueryExecutionErrors.fieldCannotBeNullError(index, fieldName)
+ }
+ inputRow.get(index)
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ // Use unnamed reference that doesn't create a local field here to reduce the number of fields
+ // because errMsgField is used only when the field is null.
+ val errMsgField = ctx.addReferenceObj("errMsg", errMsg)
+ val row = child.genCode(ctx)
+ val code = code"""
+ ${row.code}
+
+ if (${row.isNull}) {
+ throw QueryExecutionErrors.inputExternalRowCannotBeNullError();
+ }
+
+ if (${row.value}.isNullAt($index)) {
+ throw new RuntimeException($errMsgField);
+ }
+
+ final Object ${ev.value} = ${row.value}.get($index);
+ """
+ ev.copy(code = code, isNull = FalseLiteral)
+ }
+
+ override protected def withNewChildInternal(newChild: Expression): GetExternalRowField =
+ copy(child = newChild)
+}
+
+/**
+ * Validates the actual data type of input expression at runtime. If it doesn't match the
+ * expectation, throw an exception.
+ */
+case class ValidateExternalType(child: Expression, expected: DataType)
+ extends UnaryExpression with NonSQLExpression with ExpectsInputTypes {
+
+ override def inputTypes: Seq[AbstractDataType] = Seq(ObjectType(classOf[Object]))
+
+ override def nullable: Boolean = child.nullable
+
+ override val dataType: DataType = RowEncoder.externalDataTypeForInput(expected)
+
+ private lazy val errMsg = s" is not a valid external type for schema of ${expected.simpleString}"
+
+ private lazy val checkType: (Any) => Boolean = expected match {
+ case _: DecimalType =>
+ (value: Any) => {
+ value.isInstanceOf[java.math.BigDecimal] || value.isInstanceOf[scala.math.BigDecimal] ||
+ value.isInstanceOf[Decimal]
+ }
+ case _: ArrayType =>
+ (value: Any) => {
+ value.getClass.isArray || value.isInstanceOf[Seq[_]]
+ }
+ case _ =>
+ val dataTypeClazz = ScalaReflection.javaBoxedType(dataType)
+ (value: Any) => {
+ dataTypeClazz.isInstance(value)
+ }
+ }
+
+ override def eval(input: InternalRow): Any = {
+ val result = child.eval(input)
+ if (checkType(result)) {
+ result
+ } else {
+ throw new RuntimeException(s"${result.getClass.getName}$errMsg")
+ }
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ // Use unnamed reference that doesn't create a local field here to reduce the number of fields
+ // because errMsgField is used only when the type doesn't match.
+ val errMsgField = ctx.addReferenceObj("errMsg", errMsg)
+ val input = child.genCode(ctx)
+ val obj = input.value
+
+ val typeCheck = expected match {
+ case _: DecimalType =>
+ Seq(classOf[java.math.BigDecimal], classOf[scala.math.BigDecimal], classOf[Decimal])
+ .map(cls => s"$obj instanceof ${cls.getName}").mkString(" || ")
+ case _: ArrayType =>
+ s"$obj.getClass().isArray() || $obj instanceof ${classOf[scala.collection.Seq[_]].getName}"
+ case _ =>
+ s"$obj instanceof ${CodeGenerator.boxedType(dataType)}"
+ }
+
+ val code = code"""
+ ${input.code}
+ ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
+ if (!${input.isNull}) {
+ if ($typeCheck) {
+ ${ev.value} = (${CodeGenerator.boxedType(dataType)}) $obj;
+ } else {
+ throw new RuntimeException($obj.getClass().getName() + $errMsgField);
+ }
+ }
+
+ """
+ ev.copy(code = code, isNull = input.isNull)
+ }
+
+ override protected def withNewChildInternal(newChild: Expression): ValidateExternalType =
+ copy(child = newChild)
+}
diff --git a/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
new file mode 100644
index 000000000..ae320f8c6
--- /dev/null
+++ b/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
@@ -0,0 +1,909 @@
+/*
+ * 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.spark.sql.catalyst.expressions
+
+import java.util.Locale
+import java.util.regex.{Matcher, MatchResult, Pattern}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.commons.text.StringEscapeUtils
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.trees.TreePattern.{LIKE_FAMLIY, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, TreePattern}
+import org.apache.spark.sql.catalyst.util.{GenericArrayData, StringUtils}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
+
+
+abstract class StringRegexExpression extends BinaryExpression
+ with ImplicitCastInputTypes with NullIntolerant {
+
+ def escape(v: String): String
+ def matches(regex: Pattern, str: String): Boolean
+
+ override def dataType: DataType = BooleanType
+ override def inputTypes: Seq[DataType] = Seq(StringType, StringType)
+
+ // try cache foldable pattern
+ private lazy val cache: Pattern = right match {
+ case p: Expression if p.foldable =>
+ compile(p.eval().asInstanceOf[UTF8String].toString)
+ case _ => null
+ }
+
+ protected def compile(str: String): Pattern = if (str == null) {
+ null
+ } else {
+ // Let it raise exception if couldn't compile the regex string
+ Pattern.compile(escape(str))
+ }
+
+ protected def pattern(str: String) = if (cache == null) compile(str) else cache
+
+ protected override def nullSafeEval(input1: Any, input2: Any): Any = {
+ val regex = pattern(input2.asInstanceOf[UTF8String].toString)
+ if(regex == null) {
+ null
+ } else {
+ matches(regex, input1.asInstanceOf[UTF8String].toString)
+ }
+ }
+}
+
+// scalastyle:off line.contains.tab
+/**
+ * Simple RegEx pattern matching function
+ */
+@ExpressionDescription(
+ usage = "str _FUNC_ pattern[ ESCAPE escape] - Returns true if str matches `pattern` with " +
+ "`escape`, null if any arguments are null, false otherwise.",
+ arguments = """
+ Arguments:
+ * str - a string expression
+ * pattern - a string expression. The pattern is a string which is matched literally, with
+ exception to the following special symbols:
+
+ _ matches any one character in the input (similar to . in posix regular expressions)
+
+ % matches zero or more characters in the input (similar to .* in posix regular
+ expressions)
+
+ Since Spark 2.0, string literals are unescaped in our SQL parser. For example, in order
+ to match "\abc", the pattern should be "\\abc".
+
+ When SQL config 'spark.sql.parser.escapedStringLiterals' is enabled, it falls back
+ to Spark 1.6 behavior regarding string literal parsing. For example, if the config is
+ enabled, the pattern to match "\abc" should be "\abc".
+ * escape - an character added since Spark 3.0. The default escape character is the '\'.
+ If an escape character precedes a special symbol or another escape character, the
+ following character is matched literally. It is invalid to escape any other character.
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_('Spark', '_park');
+ true
+ > SET spark.sql.parser.escapedStringLiterals=true;
+ spark.sql.parser.escapedStringLiterals true
+ > SELECT '%SystemDrive%\Users\John' _FUNC_ '\%SystemDrive\%\\Users%';
+ true
+ > SET spark.sql.parser.escapedStringLiterals=false;
+ spark.sql.parser.escapedStringLiterals false
+ > SELECT '%SystemDrive%\\Users\\John' _FUNC_ '\%SystemDrive\%\\\\Users%';
+ true
+ > SELECT '%SystemDrive%/Users/John' _FUNC_ '/%SystemDrive/%//Users%' ESCAPE '/';
+ true
+ """,
+ note = """
+ Use RLIKE to match with standard regular expressions.
+ """,
+ since = "1.0.0",
+ group = "predicate_funcs")
+// scalastyle:on line.contains.tab
+case class Like(left: Expression, right: Expression, escapeChar: Char)
+ extends StringRegexExpression {
+
+ def this(left: Expression, right: Expression) = this(left, right, '\\')
+
+ override def escape(v: String): String = StringUtils.escapeLikeRegex(v, escapeChar)
+
+ override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches()
+
+ final override val nodePatterns: Seq[TreePattern] = Seq(LIKE_FAMLIY)
+
+ override def toString: String = escapeChar match {
+ case '\\' => s"$left LIKE $right"
+ case c => s"$left LIKE $right ESCAPE '$c'"
+ }
+
+ override def sql: String = s"${left.sql} ${prettyName.toUpperCase(Locale.ROOT)} ${right.sql}"
+
+ override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val patternClass = classOf[Pattern].getName
+ val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + ".escapeLikeRegex"
+
+ if (right.foldable) {
+ val rVal = right.eval()
+ if (rVal != null) {
+ val regexStr =
+ StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
+ val pattern = ctx.addMutableState(patternClass, "patternLike",
+ v => s"""$v = $patternClass.compile("$regexStr");""")
+
+ // We don't use nullSafeCodeGen here because we don't want to re-evaluate right again.
+ val eval = left.genCode(ctx)
+ ev.copy(code = code"""
+ ${eval.code}
+ boolean ${ev.isNull} = ${eval.isNull};
+ ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
+ if (!${ev.isNull}) {
+ ${ev.value} = $pattern.matcher(${eval.value}.toString()).matches();
+ }
+ """)
+ } else {
+ ev.copy(code = code"""
+ boolean ${ev.isNull} = true;
+ ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
+ """)
+ }
+ } else {
+ val pattern = ctx.freshName("pattern")
+ val rightStr = ctx.freshName("rightStr")
+ // We need to escape the escapeChar to make sure the generated code is valid.
+ // Otherwise we'll hit org.codehaus.commons.compiler.CompileException.
+ val escapedEscapeChar = StringEscapeUtils.escapeJava(escapeChar.toString)
+ nullSafeCodeGen(ctx, ev, (eval1, eval2) => {
+ s"""
+ String $rightStr = $eval2.toString();
+ $patternClass $pattern = $patternClass.compile(
+ $escapeFunc($rightStr, '$escapedEscapeChar'));
+ ${ev.value} = $pattern.matcher($eval1.toString()).matches();
+ """
+ })
+ }
+ }
+
+ override protected def withNewChildrenInternal(newLeft: Expression, newRight: Expression): Like =
+ copy(left = newLeft, right = newRight)
+}
+
+sealed abstract class MultiLikeBase
+ extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant {
+
+ protected def patterns: Seq[UTF8String]
+
+ protected def isNotSpecified: Boolean
+
+ override def inputTypes: Seq[DataType] = StringType :: Nil
+
+ override def dataType: DataType = BooleanType
+
+ override def nullable: Boolean = true
+
+ final override val nodePatterns: Seq[TreePattern] = Seq(LIKE_FAMLIY)
+
+ protected lazy val hasNull: Boolean = patterns.contains(null)
+
+ protected lazy val cache = patterns.filterNot(_ == null)
+ .map(s => Pattern.compile(StringUtils.escapeLikeRegex(s.toString, '\\')))
+
+ protected lazy val matchFunc = if (isNotSpecified) {
+ (p: Pattern, inputValue: String) => !p.matcher(inputValue).matches()
+ } else {
+ (p: Pattern, inputValue: String) => p.matcher(inputValue).matches()
+ }
+
+ protected def matches(exprValue: String): Any
+
+ override def eval(input: InternalRow): Any = {
+ val exprValue = child.eval(input)
+ if (exprValue == null) {
+ null
+ } else {
+ matches(exprValue.toString)
+ }
+ }
+}
+
+/**
+ * Optimized version of LIKE ALL, when all pattern values are literal.
+ */
+sealed abstract class LikeAllBase extends MultiLikeBase {
+
+ override def matches(exprValue: String): Any = {
+ if (cache.forall(matchFunc(_, exprValue))) {
+ if (hasNull) null else true
+ } else {
+ false
+ }
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val eval = child.genCode(ctx)
+ val patternClass = classOf[Pattern].getName
+ val javaDataType = CodeGenerator.javaType(child.dataType)
+ val pattern = ctx.freshName("pattern")
+ val valueArg = ctx.freshName("valueArg")
+ val patternCache = ctx.addReferenceObj("patternCache", cache.asJava)
+
+ val checkNotMatchCode = if (isNotSpecified) {
+ s"$pattern.matcher($valueArg.toString()).matches()"
+ } else {
+ s"!$pattern.matcher($valueArg.toString()).matches()"
+ }
+
+ ev.copy(code =
+ code"""
+ |${eval.code}
+ |boolean ${ev.isNull} = false;
+ |boolean ${ev.value} = true;
+ |if (${eval.isNull}) {
+ | ${ev.isNull} = true;
+ |} else {
+ | $javaDataType $valueArg = ${eval.value};
+ | for ($patternClass $pattern: $patternCache) {
+ | if ($checkNotMatchCode) {
+ | ${ev.value} = false;
+ | break;
+ | }
+ | }
+ | if (${ev.value} && $hasNull) ${ev.isNull} = true;
+ |}
+ """.stripMargin)
+ }
+}
+
+case class LikeAll(child: Expression, patterns: Seq[UTF8String]) extends LikeAllBase {
+ override def isNotSpecified: Boolean = false
+ override protected def withNewChildInternal(newChild: Expression): LikeAll =
+ copy(child = newChild)
+}
+
+case class NotLikeAll(child: Expression, patterns: Seq[UTF8String]) extends LikeAllBase {
+ override def isNotSpecified: Boolean = true
+ override protected def withNewChildInternal(newChild: Expression): NotLikeAll =
+ copy(child = newChild)
+}
+
+/**
+ * Optimized version of LIKE ANY, when all pattern values are literal.
+ */
+sealed abstract class LikeAnyBase extends MultiLikeBase {
+
+ override def matches(exprValue: String): Any = {
+ if (cache.exists(matchFunc(_, exprValue))) {
+ true
+ } else {
+ if (hasNull) null else false
+ }
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val eval = child.genCode(ctx)
+ val patternClass = classOf[Pattern].getName
+ val javaDataType = CodeGenerator.javaType(child.dataType)
+ val pattern = ctx.freshName("pattern")
+ val valueArg = ctx.freshName("valueArg")
+ val patternCache = ctx.addReferenceObj("patternCache", cache.asJava)
+
+ val checkMatchCode = if (isNotSpecified) {
+ s"!$pattern.matcher($valueArg.toString()).matches()"
+ } else {
+ s"$pattern.matcher($valueArg.toString()).matches()"
+ }
+
+ ev.copy(code =
+ code"""
+ |${eval.code}
+ |boolean ${ev.isNull} = false;
+ |boolean ${ev.value} = false;
+ |if (${eval.isNull}) {
+ | ${ev.isNull} = true;
+ |} else {
+ | $javaDataType $valueArg = ${eval.value};
+ | for ($patternClass $pattern: $patternCache) {
+ | if ($checkMatchCode) {
+ | ${ev.value} = true;
+ | break;
+ | }
+ | }
+ | if (!${ev.value} && $hasNull) ${ev.isNull} = true;
+ |}
+ """.stripMargin)
+ }
+}
+
+case class LikeAny(child: Expression, patterns: Seq[UTF8String]) extends LikeAnyBase {
+ override def isNotSpecified: Boolean = false
+ override protected def withNewChildInternal(newChild: Expression): LikeAny =
+ copy(child = newChild)
+}
+
+case class NotLikeAny(child: Expression, patterns: Seq[UTF8String]) extends LikeAnyBase {
+ override def isNotSpecified: Boolean = true
+ override protected def withNewChildInternal(newChild: Expression): NotLikeAny =
+ copy(child = newChild)
+}
+
+// scalastyle:off line.contains.tab
+@ExpressionDescription(
+ usage = "_FUNC_(str, regexp) - Returns true if `str` matches `regexp`, or false otherwise.",
+ arguments = """
+ Arguments:
+ * str - a string expression
+ * regexp - a string expression. The regex string should be a Java regular expression.
+
+ Since Spark 2.0, string literals (including regex patterns) are unescaped in our SQL
+ parser. For example, to match "\abc", a regular expression for `regexp` can be
+ "^\\abc$".
+
+ There is a SQL config 'spark.sql.parser.escapedStringLiterals' that can be used to
+ fallback to the Spark 1.6 behavior regarding string literal parsing. For example,
+ if the config is enabled, the `regexp` that can match "\abc" is "^\abc$".
+ """,
+ examples = """
+ Examples:
+ > SET spark.sql.parser.escapedStringLiterals=true;
+ spark.sql.parser.escapedStringLiterals true
+ > SELECT _FUNC_('%SystemDrive%\Users\John', '%SystemDrive%\\Users.*');
+ true
+ > SET spark.sql.parser.escapedStringLiterals=false;
+ spark.sql.parser.escapedStringLiterals false
+ > SELECT _FUNC_('%SystemDrive%\\Users\\John', '%SystemDrive%\\\\Users.*');
+ true
+ """,
+ note = """
+ Use LIKE to match with simple string pattern.
+ """,
+ since = "1.0.0",
+ group = "predicate_funcs")
+// scalastyle:on line.contains.tab
+case class RLike(left: Expression, right: Expression) extends StringRegexExpression {
+
+ override def escape(v: String): String = v
+ override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0)
+ override def toString: String = s"RLIKE($left, $right)"
+ override def sql: String = s"${prettyName.toUpperCase(Locale.ROOT)}(${left.sql}, ${right.sql})"
+
+ override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val patternClass = classOf[Pattern].getName
+
+ if (right.foldable) {
+ val rVal = right.eval()
+ if (rVal != null) {
+ val regexStr =
+ StringEscapeUtils.escapeJava(rVal.asInstanceOf[UTF8String].toString())
+ val pattern = ctx.addMutableState(patternClass, "patternRLike",
+ v => s"""$v = $patternClass.compile("$regexStr");""")
+
+ // We don't use nullSafeCodeGen here because we don't want to re-evaluate right again.
+ val eval = left.genCode(ctx)
+ ev.copy(code = code"""
+ ${eval.code}
+ boolean ${ev.isNull} = ${eval.isNull};
+ ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
+ if (!${ev.isNull}) {
+ ${ev.value} = $pattern.matcher(${eval.value}.toString()).find(0);
+ }
+ """)
+ } else {
+ ev.copy(code = code"""
+ boolean ${ev.isNull} = true;
+ ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
+ """)
+ }
+ } else {
+ val rightStr = ctx.freshName("rightStr")
+ val pattern = ctx.freshName("pattern")
+ nullSafeCodeGen(ctx, ev, (eval1, eval2) => {
+ s"""
+ String $rightStr = $eval2.toString();
+ $patternClass $pattern = $patternClass.compile($rightStr);
+ ${ev.value} = $pattern.matcher($eval1.toString()).find(0);
+ """
+ })
+ }
+ }
+
+ override protected def withNewChildrenInternal(newLeft: Expression, newRight: Expression): RLike =
+ copy(left = newLeft, right = newRight)
+}
+
+
+/**
+ * Splits str around matches of the given regex.
+ */
+@ExpressionDescription(
+ usage = "_FUNC_(str, regex, limit) - Splits `str` around occurrences that match `regex`" +
+ " and returns an array with a length of at most `limit`",
+ arguments = """
+ Arguments:
+ * str - a string expression to split.
+ * regex - a string representing a regular expression. The regex string should be a
+ Java regular expression.
+ * limit - an integer expression which controls the number of times the regex is applied.
+ * limit > 0: The resulting array's length will not be more than `limit`,
+ and the resulting array's last entry will contain all input
+ beyond the last matched regex.
+ * limit <= 0: `regex` will be applied as many times as possible, and
+ the resulting array can be of any size.
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_('oneAtwoBthreeC', '[ABC]');
+ ["one","two","three",""]
+ > SELECT _FUNC_('oneAtwoBthreeC', '[ABC]', -1);
+ ["one","two","three",""]
+ > SELECT _FUNC_('oneAtwoBthreeC', '[ABC]', 2);
+ ["one","twoBthreeC"]
+ """,
+ since = "1.5.0",
+ group = "string_funcs")
+case class StringSplit(str: Expression, regex: Expression, limit: Expression)
+ extends TernaryExpression with ImplicitCastInputTypes with NullIntolerant {
+
+ override def dataType: DataType = ArrayType(StringType)
+ override def inputTypes: Seq[DataType] = Seq(StringType, StringType, IntegerType)
+ override def first: Expression = str
+ override def second: Expression = regex
+ override def third: Expression = limit
+
+ def this(exp: Expression, regex: Expression) = this(exp, regex, Literal(-1));
+
+ override def nullSafeEval(string: Any, regex: Any, limit: Any): Any = {
+ val strings = string.asInstanceOf[UTF8String].split(
+ regex.asInstanceOf[UTF8String], limit.asInstanceOf[Int])
+ new GenericArrayData(strings.asInstanceOf[Array[Any]])
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val arrayClass = classOf[GenericArrayData].getName
+ nullSafeCodeGen(ctx, ev, (str, regex, limit) => {
+ // Array in java is covariant, so we don't need to cast UTF8String[] to Object[].
+ s"""${ev.value} = new $arrayClass($str.split($regex,$limit));""".stripMargin
+ })
+ }
+
+ override def prettyName: String = "split"
+
+ override protected def withNewChildrenInternal(
+ newFirst: Expression, newSecond: Expression, newThird: Expression): StringSplit =
+ copy(str = newFirst, regex = newSecond, limit = newThird)
+}
+
+
+/**
+ * Replace all substrings of str that match regexp with rep.
+ *
+ * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status.
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = "_FUNC_(str, regexp, rep[, position]) - Replaces all substrings of `str` that match `regexp` with `rep`.",
+ arguments = """
+ Arguments:
+ * str - a string expression to search for a regular expression pattern match.
+ * regexp - a string representing a regular expression. The regex string should be a
+ Java regular expression.
+
+ Since Spark 2.0, string literals (including regex patterns) are unescaped in our SQL
+ parser. For example, to match "\abc", a regular expression for `regexp` can be
+ "^\\abc$".
+
+ There is a SQL config 'spark.sql.parser.escapedStringLiterals' that can be used to
+ fallback to the Spark 1.6 behavior regarding string literal parsing. For example,
+ if the config is enabled, the `regexp` that can match "\abc" is "^\abc$".
+ * rep - a string expression to replace matched substrings.
+ * position - a positive integer literal that indicates the position within `str` to begin searching.
+ The default is 1. If position is greater than the number of characters in `str`, the result is `str`.
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_('100-200', '(\\d+)', 'num');
+ num-num
+ """,
+ since = "1.5.0",
+ group = "string_funcs")
+// scalastyle:on line.size.limit
+case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expression, pos: Expression)
+ extends QuaternaryExpression with ImplicitCastInputTypes with NullIntolerant {
+
+ def this(subject: Expression, regexp: Expression, rep: Expression) =
+ this(subject, regexp, rep, Literal(1))
+
+ override def checkInputDataTypes(): TypeCheckResult = {
+ if (!pos.foldable) {
+ return TypeCheckFailure(s"Position expression must be foldable, but got $pos")
+ }
+
+ val posEval = pos.eval()
+ if (posEval == null || posEval.asInstanceOf[Int] > 0) {
+ TypeCheckSuccess
+ } else {
+ TypeCheckFailure(s"Position expression must be positive, but got: $posEval")
+ }
+ }
+
+ // last regex in string, we will update the pattern iff regexp value changed.
+ @transient private var lastRegex: UTF8String = _
+ // last regex pattern, we cache it for performance concern
+ @transient private var pattern: Pattern = _
+ // last replacement string, we don't want to convert a UTF8String => java.langString every time.
+ @transient private var lastReplacement: String = _
+ @transient private var lastReplacementInUTF8: UTF8String = _
+ // result buffer write by Matcher
+ @transient private lazy val result: StringBuffer = new StringBuffer
+ final override val nodePatterns: Seq[TreePattern] = Seq(REGEXP_REPLACE)
+
+ override def nullSafeEval(s: Any, p: Any, r: Any, i: Any): Any = {
+ if (!p.equals(lastRegex)) {
+ // regex value changed
+ lastRegex = p.asInstanceOf[UTF8String].clone()
+ pattern = Pattern.compile(lastRegex.toString)
+ }
+ if (!r.equals(lastReplacementInUTF8)) {
+ // replacement string changed
+ lastReplacementInUTF8 = r.asInstanceOf[UTF8String].clone()
+ lastReplacement = lastReplacementInUTF8.toString
+ }
+ val source = s.toString()
+ val position = i.asInstanceOf[Int] - 1
+ if (position < source.length) {
+ val m = pattern.matcher(source)
+ m.region(position, source.length)
+ result.delete(0, result.length())
+ while (m.find) {
+ m.appendReplacement(result, lastReplacement)
+ }
+ m.appendTail(result)
+ UTF8String.fromString(result.toString)
+ } else {
+ s
+ }
+ }
+
+ override def dataType: DataType = StringType
+ override def inputTypes: Seq[AbstractDataType] =
+ Seq(StringType, StringType, StringType, IntegerType)
+ override def prettyName: String = "regexp_replace"
+
+ override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val termResult = ctx.freshName("termResult")
+
+ val classNamePattern = classOf[Pattern].getCanonicalName
+ val classNameStringBuffer = classOf[java.lang.StringBuffer].getCanonicalName
+
+ val matcher = ctx.freshName("matcher")
+ val source = ctx.freshName("source")
+ val position = ctx.freshName("position")
+
+ val termLastRegex = ctx.addMutableState("UTF8String", "lastRegex")
+ val termPattern = ctx.addMutableState(classNamePattern, "pattern")
+ val termLastReplacement = ctx.addMutableState("String", "lastReplacement")
+ val termLastReplacementInUTF8 = ctx.addMutableState("UTF8String", "lastReplacementInUTF8")
+
+ val setEvNotNull = if (nullable) {
+ s"${ev.isNull} = false;"
+ } else {
+ ""
+ }
+
+ nullSafeCodeGen(ctx, ev, (subject, regexp, rep, pos) => {
+ s"""
+ if (!$regexp.equals($termLastRegex)) {
+ // regex value changed
+ $termLastRegex = $regexp.clone();
+ $termPattern = $classNamePattern.compile($termLastRegex.toString());
+ }
+ if (!$rep.equals($termLastReplacementInUTF8)) {
+ // replacement string changed
+ $termLastReplacementInUTF8 = $rep.clone();
+ $termLastReplacement = $termLastReplacementInUTF8.toString();
+ }
+ String $source = $subject.toString();
+ int $position = $pos - 1;
+ if ($position < $source.length()) {
+ $classNameStringBuffer $termResult = new $classNameStringBuffer();
+ java.util.regex.Matcher $matcher = $termPattern.matcher($source);
+ $matcher.region($position, $source.length());
+
+ while ($matcher.find()) {
+ $matcher.appendReplacement($termResult, $termLastReplacement);
+ }
+ $matcher.appendTail($termResult);
+ ${ev.value} = UTF8String.fromString($termResult.toString());
+ $termResult = null;
+ } else {
+ ${ev.value} = $subject;
+ }
+ $setEvNotNull
+ """
+ })
+ }
+
+ override def first: Expression = subject
+ override def second: Expression = regexp
+ override def third: Expression = rep
+ override def fourth: Expression = pos
+
+ override protected def withNewChildrenInternal(
+ first: Expression, second: Expression, third: Expression, fourth: Expression): RegExpReplace =
+ copy(subject = first, regexp = second, rep = third, pos = fourth)
+}
+
+object RegExpReplace {
+ def apply(subject: Expression, regexp: Expression, rep: Expression): RegExpReplace =
+ new RegExpReplace(subject, regexp, rep)
+}
+
+object RegExpExtractBase {
+ def checkGroupIndex(groupCount: Int, groupIndex: Int): Unit = {
+ if (groupIndex < 0) {
+ throw QueryExecutionErrors.regexGroupIndexLessThanZeroError
+ } else if (groupCount < groupIndex) {
+ throw QueryExecutionErrors.regexGroupIndexExceedGroupCountError(
+ groupCount, groupIndex)
+ }
+ }
+}
+
+abstract class RegExpExtractBase
+ extends TernaryExpression with ImplicitCastInputTypes with NullIntolerant {
+ def subject: Expression
+ def regexp: Expression
+ def idx: Expression
+
+ // last regex in string, we will update the pattern iff regexp value changed.
+ @transient private var lastRegex: UTF8String = _
+ // last regex pattern, we cache it for performance concern
+ @transient private var pattern: Pattern = _
+
+ final override val nodePatterns: Seq[TreePattern] = Seq(REGEXP_EXTRACT_FAMILY)
+
+ override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, IntegerType)
+ override def first: Expression = subject
+ override def second: Expression = regexp
+ override def third: Expression = idx
+
+ protected def getLastMatcher(s: Any, p: Any): Matcher = {
+ if (p != lastRegex) {
+ // regex value changed
+ lastRegex = p.asInstanceOf[UTF8String].clone()
+ pattern = Pattern.compile(lastRegex.toString)
+ }
+ pattern.matcher(s.toString)
+ }
+}
+
+/**
+ * Extract a specific(idx) group identified by a Java regex.
+ *
+ * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status.
+ */
+@ExpressionDescription(
+ usage = """
+ _FUNC_(str, regexp[, idx]) - Extract the first string in the `str` that match the `regexp`
+ expression and corresponding to the regex group index.
+ """,
+ arguments = """
+ Arguments:
+ * str - a string expression.
+ * regexp - a string representing a regular expression. The regex string should be a
+ Java regular expression.
+
+ Since Spark 2.0, string literals (including regex patterns) are unescaped in our SQL
+ parser. For example, to match "\abc", a regular expression for `regexp` can be
+ "^\\abc$".
+
+ There is a SQL config 'spark.sql.parser.escapedStringLiterals' that can be used to
+ fallback to the Spark 1.6 behavior regarding string literal parsing. For example,
+ if the config is enabled, the `regexp` that can match "\abc" is "^\abc$".
+ * idx - an integer expression that representing the group index. The regex maybe contains
+ multiple groups. `idx` indicates which regex group to extract. The group index should
+ be non-negative. The minimum value of `idx` is 0, which means matching the entire
+ regular expression. If `idx` is not specified, the default group index value is 1. The
+ `idx` parameter is the Java regex Matcher group() method index.
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_('100-200', '(\\d+)-(\\d+)', 1);
+ 100
+ """,
+ since = "1.5.0",
+ group = "string_funcs")
+case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expression)
+ extends RegExpExtractBase {
+ def this(s: Expression, r: Expression) = this(s, r, Literal(1))
+
+ override def nullSafeEval(s: Any, p: Any, r: Any): Any = {
+ val m = getLastMatcher(s, p)
+ if (m.find) {
+ val mr: MatchResult = m.toMatchResult
+ val index = r.asInstanceOf[Int]
+ RegExpExtractBase.checkGroupIndex(mr.groupCount, index)
+ val group = mr.group(index)
+ if (group == null) { // Pattern matched, but it's an optional group
+ UTF8String.EMPTY_UTF8
+ } else {
+ UTF8String.fromString(group)
+ }
+ } else {
+ UTF8String.EMPTY_UTF8
+ }
+ }
+
+ override def dataType: DataType = StringType
+ override def prettyName: String = "regexp_extract"
+
+ override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val classNamePattern = classOf[Pattern].getCanonicalName
+ val classNameRegExpExtractBase = classOf[RegExpExtractBase].getCanonicalName
+ val matcher = ctx.freshName("matcher")
+ val matchResult = ctx.freshName("matchResult")
+
+ val termLastRegex = ctx.addMutableState("UTF8String", "lastRegex")
+ val termPattern = ctx.addMutableState(classNamePattern, "pattern")
+
+ val setEvNotNull = if (nullable) {
+ s"${ev.isNull} = false;"
+ } else {
+ ""
+ }
+
+ nullSafeCodeGen(ctx, ev, (subject, regexp, idx) => {
+ s"""
+ if (!$regexp.equals($termLastRegex)) {
+ // regex value changed
+ $termLastRegex = $regexp.clone();
+ $termPattern = $classNamePattern.compile($termLastRegex.toString());
+ }
+ java.util.regex.Matcher $matcher =
+ $termPattern.matcher($subject.toString());
+ if ($matcher.find()) {
+ java.util.regex.MatchResult $matchResult = $matcher.toMatchResult();
+ $classNameRegExpExtractBase.checkGroupIndex($matchResult.groupCount(), $idx);
+ if ($matchResult.group($idx) == null) {
+ ${ev.value} = UTF8String.EMPTY_UTF8;
+ } else {
+ ${ev.value} = UTF8String.fromString($matchResult.group($idx));
+ }
+ $setEvNotNull
+ } else {
+ ${ev.value} = UTF8String.EMPTY_UTF8;
+ $setEvNotNull
+ }"""
+ })
+ }
+
+ override protected def withNewChildrenInternal(
+ newFirst: Expression, newSecond: Expression, newThird: Expression): RegExpExtract =
+ copy(subject = newFirst, regexp = newSecond, idx = newThird)
+}
+
+/**
+ * Extract all specific(idx) groups identified by a Java regex.
+ *
+ * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status.
+ */
+@ExpressionDescription(
+ usage = """
+ _FUNC_(str, regexp[, idx]) - Extract all strings in the `str` that match the `regexp`
+ expression and corresponding to the regex group index.
+ """,
+ arguments = """
+ Arguments:
+ * str - a string expression.
+ * regexp - a string representing a regular expression. The regex string should be a
+ Java regular expression.
+
+ Since Spark 2.0, string literals (including regex patterns) are unescaped in our SQL
+ parser. For example, to match "\abc", a regular expression for `regexp` can be
+ "^\\abc$".
+
+ There is a SQL config 'spark.sql.parser.escapedStringLiterals' that can be used to
+ fallback to the Spark 1.6 behavior regarding string literal parsing. For example,
+ if the config is enabled, the `regexp` that can match "\abc" is "^\abc$".
+ * idx - an integer expression that representing the group index. The regex may contains
+ multiple groups. `idx` indicates which regex group to extract. The group index should
+ be non-negative. The minimum value of `idx` is 0, which means matching the entire
+ regular expression. If `idx` is not specified, the default group index value is 1. The
+ `idx` parameter is the Java regex Matcher group() method index.
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_('100-200, 300-400', '(\\d+)-(\\d+)', 1);
+ ["100","300"]
+ """,
+ since = "3.1.0",
+ group = "string_funcs")
+case class RegExpExtractAll(subject: Expression, regexp: Expression, idx: Expression)
+ extends RegExpExtractBase {
+ def this(s: Expression, r: Expression) = this(s, r, Literal(1))
+
+ override def nullSafeEval(s: Any, p: Any, r: Any): Any = {
+ val m = getLastMatcher(s, p)
+ val matchResults = new ArrayBuffer[UTF8String]()
+ while(m.find) {
+ val mr: MatchResult = m.toMatchResult
+ val index = r.asInstanceOf[Int]
+ RegExpExtractBase.checkGroupIndex(mr.groupCount, index)
+ val group = mr.group(index)
+ if (group == null) { // Pattern matched, but it's an optional group
+ matchResults += UTF8String.EMPTY_UTF8
+ } else {
+ matchResults += UTF8String.fromString(group)
+ }
+ }
+
+ new GenericArrayData(matchResults.toArray.asInstanceOf[Array[Any]])
+ }
+
+ override def dataType: DataType = ArrayType(StringType)
+ override def prettyName: String = "regexp_extract_all"
+
+ override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val classNamePattern = classOf[Pattern].getCanonicalName
+ val classNameRegExpExtractBase = classOf[RegExpExtractBase].getCanonicalName
+ val arrayClass = classOf[GenericArrayData].getName
+ val matcher = ctx.freshName("matcher")
+ val matchResult = ctx.freshName("matchResult")
+ val matchResults = ctx.freshName("matchResults")
+
+ val termLastRegex = ctx.addMutableState("UTF8String", "lastRegex")
+ val termPattern = ctx.addMutableState(classNamePattern, "pattern")
+
+ val setEvNotNull = if (nullable) {
+ s"${ev.isNull} = false;"
+ } else {
+ ""
+ }
+ nullSafeCodeGen(ctx, ev, (subject, regexp, idx) => {
+ s"""
+ | if (!$regexp.equals($termLastRegex)) {
+ | // regex value changed
+ | $termLastRegex = $regexp.clone();
+ | $termPattern = $classNamePattern.compile($termLastRegex.toString());
+ | }
+ | java.util.regex.Matcher $matcher = $termPattern.matcher($subject.toString());
+ | java.util.ArrayList $matchResults = new java.util.ArrayList();
+ | while ($matcher.find()) {
+ | java.util.regex.MatchResult $matchResult = $matcher.toMatchResult();
+ | $classNameRegExpExtractBase.checkGroupIndex($matchResult.groupCount(), $idx);
+ | if ($matchResult.group($idx) == null) {
+ | $matchResults.add(UTF8String.EMPTY_UTF8);
+ | } else {
+ | $matchResults.add(UTF8String.fromString($matchResult.group($idx)));
+ | }
+ | }
+ | ${ev.value} =
+ | new $arrayClass($matchResults.toArray(new UTF8String[$matchResults.size()]));
+ | $setEvNotNull
+ """
+ })
+ }
+
+ override protected def withNewChildrenInternal(
+ newFirst: Expression, newSecond: Expression, newThird: Expression): RegExpExtractAll =
+ copy(subject = newFirst, regexp = newSecond, idx = newThird)
+}
diff --git a/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala b/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
new file mode 100644
index 000000000..8063d7f1c
--- /dev/null
+++ b/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InjectRuntimeFilter.scala
@@ -0,0 +1,311 @@
+/*
+ * 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.spark.sql.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate, Complete}
+import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation}
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY, PYTHON_UDF, REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE, SCALA_UDF}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+/**
+ * Insert a filter on one side of the join if the other side has a selective predicate.
+ * The filter could be an IN subquery (converted to a semi join), a bloom filter, or something
+ * else in the future.
+ */
+object InjectRuntimeFilter extends Rule[LogicalPlan] with PredicateHelper with JoinSelectionHelper {
+
+ // Wraps `expr` with a hash function if its byte size is larger than an integer.
+ private def mayWrapWithHash(expr: Expression): Expression = {
+ if (expr.dataType.defaultSize > IntegerType.defaultSize) {
+ new Murmur3Hash(Seq(expr))
+ } else {
+ expr
+ }
+ }
+
+ private def injectFilter(
+ filterApplicationSideExp: Expression,
+ filterApplicationSidePlan: LogicalPlan,
+ filterCreationSideExp: Expression,
+ filterCreationSidePlan: LogicalPlan
+ ): LogicalPlan = {
+ require(conf.runtimeFilterBloomFilterEnabled || conf.runtimeFilterSemiJoinReductionEnabled)
+ if (conf.runtimeFilterBloomFilterEnabled) {
+ injectBloomFilter(
+ filterApplicationSideExp,
+ filterApplicationSidePlan,
+ filterCreationSideExp,
+ filterCreationSidePlan
+ )
+ } else {
+ injectInSubqueryFilter(
+ filterApplicationSideExp,
+ filterApplicationSidePlan,
+ filterCreationSideExp,
+ filterCreationSidePlan
+ )
+ }
+ }
+
+ private def injectBloomFilter(
+ filterApplicationSideExp: Expression,
+ filterApplicationSidePlan: LogicalPlan,
+ filterCreationSideExp: Expression,
+ filterCreationSidePlan: LogicalPlan
+ ): LogicalPlan = {
+ // Skip if the filter creation side is too big
+ if (filterCreationSidePlan.stats.sizeInBytes > conf.runtimeFilterBloomFilterThreshold) {
+ return filterApplicationSidePlan
+ }
+ val rowCount = filterCreationSidePlan.stats.rowCount
+ val bloomFilterAgg =
+ if (rowCount.isDefined && rowCount.get.longValue > 0L) {
+ new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)),
+ Literal(rowCount.get.longValue))
+ } else {
+ new BloomFilterAggregate(new XxHash64(Seq(filterCreationSideExp)))
+ }
+ val aggExp = AggregateExpression(bloomFilterAgg, Complete, isDistinct = false, None)
+ val alias = Alias(aggExp, "bloomFilter")()
+ val aggregate = ConstantFolding(Aggregate(Nil, Seq(alias), filterCreationSidePlan))
+ val bloomFilterSubquery = ScalarSubquery(aggregate, Nil)
+ val filter = BloomFilterMightContain(bloomFilterSubquery,
+ new XxHash64(Seq(filterApplicationSideExp)))
+ Filter(filter, filterApplicationSidePlan)
+ }
+
+ private def injectInSubqueryFilter(
+ filterApplicationSideExp: Expression,
+ filterApplicationSidePlan: LogicalPlan,
+ filterCreationSideExp: Expression,
+ filterCreationSidePlan: LogicalPlan
+ ): LogicalPlan = {
+ require(filterApplicationSideExp.dataType == filterCreationSideExp.dataType)
+ val actualFilterKeyExpr = mayWrapWithHash(filterCreationSideExp)
+ val alias = Alias(actualFilterKeyExpr, actualFilterKeyExpr.toString)()
+ val aggregate = Aggregate(Seq(alias), Seq(alias), filterCreationSidePlan)
+ if (!canBroadcastBySize(aggregate, conf)) {
+ // Skip the InSubquery filter if the size of `aggregate` is beyond broadcast join threshold,
+ // i.e., the semi-join will be a shuffled join, which is not worthwhile.
+ return filterApplicationSidePlan
+ }
+ val filter = InSubquery(Seq(mayWrapWithHash(filterApplicationSideExp)),
+ ListQuery(aggregate, childOutputs = aggregate.output))
+ Filter(filter, filterApplicationSidePlan)
+ }
+
+ /**
+ * Returns whether the plan is a simple filter over scan and the filter is likely selective
+ * Also check if the plan only has simple expressions (attribute reference, literals) so that we
+ * do not add a subquery that might have an expensive computation
+ */
+ private def isSelectiveFilterOverScan(plan: LogicalPlan): Boolean = {
+ plan.expressions
+ val ret = plan match {
+ case PhysicalOperation(_, filters, child) if child.isInstanceOf[LeafNode] =>
+ filters.forall(isSimpleExpression) &&
+ filters.exists(isLikelySelective)
+ case _ => false
+ }
+ !plan.isStreaming && ret
+ }
+
+ private def isSimpleExpression(e: Expression): Boolean = {
+ !e.containsAnyPattern(PYTHON_UDF, SCALA_UDF, INVOKE, JSON_TO_STRUCT, LIKE_FAMLIY,
+ REGEXP_EXTRACT_FAMILY, REGEXP_REPLACE)
+ }
+
+ /**
+ * Returns whether an expression is likely to be selective
+ */
+ private def isLikelySelective(e: Expression): Boolean = e match {
+ case Not(expr) => isLikelySelective(expr)
+ case And(l, r) => isLikelySelective(l) || isLikelySelective(r)
+ case Or(l, r) => isLikelySelective(l) && isLikelySelective(r)
+ case _: StringRegexExpression => true
+ case _: BinaryComparison => true
+ case _: In | _: InSet => true
+ case _: StringPredicate => true
+ case _: MultiLikeBase => true
+ case _ => false
+ }
+
+ private def canFilterLeft(joinType: JoinType): Boolean = joinType match {
+ case Inner | RightOuter => true
+ case _ => false
+ }
+
+ private def canFilterRight(joinType: JoinType): Boolean = joinType match {
+ case Inner | LeftOuter => true
+ case _ => false
+ }
+
+ private def isProbablyShuffleJoin(left: LogicalPlan,
+ right: LogicalPlan, hint: JoinHint): Boolean = {
+ !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+ !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf)
+ }
+
+ private def probablyHasShuffle(plan: LogicalPlan): Boolean = {
+ plan.collect {
+ case j@Join(left, right, _, _, hint)
+ if !hintToBroadcastLeft(hint) && !hintToBroadcastRight(hint) &&
+ !canBroadcastBySize(left, conf) && !canBroadcastBySize(right, conf) => j
+ case a: Aggregate => a
+ }.nonEmpty
+ }
+
+ // Returns the max scan byte size in the subtree rooted at `filterApplicationSide`.
+ private def maxScanByteSize(filterApplicationSide: LogicalPlan): BigInt = {
+ val defaultSizeInBytes = conf.getConf(SQLConf.DEFAULT_SIZE_IN_BYTES)
+ filterApplicationSide.collect({
+ case leaf: LeafNode => leaf
+ }).map(scan => {
+ // DEFAULT_SIZE_IN_BYTES means there's no byte size information in stats. Since we avoid
+ // creating a Bloom filter when the filter application side is very small, so using 0
+ // as the byte size when the actual size is unknown can avoid regression by applying BF
+ // on a small table.
+ if (scan.stats.sizeInBytes == defaultSizeInBytes) BigInt(0) else scan.stats.sizeInBytes
+ }).max
+ }
+
+ // Returns true if `filterApplicationSide` satisfies the byte size requirement to apply a
+ // Bloom filter; false otherwise.
+ private def satisfyByteSizeRequirement(filterApplicationSide: LogicalPlan): Boolean = {
+ // In case `filterApplicationSide` is a union of many small tables, disseminating the Bloom
+ // filter to each small task might be more costly than scanning them itself. Thus, we use max
+ // rather than sum here.
+ val maxScanSize = maxScanByteSize(filterApplicationSide)
+ maxScanSize >=
+ conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD)
+ }
+
+ private def filteringHasBenefit(
+ filterApplicationSide: LogicalPlan,
+ filterCreationSide: LogicalPlan,
+ filterApplicationSideExp: Expression,
+ hint: JoinHint): Boolean = {
+ // Check that:
+ // 1. The filterApplicationSideJoinExp can be pushed down through joins and aggregates (ie the
+ // expression references originate from a single leaf node)
+ // 2. The filter creation side has a selective predicate
+ // 3. The current join is a shuffle join or a broadcast join that has a shuffle or aggregate
+ // in the filter application side
+ // 4. The filterApplicationSide is larger than the filterCreationSide by a configurable
+ // threshold
+ findExpressionAndTrackLineageDown(filterApplicationSideExp,
+ filterApplicationSide).isDefined && isSelectiveFilterOverScan(filterCreationSide) &&
+ (isProbablyShuffleJoin(filterApplicationSide, filterCreationSide, hint) ||
+ probablyHasShuffle(filterApplicationSide)) &&
+ satisfyByteSizeRequirement(filterApplicationSide)
+ }
+
+ def hasRuntimeFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+ rightKey: Expression): Boolean = {
+ if (conf.runtimeFilterBloomFilterEnabled) {
+ hasBloomFilter(left, right, leftKey, rightKey)
+ } else {
+ hasInSubquery(left, right, leftKey, rightKey)
+ }
+ }
+
+ // This checks if there is already a DPP filter, as this rule is called just after DPP.
+ def hasDynamicPruningSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+ rightKey: Expression): Boolean = {
+ (left, right) match {
+ case (Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan), _) =>
+ pruningKey.fastEquals(leftKey) || hasDynamicPruningSubquery(plan, right, leftKey, rightKey)
+ case (_, Filter(DynamicPruningSubquery(pruningKey, _, _, _, _, _), plan)) =>
+ pruningKey.fastEquals(rightKey) ||
+ hasDynamicPruningSubquery(left, plan, leftKey, rightKey)
+ case _ => false
+ }
+ }
+
+ def hasBloomFilter(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+ rightKey: Expression): Boolean = {
+ findBloomFilterWithExp(left, leftKey) || findBloomFilterWithExp(right, rightKey)
+ }
+
+ private def findBloomFilterWithExp(plan: LogicalPlan, key: Expression): Boolean = {
+ plan.find {
+ case Filter(condition, _) =>
+ splitConjunctivePredicates(condition).exists {
+ case BloomFilterMightContain(_, XxHash64(Seq(valueExpression), _))
+ if valueExpression.fastEquals(key) => true
+ case _ => false
+ }
+ case _ => false
+ }.isDefined
+ }
+
+ def hasInSubquery(left: LogicalPlan, right: LogicalPlan, leftKey: Expression,
+ rightKey: Expression): Boolean = {
+ (left, right) match {
+ case (Filter(InSubquery(Seq(key),
+ ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _), _) =>
+ key.fastEquals(leftKey) || key.fastEquals(new Murmur3Hash(Seq(leftKey)))
+ case (_, Filter(InSubquery(Seq(key),
+ ListQuery(Aggregate(Seq(Alias(_, _)), Seq(Alias(_, _)), _), _, _, _, _)), _)) =>
+ key.fastEquals(rightKey) || key.fastEquals(new Murmur3Hash(Seq(rightKey)))
+ case _ => false
+ }
+ }
+
+ private def tryInjectRuntimeFilter(plan: LogicalPlan): LogicalPlan = {
+ var filterCounter = 0
+ val numFilterThreshold = conf.getConf(SQLConf.RUNTIME_FILTER_NUMBER_THRESHOLD)
+ plan transformUp {
+ case join @ ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, left, right, hint) =>
+ var newLeft = left
+ var newRight = right
+ (leftKeys, rightKeys).zipped.foreach((l, r) => {
+ // Check if:
+ // 1. There is already a DPP filter on the key
+ // 2. There is already a runtime filter (Bloom filter or IN subquery) on the key
+ // 3. The keys are simple cheap expressions
+ if (filterCounter < numFilterThreshold &&
+ !hasDynamicPruningSubquery(left, right, l, r) &&
+ !hasRuntimeFilter(newLeft, newRight, l, r) &&
+ isSimpleExpression(l) && isSimpleExpression(r)) {
+ if (canFilterLeft(joinType) && filteringHasBenefit(left, right, l, hint)) {
+ newLeft = injectFilter(l, newLeft, r, right)
+ filterCounter = filterCounter + 1
+ } else if (canFilterRight(joinType) && filteringHasBenefit(right, left, r, hint)) {
+ newRight = injectFilter(r, newRight, l, left)
+ filterCounter = filterCounter + 1
+ }
+ }
+ })
+ Join(newLeft, newRight, joinType, join.condition, hint)
+ }
+ }
+
+ override def apply(plan: LogicalPlan): LogicalPlan = plan match {
+ case s: Subquery if s.correlated => plan
+ case _ if !conf.runtimeFilterSemiJoinReductionEnabled &&
+ !conf.runtimeFilterBloomFilterEnabled => plan
+ case _ => tryInjectRuntimeFilter(plan)
+ }
+
+}
diff --git a/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala
new file mode 100644
index 000000000..c3a0a90d8
--- /dev/null
+++ b/shims/spark321/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala
@@ -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.spark.sql.catalyst.trees
+
+// Enums for commonly encountered tree patterns in rewrite rules.
+object TreePattern extends Enumeration {
+ type TreePattern = Value
+
+ // Enum Ids start from 0.
+ // Expression patterns (alphabetically ordered)
+ val AGGREGATE_EXPRESSION = Value(0)
+ val ALIAS: Value = Value
+ val AND_OR: Value = Value
+ val ARRAYS_ZIP: Value = Value
+ val ATTRIBUTE_REFERENCE: Value = Value
+ val APPEND_COLUMNS: Value = Value
+ val AVERAGE: Value = Value
+ val GROUPING_ANALYTICS: Value = Value
+ val BINARY_ARITHMETIC: Value = Value
+ val BINARY_COMPARISON: Value = Value
+ val BOOL_AGG: Value = Value
+ val CASE_WHEN: Value = Value
+ val CAST: Value = Value
+ val CONCAT: Value = Value
+ val COUNT: Value = Value
+ val COUNT_IF: Value = Value
+ val CREATE_NAMED_STRUCT: Value = Value
+ val CURRENT_LIKE: Value = Value
+ val DESERIALIZE_TO_OBJECT: Value = Value
+ val DYNAMIC_PRUNING_EXPRESSION: Value = Value
+ val DYNAMIC_PRUNING_SUBQUERY: Value = Value
+ val EXISTS_SUBQUERY = Value
+ val EXPRESSION_WITH_RANDOM_SEED: Value = Value
+ val EXTRACT_VALUE: Value = Value
+ val GENERATE: Value = Value
+ val GENERATOR: Value = Value
+ val HIGH_ORDER_FUNCTION: Value = Value
+ val IF: Value = Value
+ val IN: Value = Value
+ val IN_SUBQUERY: Value = Value
+ val INSET: Value = Value
+ val INTERSECT: Value = Value
+ val INVOKE: Value = Value
+ val JSON_TO_STRUCT: Value = Value
+ val LAMBDA_FUNCTION: Value = Value
+ val LAMBDA_VARIABLE: Value = Value
+ val LATERAL_SUBQUERY: Value = Value
+ val LIKE_FAMLIY: Value = Value
+ val LIST_SUBQUERY: Value = Value
+ val LITERAL: Value = Value
+ val MAP_OBJECTS: Value = Value
+ val MULTI_ALIAS: Value = Value
+ val NEW_INSTANCE: Value = Value
+ val NOT: Value = Value
+ val NULL_CHECK: Value = Value
+ val NULL_LITERAL: Value = Value
+ val SERIALIZE_FROM_OBJECT: Value = Value
+ val OUTER_REFERENCE: Value = Value
+ val PIVOT: Value = Value
+ val PLAN_EXPRESSION: Value = Value
+ val PYTHON_UDF: Value = Value
+ val REGEXP_EXTRACT_FAMILY: Value = Value
+ val REGEXP_REPLACE: Value = Value
+ val RUNTIME_REPLACEABLE: Value = Value
+ val SCALAR_SUBQUERY: Value = Value
+ val SCALA_UDF: Value = Value
+ val SORT: Value = Value
+ val SUBQUERY_ALIAS: Value = Value
+ val SUM: Value = Value
+ val TIME_WINDOW: Value = Value
+ val TIME_ZONE_AWARE_EXPRESSION: Value = Value
+ val TRUE_OR_FALSE_LITERAL: Value = Value
+ val WINDOW_EXPRESSION: Value = Value
+ val UNARY_POSITIVE: Value = Value
+ val UPDATE_FIELDS: Value = Value
+ val UPPER_OR_LOWER: Value = Value
+ val UP_CAST: Value = Value
+
+ // Logical plan patterns (alphabetically ordered)
+ val AGGREGATE: Value = Value
+ val COMMAND: Value = Value
+ val CTE: Value = Value
+ val DISTINCT_LIKE: Value = Value
+ val EVENT_TIME_WATERMARK: Value = Value
+ val EXCEPT: Value = Value
+ val FILTER: Value = Value
+ val INNER_LIKE_JOIN: Value = Value
+ val JOIN: Value = Value
+ val LATERAL_JOIN: Value = Value
+ val LEFT_SEMI_OR_ANTI_JOIN: Value = Value
+ val LIMIT: Value = Value
+ val LOCAL_RELATION: Value = Value
+ val LOGICAL_QUERY_STAGE: Value = Value
+ val NATURAL_LIKE_JOIN: Value = Value
+ val OUTER_JOIN: Value = Value
+ val PROJECT: Value = Value
+ val REPARTITION_OPERATION: Value = Value
+ val UNION: Value = Value
+ val UNRESOLVED_RELATION: Value = Value
+ val TYPED_FILTER: Value = Value
+ val WINDOW: Value = Value
+ val WITH_WINDOW_DEFINITION: Value = Value
+
+ // Unresolved expression patterns (Alphabetically ordered)
+ val UNRESOLVED_ALIAS: Value = Value
+ val UNRESOLVED_ATTRIBUTE: Value = Value
+ val UNRESOLVED_DESERIALIZER: Value = Value
+ val UNRESOLVED_ORDINAL: Value = Value
+ val UNRESOLVED_FUNCTION: Value = Value
+ val UNRESOLVED_HINT: Value = Value
+ val UNRESOLVED_WINDOW_EXPRESSION: Value = Value
+
+ // Unresolved Plan patterns (Alphabetically ordered)
+ val UNRESOLVED_SUBQUERY_COLUMN_ALIAS: Value = Value
+ val UNRESOLVED_FUNC: Value = Value
+
+ // Execution expression patterns (alphabetically ordered)
+ val IN_SUBQUERY_EXEC: Value = Value
+
+ // Execution Plan patterns (alphabetically ordered)
+ val EXCHANGE: Value = Value
+}
diff --git a/shims/spark321/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/shims/spark321/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala
new file mode 100644
index 000000000..da11df664
--- /dev/null
+++ b/shims/spark321/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala
@@ -0,0 +1,92 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.spark.sql.ExperimentalMethods
+import org.apache.spark.sql.catalyst.catalog.SessionCatalog
+import org.apache.spark.sql.catalyst.optimizer._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions
+import org.apache.spark.sql.execution.datasources.SchemaPruning
+import org.apache.spark.sql.execution.datasources.v2.{V2ScanRelationPushDown, V2Writes}
+import org.apache.spark.sql.execution.dynamicpruning.{CleanupDynamicPruningFilters, PartitionPruning}
+import org.apache.spark.sql.execution.python.{ExtractGroupingPythonUDFFromAggregate, ExtractPythonUDFFromAggregate, ExtractPythonUDFs}
+
+class SparkOptimizer(
+ catalogManager: CatalogManager,
+ catalog: SessionCatalog,
+ experimentalMethods: ExperimentalMethods)
+ extends Optimizer(catalogManager) {
+
+ override def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] =
+ // TODO: move SchemaPruning into catalyst
+ SchemaPruning :: V2ScanRelationPushDown :: V2Writes :: PruneFileSourcePartitions :: Nil
+
+ override def defaultBatches: Seq[Batch] = (preOptimizationBatches ++ super.defaultBatches :+
+ Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+
+ Batch("PartitionPruning", Once,
+ PartitionPruning) :+
+ Batch("InjectRuntimeFilter", FixedPoint(1),
+ InjectRuntimeFilter,
+ RewritePredicateSubquery) :+
+ Batch("Pushdown Filters from PartitionPruning", fixedPoint,
+ PushDownPredicates) :+
+ Batch("Cleanup filters that cannot be pushed down", Once,
+ CleanupDynamicPruningFilters,
+ PruneFilters)) ++
+ postHocOptimizationBatches :+
+ Batch("Extract Python UDFs", Once,
+ ExtractPythonUDFFromJoinCondition,
+ // `ExtractPythonUDFFromJoinCondition` can convert a join to a cartesian product.
+ // Here, we rerun cartesian product check.
+ CheckCartesianProducts,
+ ExtractPythonUDFFromAggregate,
+ // This must be executed after `ExtractPythonUDFFromAggregate` and before `ExtractPythonUDFs`.
+ ExtractGroupingPythonUDFFromAggregate,
+ ExtractPythonUDFs,
+ // The eval-python node may be between Project/Filter and the scan node, which breaks
+ // column pruning and filter push-down. Here we rerun the related optimizer rules.
+ ColumnPruning,
+ PushPredicateThroughNonJoin,
+ RemoveNoopOperators) :+
+ Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*)
+
+ override def nonExcludableRules: Seq[String] = super.nonExcludableRules :+
+ ExtractPythonUDFFromJoinCondition.ruleName :+
+ ExtractPythonUDFFromAggregate.ruleName :+ ExtractGroupingPythonUDFFromAggregate.ruleName :+
+ ExtractPythonUDFs.ruleName :+
+ V2ScanRelationPushDown.ruleName :+
+ V2Writes.ruleName
+
+ /**
+ * Optimization batches that are executed before the regular optimization batches (also before
+ * the finish analysis batch).
+ */
+ def preOptimizationBatches: Seq[Batch] = Nil
+
+ /**
+ * Optimization batches that are executed after the regular optimization batches, but before the
+ * batch executing the [[ExperimentalMethods]] optimizer rules. This hook can be used to add
+ * custom optimizer batches to the Spark optimizer.
+ *
+ * Note that 'Extract Python UDFs' batch is an exception and ran after the batches defined here.
+ */
+ def postHocOptimizationBatches: Seq[Batch] = Nil
+}
diff --git a/shims/spark321/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/shims/spark321/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
new file mode 100644
index 000000000..894df3688
--- /dev/null
+++ b/shims/spark321/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -0,0 +1,4356 @@
+/*
+ * 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.spark.sql.internal
+
+import java.util.{Locale, NoSuchElementException, Properties, TimeZone}
+import java.util
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicReference
+import java.util.zip.Deflater
+
+import scala.collection.JavaConverters._
+import scala.collection.immutable
+import scala.util.Try
+import scala.util.control.NonFatal
+import scala.util.matching.Regex
+
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.{SparkConf, SparkContext, TaskContext}
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.internal.config.{IGNORE_MISSING_FILES => SPARK_IGNORE_MISSING_FILES}
+import org.apache.spark.network.util.ByteUnit
+import org.apache.spark.sql.catalyst.ScalaReflection
+import org.apache.spark.sql.catalyst.analysis.{HintErrorLogger, Resolver}
+import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode
+import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator
+import org.apache.spark.sql.catalyst.plans.logical.HintErrorHandler
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME
+import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
+import org.apache.spark.sql.types.{AtomicType, TimestampNTZType, TimestampType}
+import org.apache.spark.unsafe.array.ByteArrayMethods
+import org.apache.spark.util.Utils
+
+////////////////////////////////////////////////////////////////////////////////////////////////////
+// This file defines the configuration options for Spark SQL.
+////////////////////////////////////////////////////////////////////////////////////////////////////
+
+
+object SQLConf {
+
+ private[this] val sqlConfEntriesUpdateLock = new Object
+
+ @volatile
+ private[this] var sqlConfEntries: util.Map[String, ConfigEntry[_]] = util.Collections.emptyMap()
+
+ private[this] val staticConfKeysUpdateLock = new Object
+
+ @volatile
+ private[this] var staticConfKeys: java.util.Set[String] = util.Collections.emptySet()
+
+ private def register(entry: ConfigEntry[_]): Unit = sqlConfEntriesUpdateLock.synchronized {
+ require(!sqlConfEntries.containsKey(entry.key),
+ s"Duplicate SQLConfigEntry. ${entry.key} has been registered")
+ val updatedMap = new java.util.HashMap[String, ConfigEntry[_]](sqlConfEntries)
+ updatedMap.put(entry.key, entry)
+ sqlConfEntries = updatedMap
+ }
+
+ // For testing only
+ private[sql] def unregister(entry: ConfigEntry[_]): Unit = sqlConfEntriesUpdateLock.synchronized {
+ val updatedMap = new java.util.HashMap[String, ConfigEntry[_]](sqlConfEntries)
+ updatedMap.remove(entry.key)
+ sqlConfEntries = updatedMap
+ }
+
+ private[internal] def getConfigEntry(key: String): ConfigEntry[_] = {
+ sqlConfEntries.get(key)
+ }
+
+ private[internal] def getConfigEntries(): util.Collection[ConfigEntry[_]] = {
+ sqlConfEntries.values()
+ }
+
+ private[internal] def containsConfigEntry(entry: ConfigEntry[_]): Boolean = {
+ getConfigEntry(entry.key) == entry
+ }
+
+ private[sql] def containsConfigKey(key: String): Boolean = {
+ sqlConfEntries.containsKey(key)
+ }
+
+ def registerStaticConfigKey(key: String): Unit = staticConfKeysUpdateLock.synchronized {
+ val updated = new util.HashSet[String](staticConfKeys)
+ updated.add(key)
+ staticConfKeys = updated
+ }
+
+ def isStaticConfigKey(key: String): Boolean = staticConfKeys.contains(key)
+
+ def buildConf(key: String): ConfigBuilder = ConfigBuilder(key).onCreate(register)
+
+ def buildStaticConf(key: String): ConfigBuilder = {
+ ConfigBuilder(key).onCreate { entry =>
+ SQLConf.registerStaticConfigKey(entry.key)
+ SQLConf.register(entry)
+ }
+ }
+
+ /**
+ * Merge all non-static configs to the SQLConf. For example, when the 1st [[SparkSession]] and
+ * the global [[SharedState]] have been initialized, all static configs have taken affect and
+ * should not be set to other values. Other later created sessions should respect all static
+ * configs and only be able to change non-static configs.
+ */
+ private[sql] def mergeNonStaticSQLConfigs(
+ sqlConf: SQLConf,
+ configs: Map[String, String]): Unit = {
+ for ((k, v) <- configs if !staticConfKeys.contains(k)) {
+ sqlConf.setConfString(k, v)
+ }
+ }
+
+ /**
+ * Extract entries from `SparkConf` and put them in the `SQLConf`
+ */
+ private[sql] def mergeSparkConf(sqlConf: SQLConf, sparkConf: SparkConf): Unit = {
+ sparkConf.getAll.foreach { case (k, v) =>
+ sqlConf.setConfString(k, v)
+ }
+ }
+
+ /**
+ * Default config. Only used when there is no active SparkSession for the thread.
+ * See [[get]] for more information.
+ */
+ private lazy val fallbackConf = new ThreadLocal[SQLConf] {
+ override def initialValue: SQLConf = new SQLConf
+ }
+
+ /** See [[get]] for more information. */
+ def getFallbackConf: SQLConf = fallbackConf.get()
+
+ private lazy val existingConf = new ThreadLocal[SQLConf] {
+ override def initialValue: SQLConf = null
+ }
+
+ def withExistingConf[T](conf: SQLConf)(f: => T): T = {
+ val old = existingConf.get()
+ existingConf.set(conf)
+ try {
+ f
+ } finally {
+ if (old != null) {
+ existingConf.set(old)
+ } else {
+ existingConf.remove()
+ }
+ }
+ }
+
+ /**
+ * Defines a getter that returns the SQLConf within scope.
+ * See [[get]] for more information.
+ */
+ private val confGetter = new AtomicReference[() => SQLConf](() => fallbackConf.get())
+
+ /**
+ * Sets the active config object within the current scope.
+ * See [[get]] for more information.
+ */
+ def setSQLConfGetter(getter: () => SQLConf): Unit = {
+ confGetter.set(getter)
+ }
+
+ /**
+ * Returns the active config object within the current scope. If there is an active SparkSession,
+ * the proper SQLConf associated with the thread's active session is used. If it's called from
+ * tasks in the executor side, a SQLConf will be created from job local properties, which are set
+ * and propagated from the driver side, unless a `SQLConf` has been set in the scope by
+ * `withExistingConf` as done for propagating SQLConf for operations performed on RDDs created
+ * from DataFrames.
+ *
+ * The way this works is a little bit convoluted, due to the fact that config was added initially
+ * only for physical plans (and as a result not in sql/catalyst module).
+ *
+ * The first time a SparkSession is instantiated, we set the [[confGetter]] to return the
+ * active SparkSession's config. If there is no active SparkSession, it returns using the thread
+ * local [[fallbackConf]]. The reason [[fallbackConf]] is a thread local (rather than just a conf)
+ * is to support setting different config options for different threads so we can potentially
+ * run tests in parallel. At the time this feature was implemented, this was a no-op since we
+ * run unit tests (that does not involve SparkSession) in serial order.
+ */
+ def get: SQLConf = {
+ if (TaskContext.get != null) {
+ val conf = existingConf.get()
+ if (conf != null) {
+ conf
+ } else {
+ new ReadOnlySQLConf(TaskContext.get())
+ }
+ } else {
+ val isSchedulerEventLoopThread = SparkContext.getActive
+ .flatMap { sc => Option(sc.dagScheduler) }
+ .map(_.eventProcessLoop.eventThread)
+ .exists(_.getId == Thread.currentThread().getId)
+ if (isSchedulerEventLoopThread) {
+ // DAGScheduler event loop thread does not have an active SparkSession, the `confGetter`
+ // will return `fallbackConf` which is unexpected. Here we require the caller to get the
+ // conf within `withExistingConf`, otherwise fail the query.
+ val conf = existingConf.get()
+ if (conf != null) {
+ conf
+ } else if (Utils.isTesting) {
+ throw QueryExecutionErrors.cannotGetSQLConfInSchedulerEventLoopThreadError()
+ } else {
+ confGetter.get()()
+ }
+ } else {
+ val conf = existingConf.get()
+ if (conf != null) {
+ conf
+ } else {
+ confGetter.get()()
+ }
+ }
+ }
+ }
+
+ val ANALYZER_MAX_ITERATIONS = buildConf("spark.sql.analyzer.maxIterations")
+ .internal()
+ .doc("The max number of iterations the analyzer runs.")
+ .version("3.0.0")
+ .intConf
+ .createWithDefault(100)
+
+ val OPTIMIZER_EXCLUDED_RULES = buildConf("spark.sql.optimizer.excludedRules")
+ .doc("Configures a list of rules to be disabled in the optimizer, in which the rules are " +
+ "specified by their rule names and separated by comma. It is not guaranteed that all the " +
+ "rules in this configuration will eventually be excluded, as some rules are necessary " +
+ "for correctness. The optimizer will log the rules that have indeed been excluded.")
+ .version("2.4.0")
+ .stringConf
+ .createOptional
+
+ val OPTIMIZER_MAX_ITERATIONS = buildConf("spark.sql.optimizer.maxIterations")
+ .internal()
+ .doc("The max number of iterations the optimizer runs.")
+ .version("2.0.0")
+ .intConf
+ .createWithDefault(100)
+
+ val OPTIMIZER_INSET_CONVERSION_THRESHOLD =
+ buildConf("spark.sql.optimizer.inSetConversionThreshold")
+ .internal()
+ .doc("The threshold of set size for InSet conversion.")
+ .version("2.0.0")
+ .intConf
+ .createWithDefault(10)
+
+ val OPTIMIZER_INSET_SWITCH_THRESHOLD =
+ buildConf("spark.sql.optimizer.inSetSwitchThreshold")
+ .internal()
+ .doc("Configures the max set size in InSet for which Spark will generate code with " +
+ "switch statements. This is applicable only to bytes, shorts, ints, dates.")
+ .version("3.0.0")
+ .intConf
+ .checkValue(threshold => threshold >= 0 && threshold <= 600, "The max set size " +
+ "for using switch statements in InSet must be non-negative and less than or equal to 600")
+ .createWithDefault(400)
+
+ val PLAN_CHANGE_LOG_LEVEL = buildConf("spark.sql.planChangeLog.level")
+ .internal()
+ .doc("Configures the log level for logging the change from the original plan to the new " +
+ "plan after a rule or batch is applied. The value can be 'trace', 'debug', 'info', " +
+ "'warn', or 'error'. The default log level is 'trace'.")
+ .version("3.1.0")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValue(logLevel => Set("TRACE", "DEBUG", "INFO", "WARN", "ERROR").contains(logLevel),
+ "Invalid value for 'spark.sql.planChangeLog.level'. Valid values are " +
+ "'trace', 'debug', 'info', 'warn' and 'error'.")
+ .createWithDefault("trace")
+
+ val PLAN_CHANGE_LOG_RULES = buildConf("spark.sql.planChangeLog.rules")
+ .internal()
+ .doc("Configures a list of rules for logging plan changes, in which the rules are " +
+ "specified by their rule names and separated by comma.")
+ .version("3.1.0")
+ .stringConf
+ .createOptional
+
+ val PLAN_CHANGE_LOG_BATCHES = buildConf("spark.sql.planChangeLog.batches")
+ .internal()
+ .doc("Configures a list of batches for logging plan changes, in which the batches " +
+ "are specified by their batch names and separated by comma.")
+ .version("3.1.0")
+ .stringConf
+ .createOptional
+
+ val DYNAMIC_PARTITION_PRUNING_ENABLED =
+ buildConf("spark.sql.optimizer.dynamicPartitionPruning.enabled")
+ .doc("When true, we will generate predicate for partition column when it's used as join key")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val DYNAMIC_PARTITION_PRUNING_USE_STATS =
+ buildConf("spark.sql.optimizer.dynamicPartitionPruning.useStats")
+ .internal()
+ .doc("When true, distinct count statistics will be used for computing the data size of the " +
+ "partitioned table after dynamic partition pruning, in order to evaluate if it is worth " +
+ "adding an extra subquery as the pruning filter if broadcast reuse is not applicable.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO =
+ buildConf("spark.sql.optimizer.dynamicPartitionPruning.fallbackFilterRatio")
+ .internal()
+ .doc("When statistics are not available or configured not to be used, this config will be " +
+ "used as the fallback filter ratio for computing the data size of the partitioned table " +
+ "after dynamic partition pruning, in order to evaluate if it is worth adding an extra " +
+ "subquery as the pruning filter if broadcast reuse is not applicable.")
+ .version("3.0.0")
+ .doubleConf
+ .createWithDefault(0.5)
+
+ val DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY =
+ buildConf("spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly")
+ .internal()
+ .doc("When true, dynamic partition pruning will only apply when the broadcast exchange of " +
+ "a broadcast hash join operation can be reused as the dynamic pruning filter.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val RUNTIME_FILTER_SEMI_JOIN_REDUCTION_ENABLED =
+ buildConf("spark.sql.optimizer.runtimeFilter.semiJoinReduction.enabled")
+ .doc("When true and if one side of a shuffle join has a selective predicate, we attempt " +
+ "to insert a semi join in the other side to reduce the amount of shuffle data")
+ .version("3.3.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val RUNTIME_FILTER_NUMBER_THRESHOLD =
+ buildConf("spark.sql.optimizer.runtimeFilter.number.threshold")
+ .doc("The total number of injected runtime filters (non-DPP) for a single " +
+ "query. This is to prevent driver OOMs with too many Bloom filters")
+ .version("3.3.0")
+ .intConf
+ .checkValue(threshold => threshold >= 0, "The threshold should be >= 0")
+ .createWithDefault(10)
+
+ lazy val RUNTIME_BLOOM_FILTER_ENABLED =
+ buildConf("spark.sql.optimizer.runtime.bloomFilter.enabled")
+ .doc("When true and if one side of a shuffle join has a selective predicate, we attempt " +
+ "to insert a bloom filter in the other side to reduce the amount of shuffle data")
+ .version("3.3.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val RUNTIME_BLOOM_FILTER_THRESHOLD =
+ buildConf("spark.sql.optimizer.runtime.bloomFilter.threshold")
+ .doc("Size threshold of the bloom filter creation side plan. Estimated size needs to be " +
+ "under this value to try to inject bloom filter")
+ .version("3.3.0")
+ .bytesConf(ByteUnit.BYTE)
+ .createWithDefaultString("10MB")
+
+ val RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD =
+ buildConf("spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizethreshold")
+ .doc("Byte size threshold of the Bloom filter application side plan's aggregated scan " +
+ "size. Aggregated scan byte size of the Bloom filter application side needs to be over " +
+ "this value to inject a bloom filter")
+ .version("3.3.0")
+ .bytesConf(ByteUnit.BYTE)
+ .createWithDefaultString("10GB")
+
+ val COMPRESS_CACHED = buildConf("spark.sql.inMemoryColumnarStorage.compressed")
+ .doc("When set to true Spark SQL will automatically select a compression codec for each " +
+ "column based on statistics of the data.")
+ .version("1.0.1")
+ .booleanConf
+ .createWithDefault(true)
+
+ val COLUMN_BATCH_SIZE = buildConf("spark.sql.inMemoryColumnarStorage.batchSize")
+ .doc("Controls the size of batches for columnar caching. Larger batch sizes can improve " +
+ "memory utilization and compression, but risk OOMs when caching data.")
+ .version("1.1.1")
+ .intConf
+ .createWithDefault(10000)
+
+ val IN_MEMORY_PARTITION_PRUNING =
+ buildConf("spark.sql.inMemoryColumnarStorage.partitionPruning")
+ .internal()
+ .doc("When true, enable partition pruning for in-memory columnar tables.")
+ .version("1.2.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val IN_MEMORY_TABLE_SCAN_STATISTICS_ENABLED =
+ buildConf("spark.sql.inMemoryTableScanStatistics.enable")
+ .internal()
+ .doc("When true, enable in-memory table scan accumulators.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val CACHE_VECTORIZED_READER_ENABLED =
+ buildConf("spark.sql.inMemoryColumnarStorage.enableVectorizedReader")
+ .doc("Enables vectorized reader for columnar caching.")
+ .version("2.3.1")
+ .booleanConf
+ .createWithDefault(true)
+
+ val COLUMN_VECTOR_OFFHEAP_ENABLED =
+ buildConf("spark.sql.columnVector.offheap.enabled")
+ .internal()
+ .doc("When true, use OffHeapColumnVector in ColumnarBatch.")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val PREFER_SORTMERGEJOIN = buildConf("spark.sql.join.preferSortMergeJoin")
+ .internal()
+ .doc("When true, prefer sort merge join over shuffled hash join. " +
+ "Sort merge join consumes less memory than shuffled hash join and it works efficiently " +
+ "when both join tables are large. On the other hand, shuffled hash join can improve " +
+ "performance (e.g., of full outer joins) when one of join tables is much smaller.")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val RADIX_SORT_ENABLED = buildConf("spark.sql.sort.enableRadixSort")
+ .internal()
+ .doc("When true, enable use of radix sort when possible. Radix sort is much faster but " +
+ "requires additional memory to be reserved up-front. The memory overhead may be " +
+ "significant when sorting very small rows (up to 50% more in this case).")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val AUTO_BROADCASTJOIN_THRESHOLD = buildConf("spark.sql.autoBroadcastJoinThreshold")
+ .doc("Configures the maximum size in bytes for a table that will be broadcast to all worker " +
+ "nodes when performing a join. By setting this value to -1 broadcasting can be disabled. " +
+ "Note that currently statistics are only supported for Hive Metastore tables where the " +
+ "command `ANALYZE TABLE COMPUTE STATISTICS noscan` has been " +
+ "run, and file-based data source tables where the statistics are computed directly on " +
+ "the files of data.")
+ .version("1.1.0")
+ .bytesConf(ByteUnit.BYTE)
+ .createWithDefaultString("10MB")
+
+ val LIMIT_SCALE_UP_FACTOR = buildConf("spark.sql.limit.scaleUpFactor")
+ .internal()
+ .doc("Minimal increase rate in number of partitions between attempts when executing a take " +
+ "on a query. Higher values lead to more partitions read. Lower values might lead to " +
+ "longer execution times as more jobs will be run")
+ .version("2.1.1")
+ .intConf
+ .createWithDefault(4)
+
+ val ADVANCED_PARTITION_PREDICATE_PUSHDOWN =
+ buildConf("spark.sql.hive.advancedPartitionPredicatePushdown.enabled")
+ .internal()
+ .doc("When true, advanced partition predicate pushdown into Hive metastore is enabled.")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val LEAF_NODE_DEFAULT_PARALLELISM = buildConf("spark.sql.leafNodeDefaultParallelism")
+ .doc("The default parallelism of Spark SQL leaf nodes that produce data, such as the file " +
+ "scan node, the local data scan node, the range node, etc. The default value of this " +
+ "config is 'SparkContext#defaultParallelism'.")
+ .version("3.2.0")
+ .intConf
+ .checkValue(_ > 0, "The value of spark.sql.leafNodeDefaultParallelism must be positive.")
+ .createOptional
+
+ val SHUFFLE_PARTITIONS = buildConf("spark.sql.shuffle.partitions")
+ .doc("The default number of partitions to use when shuffling data for joins or aggregations. " +
+ "Note: For structured streaming, this configuration cannot be changed between query " +
+ "restarts from the same checkpoint location.")
+ .version("1.1.0")
+ .intConf
+ .checkValue(_ > 0, "The value of spark.sql.shuffle.partitions must be positive")
+ .createWithDefault(200)
+
+ val SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE =
+ buildConf("spark.sql.adaptive.shuffle.targetPostShuffleInputSize")
+ .internal()
+ .doc("(Deprecated since Spark 3.0)")
+ .version("1.6.0")
+ .bytesConf(ByteUnit.BYTE)
+ .checkValue(_ > 0, "advisoryPartitionSizeInBytes must be positive")
+ .createWithDefaultString("64MB")
+
+ val ADAPTIVE_EXECUTION_ENABLED = buildConf("spark.sql.adaptive.enabled")
+ .doc("When true, enable adaptive query execution, which re-optimizes the query plan in the " +
+ "middle of query execution, based on accurate runtime statistics.")
+ .version("1.6.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val ADAPTIVE_EXECUTION_FORCE_APPLY = buildConf("spark.sql.adaptive.forceApply")
+ .internal()
+ .doc("Adaptive query execution is skipped when the query does not have exchanges or " +
+ "sub-queries. By setting this config to true (together with " +
+ s"'${ADAPTIVE_EXECUTION_ENABLED.key}' set to true), Spark will force apply adaptive query " +
+ "execution for all supported queries.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val ADAPTIVE_EXECUTION_LOG_LEVEL = buildConf("spark.sql.adaptive.logLevel")
+ .internal()
+ .doc("Configures the log level for adaptive execution logging of plan changes. The value " +
+ "can be 'trace', 'debug', 'info', 'warn', or 'error'. The default log level is 'debug'.")
+ .version("3.0.0")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(Set("TRACE", "DEBUG", "INFO", "WARN", "ERROR"))
+ .createWithDefault("debug")
+
+ val ADVISORY_PARTITION_SIZE_IN_BYTES =
+ buildConf("spark.sql.adaptive.advisoryPartitionSizeInBytes")
+ .doc("The advisory size in bytes of the shuffle partition during adaptive optimization " +
+ s"(when ${ADAPTIVE_EXECUTION_ENABLED.key} is true). It takes effect when Spark " +
+ "coalesces small shuffle partitions or splits skewed shuffle partition.")
+ .version("3.0.0")
+ .fallbackConf(SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE)
+
+ val COALESCE_PARTITIONS_ENABLED =
+ buildConf("spark.sql.adaptive.coalescePartitions.enabled")
+ .doc(s"When true and '${ADAPTIVE_EXECUTION_ENABLED.key}' is true, Spark will coalesce " +
+ "contiguous shuffle partitions according to the target size (specified by " +
+ s"'${ADVISORY_PARTITION_SIZE_IN_BYTES.key}'), to avoid too many small tasks.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val COALESCE_PARTITIONS_PARALLELISM_FIRST =
+ buildConf("spark.sql.adaptive.coalescePartitions.parallelismFirst")
+ .doc("When true, Spark does not respect the target size specified by " +
+ s"'${ADVISORY_PARTITION_SIZE_IN_BYTES.key}' (default 64MB) when coalescing contiguous " +
+ "shuffle partitions, but adaptively calculate the target size according to the default " +
+ "parallelism of the Spark cluster. The calculated size is usually smaller than the " +
+ "configured target size. This is to maximize the parallelism and avoid performance " +
+ "regression when enabling adaptive query execution. It's recommended to set this config " +
+ "to false and respect the configured target size.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val COALESCE_PARTITIONS_MIN_PARTITION_SIZE =
+ buildConf("spark.sql.adaptive.coalescePartitions.minPartitionSize")
+ .doc("The minimum size of shuffle partitions after coalescing. This is useful when the " +
+ "adaptively calculated target size is too small during partition coalescing.")
+ .version("3.2.0")
+ .bytesConf(ByteUnit.BYTE)
+ .checkValue(_ > 0, "minPartitionSize must be positive")
+ .createWithDefaultString("1MB")
+
+ val COALESCE_PARTITIONS_MIN_PARTITION_NUM =
+ buildConf("spark.sql.adaptive.coalescePartitions.minPartitionNum")
+ .internal()
+ .doc("(deprecated) The suggested (not guaranteed) minimum number of shuffle partitions " +
+ "after coalescing. If not set, the default value is the default parallelism of the " +
+ "Spark cluster. This configuration only has an effect when " +
+ s"'${ADAPTIVE_EXECUTION_ENABLED.key}' and " +
+ s"'${COALESCE_PARTITIONS_ENABLED.key}' are both true.")
+ .version("3.0.0")
+ .intConf
+ .checkValue(_ > 0, "The minimum number of partitions must be positive.")
+ .createOptional
+
+ val COALESCE_PARTITIONS_INITIAL_PARTITION_NUM =
+ buildConf("spark.sql.adaptive.coalescePartitions.initialPartitionNum")
+ .doc("The initial number of shuffle partitions before coalescing. If not set, it equals to " +
+ s"${SHUFFLE_PARTITIONS.key}. This configuration only has an effect when " +
+ s"'${ADAPTIVE_EXECUTION_ENABLED.key}' and '${COALESCE_PARTITIONS_ENABLED.key}' " +
+ "are both true.")
+ .version("3.0.0")
+ .intConf
+ .checkValue(_ > 0, "The initial number of partitions must be positive.")
+ .createOptional
+
+ val FETCH_SHUFFLE_BLOCKS_IN_BATCH =
+ buildConf("spark.sql.adaptive.fetchShuffleBlocksInBatch")
+ .internal()
+ .doc("Whether to fetch the contiguous shuffle blocks in batch. Instead of fetching blocks " +
+ "one by one, fetching contiguous shuffle blocks for the same map task in batch can " +
+ "reduce IO and improve performance. Note, multiple contiguous blocks exist in single " +
+ s"fetch request only happen when '${ADAPTIVE_EXECUTION_ENABLED.key}' and " +
+ s"'${COALESCE_PARTITIONS_ENABLED.key}' are both true. This feature also depends " +
+ "on a relocatable serializer, the concatenation support codec in use, the new version " +
+ "shuffle fetch protocol and io encryption is disabled.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val LOCAL_SHUFFLE_READER_ENABLED =
+ buildConf("spark.sql.adaptive.localShuffleReader.enabled")
+ .doc(s"When true and '${ADAPTIVE_EXECUTION_ENABLED.key}' is true, Spark tries to use local " +
+ "shuffle reader to read the shuffle data when the shuffle partitioning is not needed, " +
+ "for example, after converting sort-merge join to broadcast-hash join.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val SKEW_JOIN_ENABLED =
+ buildConf("spark.sql.adaptive.skewJoin.enabled")
+ .doc(s"When true and '${ADAPTIVE_EXECUTION_ENABLED.key}' is true, Spark dynamically " +
+ "handles skew in shuffled join (sort-merge and shuffled hash) by splitting (and " +
+ "replicating if needed) skewed partitions.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val SKEW_JOIN_SKEWED_PARTITION_FACTOR =
+ buildConf("spark.sql.adaptive.skewJoin.skewedPartitionFactor")
+ .doc("A partition is considered as skewed if its size is larger than this factor " +
+ "multiplying the median partition size and also larger than " +
+ "'spark.sql.adaptive.skewJoin.skewedPartitionThresholdInBytes'")
+ .version("3.0.0")
+ .intConf
+ .checkValue(_ >= 0, "The skew factor cannot be negative.")
+ .createWithDefault(5)
+
+ val SKEW_JOIN_SKEWED_PARTITION_THRESHOLD =
+ buildConf("spark.sql.adaptive.skewJoin.skewedPartitionThresholdInBytes")
+ .doc("A partition is considered as skewed if its size in bytes is larger than this " +
+ s"threshold and also larger than '${SKEW_JOIN_SKEWED_PARTITION_FACTOR.key}' " +
+ "multiplying the median partition size. Ideally this config should be set larger " +
+ s"than '${ADVISORY_PARTITION_SIZE_IN_BYTES.key}'.")
+ .version("3.0.0")
+ .bytesConf(ByteUnit.BYTE)
+ .createWithDefaultString("256MB")
+
+ val NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN =
+ buildConf("spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin")
+ .internal()
+ .doc("The relation with a non-empty partition ratio lower than this config will not be " +
+ "considered as the build side of a broadcast-hash join in adaptive execution regardless " +
+ "of its size.This configuration only has an effect when " +
+ s"'${ADAPTIVE_EXECUTION_ENABLED.key}' is true.")
+ .version("3.0.0")
+ .doubleConf
+ .checkValue(_ >= 0, "The non-empty partition ratio must be positive number.")
+ .createWithDefault(0.2)
+
+ val ADAPTIVE_OPTIMIZER_EXCLUDED_RULES =
+ buildConf("spark.sql.adaptive.optimizer.excludedRules")
+ .doc("Configures a list of rules to be disabled in the adaptive optimizer, in which the " +
+ "rules are specified by their rule names and separated by comma. The optimizer will log " +
+ "the rules that have indeed been excluded.")
+ .version("3.1.0")
+ .stringConf
+ .createOptional
+
+ val ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD =
+ buildConf("spark.sql.adaptive.autoBroadcastJoinThreshold")
+ .doc("Configures the maximum size in bytes for a table that will be broadcast to all " +
+ "worker nodes when performing a join. By setting this value to -1 broadcasting can be " +
+ s"disabled. The default value is same with ${AUTO_BROADCASTJOIN_THRESHOLD.key}. " +
+ "Note that, this config is used only in adaptive framework.")
+ .version("3.2.0")
+ .bytesConf(ByteUnit.BYTE)
+ .createOptional
+
+ val ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD =
+ buildConf("spark.sql.adaptive.maxShuffledHashJoinLocalMapThreshold")
+ .doc("Configures the maximum size in bytes per partition that can be allowed to build " +
+ "local hash map. If this value is not smaller than " +
+ s"${ADVISORY_PARTITION_SIZE_IN_BYTES.key} and all the partition size are not larger " +
+ "than this config, join selection prefer to use shuffled hash join instead of " +
+ s"sort merge join regardless of the value of ${PREFER_SORTMERGEJOIN.key}.")
+ .version("3.2.0")
+ .bytesConf(ByteUnit.BYTE)
+ .createWithDefault(0L)
+
+ val ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED =
+ buildConf("spark.sql.adaptive.optimizeSkewsInRebalancePartitions.enabled")
+ .doc(s"When true and '${ADAPTIVE_EXECUTION_ENABLED.key}' is true, Spark will optimize the " +
+ "skewed shuffle partitions in RebalancePartitions and split them to smaller ones " +
+ s"according to the target size (specified by '${ADVISORY_PARTITION_SIZE_IN_BYTES.key}'), " +
+ "to avoid data skew.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS =
+ buildConf("spark.sql.adaptive.customCostEvaluatorClass")
+ .doc("The custom cost evaluator class to be used for adaptive execution. If not being set," +
+ " Spark will use its own SimpleCostEvaluator by default.")
+ .version("3.2.0")
+ .stringConf
+ .createOptional
+
+ val SUBEXPRESSION_ELIMINATION_ENABLED =
+ buildConf("spark.sql.subexpressionElimination.enabled")
+ .internal()
+ .doc("When true, common subexpressions will be eliminated.")
+ .version("1.6.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val SUBEXPRESSION_ELIMINATION_CACHE_MAX_ENTRIES =
+ buildConf("spark.sql.subexpressionElimination.cache.maxEntries")
+ .internal()
+ .doc("The maximum entries of the cache used for interpreted subexpression elimination.")
+ .version("3.1.0")
+ .intConf
+ .checkValue(_ >= 0, "The maximum must not be negative")
+ .createWithDefault(100)
+
+ val CASE_SENSITIVE = buildConf("spark.sql.caseSensitive")
+ .internal()
+ .doc("Whether the query analyzer should be case sensitive or not. " +
+ "Default to case insensitive. It is highly discouraged to turn on case sensitive mode.")
+ .version("1.4.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val CONSTRAINT_PROPAGATION_ENABLED = buildConf("spark.sql.constraintPropagation.enabled")
+ .internal()
+ .doc("When true, the query optimizer will infer and propagate data constraints in the query " +
+ "plan to optimize them. Constraint propagation can sometimes be computationally expensive " +
+ "for certain kinds of query plans (such as those with a large number of predicates and " +
+ "aliases) which might negatively impact overall runtime.")
+ .version("2.2.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val ESCAPED_STRING_LITERALS = buildConf("spark.sql.parser.escapedStringLiterals")
+ .internal()
+ .doc("When true, string literals (including regex patterns) remain escaped in our SQL " +
+ "parser. The default is false since Spark 2.0. Setting it to true can restore the behavior " +
+ "prior to Spark 2.0.")
+ .version("2.2.1")
+ .booleanConf
+ .createWithDefault(false)
+
+ val FILE_COMPRESSION_FACTOR = buildConf("spark.sql.sources.fileCompressionFactor")
+ .internal()
+ .doc("When estimating the output data size of a table scan, multiply the file size with this " +
+ "factor as the estimated data size, in case the data is compressed in the file and lead to" +
+ " a heavily underestimated result.")
+ .version("2.3.1")
+ .doubleConf
+ .checkValue(_ > 0, "the value of fileCompressionFactor must be greater than 0")
+ .createWithDefault(1.0)
+
+ val PARQUET_SCHEMA_MERGING_ENABLED = buildConf("spark.sql.parquet.mergeSchema")
+ .doc("When true, the Parquet data source merges schemas collected from all data files, " +
+ "otherwise the schema is picked from the summary file or a random data file " +
+ "if no summary file is available.")
+ .version("1.5.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val PARQUET_SCHEMA_RESPECT_SUMMARIES = buildConf("spark.sql.parquet.respectSummaryFiles")
+ .doc("When true, we make assumption that all part-files of Parquet are consistent with " +
+ "summary files and we will ignore them when merging schema. Otherwise, if this is " +
+ "false, which is the default, we will merge all part-files. This should be considered " +
+ "as expert-only option, and shouldn't be enabled before knowing what it means exactly.")
+ .version("1.5.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val PARQUET_BINARY_AS_STRING = buildConf("spark.sql.parquet.binaryAsString")
+ .doc("Some other Parquet-producing systems, in particular Impala and older versions of " +
+ "Spark SQL, do not differentiate between binary data and strings when writing out the " +
+ "Parquet schema. This flag tells Spark SQL to interpret binary data as a string to provide " +
+ "compatibility with these systems.")
+ .version("1.1.1")
+ .booleanConf
+ .createWithDefault(false)
+
+ val PARQUET_INT96_AS_TIMESTAMP = buildConf("spark.sql.parquet.int96AsTimestamp")
+ .doc("Some Parquet-producing systems, in particular Impala, store Timestamp into INT96. " +
+ "Spark would also store Timestamp as INT96 because we need to avoid precision lost of the " +
+ "nanoseconds field. This flag tells Spark SQL to interpret INT96 data as a timestamp to " +
+ "provide compatibility with these systems.")
+ .version("1.3.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val PARQUET_INT96_TIMESTAMP_CONVERSION = buildConf("spark.sql.parquet.int96TimestampConversion")
+ .doc("This controls whether timestamp adjustments should be applied to INT96 data when " +
+ "converting to timestamps, for data written by Impala. This is necessary because Impala " +
+ "stores INT96 data with a different timezone offset than Hive & Spark.")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ object ParquetOutputTimestampType extends Enumeration {
+ val INT96, TIMESTAMP_MICROS, TIMESTAMP_MILLIS = Value
+ }
+
+ val PARQUET_OUTPUT_TIMESTAMP_TYPE = buildConf("spark.sql.parquet.outputTimestampType")
+ .doc("Sets which Parquet timestamp type to use when Spark writes data to Parquet files. " +
+ "INT96 is a non-standard but commonly used timestamp type in Parquet. TIMESTAMP_MICROS " +
+ "is a standard timestamp type in Parquet, which stores number of microseconds from the " +
+ "Unix epoch. TIMESTAMP_MILLIS is also standard, but with millisecond precision, which " +
+ "means Spark has to truncate the microsecond portion of its timestamp value.")
+ .version("2.3.0")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(ParquetOutputTimestampType.values.map(_.toString))
+ .createWithDefault(ParquetOutputTimestampType.INT96.toString)
+
+ val PARQUET_COMPRESSION = buildConf("spark.sql.parquet.compression.codec")
+ .doc("Sets the compression codec used when writing Parquet files. If either `compression` or " +
+ "`parquet.compression` is specified in the table-specific options/properties, the " +
+ "precedence would be `compression`, `parquet.compression`, " +
+ "`spark.sql.parquet.compression.codec`. Acceptable values include: none, uncompressed, " +
+ "snappy, gzip, lzo, brotli, lz4, zstd.")
+ .version("1.1.1")
+ .stringConf
+ .transform(_.toLowerCase(Locale.ROOT))
+ .checkValues(Set("none", "uncompressed", "snappy", "gzip", "lzo", "lz4", "brotli", "zstd"))
+ .createWithDefault("snappy")
+
+ val PARQUET_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.parquet.filterPushdown")
+ .doc("Enables Parquet filter push-down optimization when set to true.")
+ .version("1.2.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val PARQUET_FILTER_PUSHDOWN_DATE_ENABLED = buildConf("spark.sql.parquet.filterPushdown.date")
+ .doc("If true, enables Parquet filter push-down optimization for Date. " +
+ s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " +
+ "enabled.")
+ .version("2.4.0")
+ .internal()
+ .booleanConf
+ .createWithDefault(true)
+
+ val PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED =
+ buildConf("spark.sql.parquet.filterPushdown.timestamp")
+ .doc("If true, enables Parquet filter push-down optimization for Timestamp. " +
+ s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " +
+ "enabled and Timestamp stored as TIMESTAMP_MICROS or TIMESTAMP_MILLIS type.")
+ .version("2.4.0")
+ .internal()
+ .booleanConf
+ .createWithDefault(true)
+
+ val PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED =
+ buildConf("spark.sql.parquet.filterPushdown.decimal")
+ .doc("If true, enables Parquet filter push-down optimization for Decimal. " +
+ s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " +
+ "enabled.")
+ .version("2.4.0")
+ .internal()
+ .booleanConf
+ .createWithDefault(true)
+
+ val PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED =
+ buildConf("spark.sql.parquet.filterPushdown.string.startsWith")
+ .doc("If true, enables Parquet filter push-down optimization for string startsWith function. " +
+ s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " +
+ "enabled.")
+ .version("2.4.0")
+ .internal()
+ .booleanConf
+ .createWithDefault(true)
+
+ val PARQUET_FILTER_PUSHDOWN_INFILTERTHRESHOLD =
+ buildConf("spark.sql.parquet.pushdown.inFilterThreshold")
+ .doc("For IN predicate, Parquet filter will push-down a set of OR clauses if its " +
+ "number of values not exceeds this threshold. Otherwise, Parquet filter will push-down " +
+ "a value greater than or equal to its minimum value and less than or equal to " +
+ "its maximum value. By setting this value to 0 this feature can be disabled. " +
+ s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " +
+ "enabled.")
+ .version("2.4.0")
+ .internal()
+ .intConf
+ .checkValue(threshold => threshold >= 0, "The threshold must not be negative.")
+ .createWithDefault(10)
+
+ val PARQUET_WRITE_LEGACY_FORMAT = buildConf("spark.sql.parquet.writeLegacyFormat")
+ .doc("If true, data will be written in a way of Spark 1.4 and earlier. For example, decimal " +
+ "values will be written in Apache Parquet's fixed-length byte array format, which other " +
+ "systems such as Apache Hive and Apache Impala use. If false, the newer format in Parquet " +
+ "will be used. For example, decimals will be written in int-based format. If Parquet " +
+ "output is intended for use with systems that do not support this newer format, set to true.")
+ .version("1.6.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val PARQUET_OUTPUT_COMMITTER_CLASS = buildConf("spark.sql.parquet.output.committer.class")
+ .doc("The output committer class used by Parquet. The specified class needs to be a " +
+ "subclass of org.apache.hadoop.mapreduce.OutputCommitter. Typically, it's also a subclass " +
+ "of org.apache.parquet.hadoop.ParquetOutputCommitter. If it is not, then metadata " +
+ "summaries will never be created, irrespective of the value of " +
+ "parquet.summary.metadata.level")
+ .version("1.5.0")
+ .internal()
+ .stringConf
+ .createWithDefault("org.apache.parquet.hadoop.ParquetOutputCommitter")
+
+ val PARQUET_VECTORIZED_READER_ENABLED =
+ buildConf("spark.sql.parquet.enableVectorizedReader")
+ .doc("Enables vectorized parquet decoding.")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val PARQUET_RECORD_FILTER_ENABLED = buildConf("spark.sql.parquet.recordLevelFilter.enabled")
+ .doc("If true, enables Parquet's native record-level filtering using the pushed down " +
+ "filters. " +
+ s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' " +
+ "is enabled and the vectorized reader is not used. You can ensure the vectorized reader " +
+ s"is not used by setting '${PARQUET_VECTORIZED_READER_ENABLED.key}' to false.")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val PARQUET_VECTORIZED_READER_BATCH_SIZE = buildConf("spark.sql.parquet.columnarReaderBatchSize")
+ .doc("The number of rows to include in a parquet vectorized reader batch. The number should " +
+ "be carefully chosen to minimize overhead and avoid OOMs in reading data.")
+ .version("2.4.0")
+ .intConf
+ .createWithDefault(4096)
+
+ val ORC_COMPRESSION = buildConf("spark.sql.orc.compression.codec")
+ .doc("Sets the compression codec used when writing ORC files. If either `compression` or " +
+ "`orc.compress` is specified in the table-specific options/properties, the precedence " +
+ "would be `compression`, `orc.compress`, `spark.sql.orc.compression.codec`." +
+ "Acceptable values include: none, uncompressed, snappy, zlib, lzo, zstd, lz4.")
+ .version("2.3.0")
+ .stringConf
+ .transform(_.toLowerCase(Locale.ROOT))
+ .checkValues(Set("none", "uncompressed", "snappy", "zlib", "lzo", "zstd", "lz4"))
+ .createWithDefault("snappy")
+
+ val ORC_IMPLEMENTATION = buildConf("spark.sql.orc.impl")
+ .doc("When native, use the native version of ORC support instead of the ORC library in Hive. " +
+ "It is 'hive' by default prior to Spark 2.4.")
+ .version("2.3.0")
+ .internal()
+ .stringConf
+ .checkValues(Set("hive", "native"))
+ .createWithDefault("native")
+
+ val ORC_VECTORIZED_READER_ENABLED = buildConf("spark.sql.orc.enableVectorizedReader")
+ .doc("Enables vectorized orc decoding.")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val ORC_VECTORIZED_READER_BATCH_SIZE = buildConf("spark.sql.orc.columnarReaderBatchSize")
+ .doc("The number of rows to include in a orc vectorized reader batch. The number should " +
+ "be carefully chosen to minimize overhead and avoid OOMs in reading data.")
+ .version("2.4.0")
+ .intConf
+ .createWithDefault(4096)
+
+ val ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED =
+ buildConf("spark.sql.orc.enableNestedColumnVectorizedReader")
+ .doc("Enables vectorized orc decoding for nested column.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val ORC_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.orc.filterPushdown")
+ .doc("When true, enable filter pushdown for ORC files.")
+ .version("1.4.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val ORC_SCHEMA_MERGING_ENABLED = buildConf("spark.sql.orc.mergeSchema")
+ .doc("When true, the Orc data source merges schemas collected from all data files, " +
+ "otherwise the schema is picked from a random data file.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val HIVE_VERIFY_PARTITION_PATH = buildConf("spark.sql.hive.verifyPartitionPath")
+ .doc("When true, check all the partition paths under the table\'s root directory " +
+ "when reading data stored in HDFS. This configuration will be deprecated in the future " +
+ s"releases and replaced by ${SPARK_IGNORE_MISSING_FILES.key}.")
+ .version("1.4.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val HIVE_METASTORE_PARTITION_PRUNING =
+ buildConf("spark.sql.hive.metastorePartitionPruning")
+ .doc("When true, some predicates will be pushed down into the Hive metastore so that " +
+ "unmatching partitions can be eliminated earlier.")
+ .version("1.5.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val HIVE_METASTORE_PARTITION_PRUNING_INSET_THRESHOLD =
+ buildConf("spark.sql.hive.metastorePartitionPruningInSetThreshold")
+ .doc("The threshold of set size for InSet predicate when pruning partitions through Hive " +
+ "Metastore. When the set size exceeds the threshold, we rewrite the InSet predicate " +
+ "to be greater than or equal to the minimum value in set and less than or equal to the " +
+ "maximum value in set. Larger values may cause Hive Metastore stack overflow. But for " +
+ "InSet inside Not with values exceeding the threshold, we won't push it to Hive Metastore."
+ )
+ .version("3.1.0")
+ .internal()
+ .intConf
+ .checkValue(_ > 0, "The value of metastorePartitionPruningInSetThreshold must be positive")
+ .createWithDefault(1000)
+
+ val HIVE_MANAGE_FILESOURCE_PARTITIONS =
+ buildConf("spark.sql.hive.manageFilesourcePartitions")
+ .doc("When true, enable metastore partition management for file source tables as well. " +
+ "This includes both datasource and converted Hive tables. When partition management " +
+ "is enabled, datasource tables store partition in the Hive metastore, and use the " +
+ s"metastore to prune partitions during query planning when " +
+ s"${HIVE_METASTORE_PARTITION_PRUNING.key} is set to true.")
+ .version("2.1.1")
+ .booleanConf
+ .createWithDefault(true)
+
+ val HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE =
+ buildConf("spark.sql.hive.filesourcePartitionFileCacheSize")
+ .doc("When nonzero, enable caching of partition file metadata in memory. All tables share " +
+ "a cache that can use up to specified num bytes for file metadata. This conf only " +
+ "has an effect when hive filesource partition management is enabled.")
+ .version("2.1.1")
+ .longConf
+ .createWithDefault(250 * 1024 * 1024)
+
+ object HiveCaseSensitiveInferenceMode extends Enumeration {
+ val INFER_AND_SAVE, INFER_ONLY, NEVER_INFER = Value
+ }
+
+ val HIVE_CASE_SENSITIVE_INFERENCE = buildConf("spark.sql.hive.caseSensitiveInferenceMode")
+ .internal()
+ .doc("Sets the action to take when a case-sensitive schema cannot be read from a Hive Serde " +
+ "table's properties when reading the table with Spark native data sources. Valid options " +
+ "include INFER_AND_SAVE (infer the case-sensitive schema from the underlying data files " +
+ "and write it back to the table properties), INFER_ONLY (infer the schema but don't " +
+ "attempt to write it to the table properties) and NEVER_INFER (the default mode-- fallback " +
+ "to using the case-insensitive metastore schema instead of inferring).")
+ .version("2.1.1")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(HiveCaseSensitiveInferenceMode.values.map(_.toString))
+ .createWithDefault(HiveCaseSensitiveInferenceMode.NEVER_INFER.toString)
+
+ val HIVE_TABLE_PROPERTY_LENGTH_THRESHOLD =
+ buildConf("spark.sql.hive.tablePropertyLengthThreshold")
+ .internal()
+ .doc("The maximum length allowed in a single cell when storing Spark-specific information " +
+ "in Hive's metastore as table properties. Currently it covers 2 things: the schema's " +
+ "JSON string, the histogram of column statistics.")
+ .version("3.2.0")
+ .intConf
+ .createOptional
+
+ val OPTIMIZER_METADATA_ONLY = buildConf("spark.sql.optimizer.metadataOnly")
+ .internal()
+ .doc("When true, enable the metadata-only query optimization that use the table's metadata " +
+ "to produce the partition columns instead of table scans. It applies when all the columns " +
+ "scanned are partition columns and the query has an aggregate operator that satisfies " +
+ "distinct semantics. By default the optimization is disabled, and deprecated as of Spark " +
+ "3.0 since it may return incorrect results when the files are empty, see also SPARK-26709." +
+ "It will be removed in the future releases. If you must use, use 'SparkSessionExtensions' " +
+ "instead to inject it as a custom rule.")
+ .version("2.1.1")
+ .booleanConf
+ .createWithDefault(false)
+
+ val COLUMN_NAME_OF_CORRUPT_RECORD = buildConf("spark.sql.columnNameOfCorruptRecord")
+ .doc("The name of internal column for storing raw/un-parsed JSON and CSV records that fail " +
+ "to parse.")
+ .version("1.2.0")
+ .stringConf
+ .createWithDefault("_corrupt_record")
+
+ val BROADCAST_TIMEOUT = buildConf("spark.sql.broadcastTimeout")
+ .doc("Timeout in seconds for the broadcast wait time in broadcast joins.")
+ .version("1.3.0")
+ .timeConf(TimeUnit.SECONDS)
+ .createWithDefaultString(s"${5 * 60}")
+
+ // This is only used for the thriftserver
+ val THRIFTSERVER_POOL = buildConf("spark.sql.thriftserver.scheduler.pool")
+ .doc("Set a Fair Scheduler pool for a JDBC client session.")
+ .version("1.1.1")
+ .stringConf
+ .createOptional
+
+ val THRIFTSERVER_INCREMENTAL_COLLECT =
+ buildConf("spark.sql.thriftServer.incrementalCollect")
+ .internal()
+ .doc("When true, enable incremental collection for execution in Thrift Server.")
+ .version("2.0.3")
+ .booleanConf
+ .createWithDefault(false)
+
+ val THRIFTSERVER_FORCE_CANCEL =
+ buildConf("spark.sql.thriftServer.interruptOnCancel")
+ .doc("When true, all running tasks will be interrupted if one cancels a query. " +
+ "When false, all running tasks will remain until finished.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val THRIFTSERVER_QUERY_TIMEOUT =
+ buildConf("spark.sql.thriftServer.queryTimeout")
+ .doc("Set a query duration timeout in seconds in Thrift Server. If the timeout is set to " +
+ "a positive value, a running query will be cancelled automatically when the timeout is " +
+ "exceeded, otherwise the query continues to run till completion. If timeout values are " +
+ "set for each statement via `java.sql.Statement.setQueryTimeout` and they are smaller " +
+ "than this configuration value, they take precedence. If you set this timeout and prefer " +
+ "to cancel the queries right away without waiting task to finish, consider enabling " +
+ s"${THRIFTSERVER_FORCE_CANCEL.key} together.")
+ .version("3.1.0")
+ .timeConf(TimeUnit.SECONDS)
+ .createWithDefault(0L)
+
+ val THRIFTSERVER_UI_STATEMENT_LIMIT =
+ buildConf("spark.sql.thriftserver.ui.retainedStatements")
+ .doc("The number of SQL statements kept in the JDBC/ODBC web UI history.")
+ .version("1.4.0")
+ .intConf
+ .createWithDefault(200)
+
+ val THRIFTSERVER_UI_SESSION_LIMIT = buildConf("spark.sql.thriftserver.ui.retainedSessions")
+ .doc("The number of SQL client sessions kept in the JDBC/ODBC web UI history.")
+ .version("1.4.0")
+ .intConf
+ .createWithDefault(200)
+
+ // This is used to set the default data source
+ val DEFAULT_DATA_SOURCE_NAME = buildConf("spark.sql.sources.default")
+ .doc("The default data source to use in input/output.")
+ .version("1.3.0")
+ .stringConf
+ .createWithDefault("parquet")
+
+ val CONVERT_CTAS = buildConf("spark.sql.hive.convertCTAS")
+ .internal()
+ .doc("When true, a table created by a Hive CTAS statement (no USING clause) " +
+ "without specifying any storage property will be converted to a data source table, " +
+ s"using the data source set by ${DEFAULT_DATA_SOURCE_NAME.key}.")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val GATHER_FASTSTAT = buildConf("spark.sql.hive.gatherFastStats")
+ .internal()
+ .doc("When true, fast stats (number of files and total size of all files) will be gathered" +
+ " in parallel while repairing table partitions to avoid the sequential listing in Hive" +
+ " metastore.")
+ .version("2.0.1")
+ .booleanConf
+ .createWithDefault(true)
+
+ val PARTITION_COLUMN_TYPE_INFERENCE =
+ buildConf("spark.sql.sources.partitionColumnTypeInference.enabled")
+ .doc("When true, automatically infer the data types for partitioned columns.")
+ .version("1.5.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val BUCKETING_ENABLED = buildConf("spark.sql.sources.bucketing.enabled")
+ .doc("When false, we will treat bucketed table as normal table")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val BUCKETING_MAX_BUCKETS = buildConf("spark.sql.sources.bucketing.maxBuckets")
+ .doc("The maximum number of buckets allowed.")
+ .version("2.4.0")
+ .intConf
+ .checkValue(_ > 0, "the value of spark.sql.sources.bucketing.maxBuckets must be greater than 0")
+ .createWithDefault(100000)
+
+ val AUTO_BUCKETED_SCAN_ENABLED =
+ buildConf("spark.sql.sources.bucketing.autoBucketedScan.enabled")
+ .doc("When true, decide whether to do bucketed scan on input tables based on query plan " +
+ "automatically. Do not use bucketed scan if 1. query does not have operators to utilize " +
+ "bucketing (e.g. join, group-by, etc), or 2. there's an exchange operator between these " +
+ s"operators and table scan. Note when '${BUCKETING_ENABLED.key}' is set to " +
+ "false, this configuration does not take any effect.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val CAN_CHANGE_CACHED_PLAN_OUTPUT_PARTITIONING =
+ buildConf("spark.sql.optimizer.canChangeCachedPlanOutputPartitioning")
+ .internal()
+ .doc("Whether to forcibly enable some optimization rules that can change the output " +
+ "partitioning of a cached query when executing it for caching. If it is set to true, " +
+ "queries may need an extra shuffle to read the cached data. This configuration is " +
+ "disabled by default. Currently, the optimization rules enabled by this configuration " +
+ s"are ${ADAPTIVE_EXECUTION_ENABLED.key} and ${AUTO_BUCKETED_SCAN_ENABLED.key}.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val CROSS_JOINS_ENABLED = buildConf("spark.sql.crossJoin.enabled")
+ .internal()
+ .doc("When false, we will throw an error if a query contains a cartesian product without " +
+ "explicit CROSS JOIN syntax.")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val ORDER_BY_ORDINAL = buildConf("spark.sql.orderByOrdinal")
+ .doc("When true, the ordinal numbers are treated as the position in the select list. " +
+ "When false, the ordinal numbers in order/sort by clause are ignored.")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val GROUP_BY_ORDINAL = buildConf("spark.sql.groupByOrdinal")
+ .doc("When true, the ordinal numbers in group by clauses are treated as the position " +
+ "in the select list. When false, the ordinal numbers are ignored.")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val GROUP_BY_ALIASES = buildConf("spark.sql.groupByAliases")
+ .doc("When true, aliases in a select list can be used in group by clauses. When false, " +
+ "an analysis exception is thrown in the case.")
+ .version("2.2.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ // The output committer class used by data sources. The specified class needs to be a
+ // subclass of org.apache.hadoop.mapreduce.OutputCommitter.
+ val OUTPUT_COMMITTER_CLASS = buildConf("spark.sql.sources.outputCommitterClass")
+ .version("1.4.0")
+ .internal()
+ .stringConf
+ .createOptional
+
+ val FILE_COMMIT_PROTOCOL_CLASS =
+ buildConf("spark.sql.sources.commitProtocolClass")
+ .version("2.1.1")
+ .internal()
+ .stringConf
+ .createWithDefault(
+ "org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol")
+
+ val PARALLEL_PARTITION_DISCOVERY_THRESHOLD =
+ buildConf("spark.sql.sources.parallelPartitionDiscovery.threshold")
+ .doc("The maximum number of paths allowed for listing files at driver side. If the number " +
+ "of detected paths exceeds this value during partition discovery, it tries to list the " +
+ "files with another Spark distributed job. This configuration is effective only when " +
+ "using file-based sources such as Parquet, JSON and ORC.")
+ .version("1.5.0")
+ .intConf
+ .checkValue(parallel => parallel >= 0, "The maximum number of paths allowed for listing " +
+ "files at driver side must not be negative")
+ .createWithDefault(32)
+
+ val PARALLEL_PARTITION_DISCOVERY_PARALLELISM =
+ buildConf("spark.sql.sources.parallelPartitionDiscovery.parallelism")
+ .doc("The number of parallelism to list a collection of path recursively, Set the " +
+ "number to prevent file listing from generating too many tasks.")
+ .version("2.1.1")
+ .internal()
+ .intConf
+ .createWithDefault(10000)
+
+ val IGNORE_DATA_LOCALITY =
+ buildConf("spark.sql.sources.ignoreDataLocality")
+ .doc("If true, Spark will not fetch the block locations for each file on " +
+ "listing files. This speeds up file listing, but the scheduler cannot " +
+ "schedule tasks to take advantage of data locality. It can be particularly " +
+ "useful if data is read from a remote cluster so the scheduler could never " +
+ "take advantage of locality anyway.")
+ .version("3.0.0")
+ .internal()
+ .booleanConf
+ .createWithDefault(false)
+
+ // Whether to automatically resolve ambiguity in join conditions for self-joins.
+ // See SPARK-6231.
+ val DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY =
+ buildConf("spark.sql.selfJoinAutoResolveAmbiguity")
+ .version("1.4.0")
+ .internal()
+ .booleanConf
+ .createWithDefault(true)
+
+ val FAIL_AMBIGUOUS_SELF_JOIN_ENABLED =
+ buildConf("spark.sql.analyzer.failAmbiguousSelfJoin")
+ .doc("When true, fail the Dataset query if it contains ambiguous self-join.")
+ .version("3.0.0")
+ .internal()
+ .booleanConf
+ .createWithDefault(true)
+
+ // Whether to retain group by columns or not in GroupedData.agg.
+ val DATAFRAME_RETAIN_GROUP_COLUMNS = buildConf("spark.sql.retainGroupColumns")
+ .version("1.4.0")
+ .internal()
+ .booleanConf
+ .createWithDefault(true)
+
+ val DATAFRAME_PIVOT_MAX_VALUES = buildConf("spark.sql.pivotMaxValues")
+ .doc("When doing a pivot without specifying values for the pivot column this is the maximum " +
+ "number of (distinct) values that will be collected without error.")
+ .version("1.6.0")
+ .intConf
+ .createWithDefault(10000)
+
+ val RUN_SQL_ON_FILES = buildConf("spark.sql.runSQLOnFiles")
+ .internal()
+ .doc("When true, we could use `datasource`.`path` as table in SQL query.")
+ .version("1.6.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val WHOLESTAGE_CODEGEN_ENABLED = buildConf("spark.sql.codegen.wholeStage")
+ .internal()
+ .doc("When true, the whole stage (of multiple operators) will be compiled into single java" +
+ " method.")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val WHOLESTAGE_CODEGEN_USE_ID_IN_CLASS_NAME =
+ buildConf("spark.sql.codegen.useIdInClassName")
+ .internal()
+ .doc("When true, embed the (whole-stage) codegen stage ID into " +
+ "the class name of the generated class as a suffix")
+ .version("2.3.1")
+ .booleanConf
+ .createWithDefault(true)
+
+ val WHOLESTAGE_MAX_NUM_FIELDS = buildConf("spark.sql.codegen.maxFields")
+ .internal()
+ .doc("The maximum number of fields (including nested fields) that will be supported before" +
+ " deactivating whole-stage codegen.")
+ .version("2.0.0")
+ .intConf
+ .createWithDefault(100)
+
+ val CODEGEN_FACTORY_MODE = buildConf("spark.sql.codegen.factoryMode")
+ .doc("This config determines the fallback behavior of several codegen generators " +
+ "during tests. `FALLBACK` means trying codegen first and then falling back to " +
+ "interpreted if any compile error happens. Disabling fallback if `CODEGEN_ONLY`. " +
+ "`NO_CODEGEN` skips codegen and goes interpreted path always. Note that " +
+ "this config works only for tests.")
+ .version("2.4.0")
+ .internal()
+ .stringConf
+ .checkValues(CodegenObjectFactoryMode.values.map(_.toString))
+ .createWithDefault(CodegenObjectFactoryMode.FALLBACK.toString)
+
+ val CODEGEN_FALLBACK = buildConf("spark.sql.codegen.fallback")
+ .internal()
+ .doc("When true, (whole stage) codegen could be temporary disabled for the part of query that" +
+ " fail to compile generated code")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val CODEGEN_LOGGING_MAX_LINES = buildConf("spark.sql.codegen.logging.maxLines")
+ .internal()
+ .doc("The maximum number of codegen lines to log when errors occur. Use -1 for unlimited.")
+ .version("2.3.0")
+ .intConf
+ .checkValue(maxLines => maxLines >= -1, "The maximum must be a positive integer, 0 to " +
+ "disable logging or -1 to apply no limit.")
+ .createWithDefault(1000)
+
+ val WHOLESTAGE_HUGE_METHOD_LIMIT = buildConf("spark.sql.codegen.hugeMethodLimit")
+ .internal()
+ .doc("The maximum bytecode size of a single compiled Java function generated by whole-stage " +
+ "codegen. When the compiled function exceeds this threshold, the whole-stage codegen is " +
+ "deactivated for this subtree of the current query plan. The default value is 65535, which " +
+ "is the largest bytecode size possible for a valid Java method. When running on HotSpot, " +
+ s"it may be preferable to set the value to ${CodeGenerator.DEFAULT_JVM_HUGE_METHOD_LIMIT} " +
+ "to match HotSpot's implementation.")
+ .version("2.3.0")
+ .intConf
+ .createWithDefault(65535)
+
+ val CODEGEN_METHOD_SPLIT_THRESHOLD = buildConf("spark.sql.codegen.methodSplitThreshold")
+ .internal()
+ .doc("The threshold of source-code splitting in the codegen. When the number of characters " +
+ "in a single Java function (without comment) exceeds the threshold, the function will be " +
+ "automatically split to multiple smaller ones. We cannot know how many bytecode will be " +
+ "generated, so use the code length as metric. When running on HotSpot, a function's " +
+ "bytecode should not go beyond 8KB, otherwise it will not be JITted; it also should not " +
+ "be too small, otherwise there will be many function calls.")
+ .version("3.0.0")
+ .intConf
+ .checkValue(threshold => threshold > 0, "The threshold must be a positive integer.")
+ .createWithDefault(1024)
+
+ val WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR =
+ buildConf("spark.sql.codegen.splitConsumeFuncByOperator")
+ .internal()
+ .doc("When true, whole stage codegen would put the logic of consuming rows of each " +
+ "physical operator into individual methods, instead of a single big method. This can be " +
+ "used to avoid oversized function that can miss the opportunity of JIT optimization.")
+ .version("2.3.1")
+ .booleanConf
+ .createWithDefault(true)
+
+ val FILES_MAX_PARTITION_BYTES = buildConf("spark.sql.files.maxPartitionBytes")
+ .doc("The maximum number of bytes to pack into a single partition when reading files. " +
+ "This configuration is effective only when using file-based sources such as Parquet, JSON " +
+ "and ORC.")
+ .version("2.0.0")
+ .bytesConf(ByteUnit.BYTE)
+ .createWithDefaultString("128MB") // parquet.block.size
+
+ val FILES_OPEN_COST_IN_BYTES = buildConf("spark.sql.files.openCostInBytes")
+ .internal()
+ .doc("The estimated cost to open a file, measured by the number of bytes could be scanned in" +
+ " the same time. This is used when putting multiple files into a partition. It's better to" +
+ " over estimated, then the partitions with small files will be faster than partitions with" +
+ " bigger files (which is scheduled first). This configuration is effective only when using" +
+ " file-based sources such as Parquet, JSON and ORC.")
+ .version("2.0.0")
+ .longConf
+ .createWithDefault(4 * 1024 * 1024)
+
+ val FILES_MIN_PARTITION_NUM = buildConf("spark.sql.files.minPartitionNum")
+ .doc("The suggested (not guaranteed) minimum number of split file partitions. " +
+ "If not set, the default value is `spark.default.parallelism`. This configuration is " +
+ "effective only when using file-based sources such as Parquet, JSON and ORC.")
+ .version("3.1.0")
+ .intConf
+ .checkValue(v => v > 0, "The min partition number must be a positive integer.")
+ .createOptional
+
+ val IGNORE_CORRUPT_FILES = buildConf("spark.sql.files.ignoreCorruptFiles")
+ .doc("Whether to ignore corrupt files. If true, the Spark jobs will continue to run when " +
+ "encountering corrupted files and the contents that have been read will still be returned. " +
+ "This configuration is effective only when using file-based sources such as Parquet, JSON " +
+ "and ORC.")
+ .version("2.1.1")
+ .booleanConf
+ .createWithDefault(false)
+
+ val IGNORE_MISSING_FILES = buildConf("spark.sql.files.ignoreMissingFiles")
+ .doc("Whether to ignore missing files. If true, the Spark jobs will continue to run when " +
+ "encountering missing files and the contents that have been read will still be returned. " +
+ "This configuration is effective only when using file-based sources such as Parquet, JSON " +
+ "and ORC.")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val MAX_RECORDS_PER_FILE = buildConf("spark.sql.files.maxRecordsPerFile")
+ .doc("Maximum number of records to write out to a single file. " +
+ "If this value is zero or negative, there is no limit.")
+ .version("2.2.0")
+ .longConf
+ .createWithDefault(0)
+
+ val EXCHANGE_REUSE_ENABLED = buildConf("spark.sql.exchange.reuse")
+ .internal()
+ .doc("When true, the planner will try to find out duplicated exchanges and re-use them.")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val SUBQUERY_REUSE_ENABLED = buildConf("spark.sql.execution.reuseSubquery")
+ .internal()
+ .doc("When true, the planner will try to find out duplicated subqueries and re-use them.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val REMOVE_REDUNDANT_PROJECTS_ENABLED = buildConf("spark.sql.execution.removeRedundantProjects")
+ .internal()
+ .doc("Whether to remove redundant project exec node based on children's output and " +
+ "ordering requirement.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val REMOVE_REDUNDANT_SORTS_ENABLED = buildConf("spark.sql.execution.removeRedundantSorts")
+ .internal()
+ .doc("Whether to remove redundant physical sort node")
+ .version("2.4.8")
+ .booleanConf
+ .createWithDefault(true)
+
+ val STATE_STORE_PROVIDER_CLASS =
+ buildConf("spark.sql.streaming.stateStore.providerClass")
+ .internal()
+ .doc(
+ "The class used to manage state data in stateful streaming queries. This class must " +
+ "be a subclass of StateStoreProvider, and must have a zero-arg constructor. " +
+ "Note: For structured streaming, this configuration cannot be changed between query " +
+ "restarts from the same checkpoint location.")
+ .version("2.3.0")
+ .stringConf
+ .createWithDefault(
+ "org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider")
+
+ val STATE_SCHEMA_CHECK_ENABLED =
+ buildConf("spark.sql.streaming.stateStore.stateSchemaCheck")
+ .doc("When true, Spark will validate the state schema against schema on existing state and " +
+ "fail query if it's incompatible.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT =
+ buildConf("spark.sql.streaming.stateStore.minDeltasForSnapshot")
+ .internal()
+ .doc("Minimum number of state store delta files that needs to be generated before they " +
+ "consolidated into snapshots.")
+ .version("2.0.0")
+ .intConf
+ .createWithDefault(10)
+
+ val STATE_STORE_FORMAT_VALIDATION_ENABLED =
+ buildConf("spark.sql.streaming.stateStore.formatValidation.enabled")
+ .internal()
+ .doc("When true, check if the data from state store is valid or not when running streaming " +
+ "queries. This can happen if the state store format has been changed. Note, the feature " +
+ "is only effective in the build-in HDFS state store provider now.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION =
+ buildConf("spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion")
+ .internal()
+ .doc("State format version used by flatMapGroupsWithState operation in a streaming query")
+ .version("2.4.0")
+ .intConf
+ .checkValue(v => Set(1, 2).contains(v), "Valid versions are 1 and 2")
+ .createWithDefault(2)
+
+ val CHECKPOINT_LOCATION = buildConf("spark.sql.streaming.checkpointLocation")
+ .doc("The default location for storing checkpoint data for streaming queries.")
+ .version("2.0.0")
+ .stringConf
+ .createOptional
+
+ val FORCE_DELETE_TEMP_CHECKPOINT_LOCATION =
+ buildConf("spark.sql.streaming.forceDeleteTempCheckpointLocation")
+ .doc("When true, enable temporary checkpoint locations force delete.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val MIN_BATCHES_TO_RETAIN = buildConf("spark.sql.streaming.minBatchesToRetain")
+ .internal()
+ .doc("The minimum number of batches that must be retained and made recoverable.")
+ .version("2.1.1")
+ .intConf
+ .createWithDefault(100)
+
+ val MAX_BATCHES_TO_RETAIN_IN_MEMORY = buildConf("spark.sql.streaming.maxBatchesToRetainInMemory")
+ .internal()
+ .doc("The maximum number of batches which will be retained in memory to avoid " +
+ "loading from files. The value adjusts a trade-off between memory usage vs cache miss: " +
+ "'2' covers both success and direct failure cases, '1' covers only success case, " +
+ "and '0' covers extreme case - disable cache to maximize memory size of executors.")
+ .version("2.4.0")
+ .intConf
+ .createWithDefault(2)
+
+ val STREAMING_MAINTENANCE_INTERVAL =
+ buildConf("spark.sql.streaming.stateStore.maintenanceInterval")
+ .internal()
+ .doc("The interval in milliseconds between triggering maintenance tasks in StateStore. " +
+ "The maintenance task executes background maintenance task in all the loaded store " +
+ "providers if they are still the active instances according to the coordinator. If not, " +
+ "inactive instances of store providers will be closed.")
+ .version("2.0.0")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefault(TimeUnit.MINUTES.toMillis(1)) // 1 minute
+
+ val STATE_STORE_COMPRESSION_CODEC =
+ buildConf("spark.sql.streaming.stateStore.compression.codec")
+ .internal()
+ .doc("The codec used to compress delta and snapshot files generated by StateStore. " +
+ "By default, Spark provides four codecs: lz4, lzf, snappy, and zstd. You can also " +
+ "use fully qualified class names to specify the codec. Default codec is lz4.")
+ .version("3.1.0")
+ .stringConf
+ .createWithDefault("lz4")
+
+ /**
+ * Note: this is defined in `RocksDBConf.FORMAT_VERSION`. These two places should be updated
+ * together.
+ */
+ val STATE_STORE_ROCKSDB_FORMAT_VERSION =
+ buildConf("spark.sql.streaming.stateStore.rocksdb.formatVersion")
+ .internal()
+ .doc("Set the RocksDB format version. This will be stored in the checkpoint when starting " +
+ "a streaming query. The checkpoint will use this RocksDB format version in the entire " +
+ "lifetime of the query.")
+ .version("3.2.0")
+ .intConf
+ .checkValue(_ >= 0, "Must not be negative")
+ // 5 is the default table format version for RocksDB 6.20.3.
+ .createWithDefault(5)
+
+ val STREAMING_AGGREGATION_STATE_FORMAT_VERSION =
+ buildConf("spark.sql.streaming.aggregation.stateFormatVersion")
+ .internal()
+ .doc("State format version used by streaming aggregation operations in a streaming query. " +
+ "State between versions are tend to be incompatible, so state format version shouldn't " +
+ "be modified after running.")
+ .version("2.4.0")
+ .intConf
+ .checkValue(v => Set(1, 2).contains(v), "Valid versions are 1 and 2")
+ .createWithDefault(2)
+
+ val STREAMING_STOP_ACTIVE_RUN_ON_RESTART =
+ buildConf("spark.sql.streaming.stopActiveRunOnRestart")
+ .doc("Running multiple runs of the same streaming query concurrently is not supported. " +
+ "If we find a concurrent active run for a streaming query (in the same or different " +
+ "SparkSessions on the same cluster) and this flag is true, we will stop the old streaming " +
+ "query run to start the new one.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val STREAMING_JOIN_STATE_FORMAT_VERSION =
+ buildConf("spark.sql.streaming.join.stateFormatVersion")
+ .internal()
+ .doc("State format version used by streaming join operations in a streaming query. " +
+ "State between versions are tend to be incompatible, so state format version shouldn't " +
+ "be modified after running.")
+ .version("3.0.0")
+ .intConf
+ .checkValue(v => Set(1, 2).contains(v), "Valid versions are 1 and 2")
+ .createWithDefault(2)
+
+ val STREAMING_SESSION_WINDOW_MERGE_SESSIONS_IN_LOCAL_PARTITION =
+ buildConf("spark.sql.streaming.sessionWindow.merge.sessions.in.local.partition")
+ .internal()
+ .doc("When true, streaming session window sorts and merge sessions in local partition " +
+ "prior to shuffle. This is to reduce the rows to shuffle, but only beneficial when " +
+ "there're lots of rows in a batch being assigned to same sessions.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val STREAMING_SESSION_WINDOW_STATE_FORMAT_VERSION =
+ buildConf("spark.sql.streaming.sessionWindow.stateFormatVersion")
+ .internal()
+ .doc("State format version used by streaming session window in a streaming query. " +
+ "State between versions are tend to be incompatible, so state format version shouldn't " +
+ "be modified after running.")
+ .version("3.2.0")
+ .intConf
+ .checkValue(v => Set(1).contains(v), "Valid version is 1")
+ .createWithDefault(1)
+
+ val UNSUPPORTED_OPERATION_CHECK_ENABLED =
+ buildConf("spark.sql.streaming.unsupportedOperationCheck")
+ .internal()
+ .doc("When true, the logical plan for streaming query will be checked for unsupported" +
+ " operations.")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val USE_DEPRECATED_KAFKA_OFFSET_FETCHING =
+ buildConf("spark.sql.streaming.kafka.useDeprecatedOffsetFetching")
+ .internal()
+ .doc("When true, the deprecated Consumer based offset fetching used which could cause " +
+ "infinite wait in Spark queries. Such cases query restart is the only workaround. " +
+ "For further details please see Offset Fetching chapter of Structured Streaming Kafka " +
+ "Integration Guide.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val STATEFUL_OPERATOR_CHECK_CORRECTNESS_ENABLED =
+ buildConf("spark.sql.streaming.statefulOperator.checkCorrectness.enabled")
+ .internal()
+ .doc("When true, the stateful operators for streaming query will be checked for possible " +
+ "correctness issue due to global watermark. The correctness issue comes from queries " +
+ "containing stateful operation which can emit rows older than the current watermark " +
+ "plus allowed late record delay, which are \"late rows\" in downstream stateful " +
+ "operations and these rows can be discarded. Please refer the programming guide doc for " +
+ "more details. Once the issue is detected, Spark will throw analysis exception. " +
+ "When this config is disabled, Spark will just print warning message for users. " +
+ "Prior to Spark 3.1.0, the behavior is disabling this config.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val FILESTREAM_SINK_METADATA_IGNORED =
+ buildConf("spark.sql.streaming.fileStreamSink.ignoreMetadata")
+ .internal()
+ .doc("If this is enabled, when Spark reads from the results of a streaming query written " +
+ "by `FileStreamSink`, Spark will ignore the metadata log and treat it as normal path to " +
+ "read, e.g. listing files using HDFS APIs.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val VARIABLE_SUBSTITUTE_ENABLED =
+ buildConf("spark.sql.variable.substitute")
+ .doc("This enables substitution using syntax like `${var}`, `${system:var}`, " +
+ "and `${env:var}`.")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val ENABLE_TWOLEVEL_AGG_MAP =
+ buildConf("spark.sql.codegen.aggregate.map.twolevel.enabled")
+ .internal()
+ .doc("Enable two-level aggregate hash map. When enabled, records will first be " +
+ "inserted/looked-up at a 1st-level, small, fast map, and then fallback to a " +
+ "2nd-level, larger, slower map when 1st level is full or keys cannot be found. " +
+ "When disabled, records go directly to the 2nd level.")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val ENABLE_TWOLEVEL_AGG_MAP_PARTIAL_ONLY =
+ buildConf("spark.sql.codegen.aggregate.map.twolevel.partialOnly")
+ .internal()
+ .doc("Enable two-level aggregate hash map for partial aggregate only, " +
+ "because final aggregate might get more distinct keys compared to partial aggregate. " +
+ "Overhead of looking up 1st-level map might dominate when having a lot of distinct keys.")
+ .version("3.2.1")
+ .booleanConf
+ .createWithDefault(true)
+
+ val ENABLE_VECTORIZED_HASH_MAP =
+ buildConf("spark.sql.codegen.aggregate.map.vectorized.enable")
+ .internal()
+ .doc("Enable vectorized aggregate hash map. This is for testing/benchmarking only.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val CODEGEN_SPLIT_AGGREGATE_FUNC =
+ buildConf("spark.sql.codegen.aggregate.splitAggregateFunc.enabled")
+ .internal()
+ .doc("When true, the code generator would split aggregate code into individual methods " +
+ "instead of a single big method. This can be used to avoid oversized function that " +
+ "can miss the opportunity of JIT optimization.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val MAX_NESTED_VIEW_DEPTH =
+ buildConf("spark.sql.view.maxNestedViewDepth")
+ .internal()
+ .doc("The maximum depth of a view reference in a nested view. A nested view may reference " +
+ "other nested views, the dependencies are organized in a directed acyclic graph (DAG). " +
+ "However the DAG depth may become too large and cause unexpected behavior. This " +
+ "configuration puts a limit on this: when the depth of a view exceeds this value during " +
+ "analysis, we terminate the resolution to avoid potential errors.")
+ .version("2.2.0")
+ .intConf
+ .checkValue(depth => depth > 0, "The maximum depth of a view reference in a nested view " +
+ "must be positive.")
+ .createWithDefault(100)
+
+ val ALLOW_PARAMETERLESS_COUNT =
+ buildConf("spark.sql.legacy.allowParameterlessCount")
+ .internal()
+ .doc("When true, the SQL function 'count' is allowed to take no parameters.")
+ .version("3.1.1")
+ .booleanConf
+ .createWithDefault(false)
+
+ val ALLOW_NON_EMPTY_LOCATION_IN_CTAS =
+ buildConf("spark.sql.legacy.allowNonEmptyLocationInCTAS")
+ .internal()
+ .doc("When false, CTAS with LOCATION throws an analysis exception if the " +
+ "location is not empty.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val ALLOW_STAR_WITH_SINGLE_TABLE_IDENTIFIER_IN_COUNT =
+ buildConf("spark.sql.legacy.allowStarWithSingleTableIdentifierInCount")
+ .internal()
+ .doc("When true, the SQL function 'count' is allowed to take single 'tblName.*' as parameter")
+ .version("3.2")
+ .booleanConf
+ .createWithDefault(false)
+
+ val USE_CURRENT_SQL_CONFIGS_FOR_VIEW =
+ buildConf("spark.sql.legacy.useCurrentConfigsForView")
+ .internal()
+ .doc("When true, SQL Configs of the current active SparkSession instead of the captured " +
+ "ones will be applied during the parsing and analysis phases of the view resolution.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val STORE_ANALYZED_PLAN_FOR_VIEW =
+ buildConf("spark.sql.legacy.storeAnalyzedPlanForView")
+ .internal()
+ .doc("When true, analyzed plan instead of SQL text will be stored when creating " +
+ "temporary view")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val ALLOW_AUTO_GENERATED_ALIAS_FOR_VEW =
+ buildConf("spark.sql.legacy.allowAutoGeneratedAliasForView")
+ .internal()
+ .doc("When true, it's allowed to use a input query without explicit alias when creating " +
+ "a permanent view.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val STREAMING_FILE_COMMIT_PROTOCOL_CLASS =
+ buildConf("spark.sql.streaming.commitProtocolClass")
+ .version("2.1.0")
+ .internal()
+ .stringConf
+ .createWithDefault("org.apache.spark.sql.execution.streaming.ManifestFileCommitProtocol")
+
+ val STREAMING_MULTIPLE_WATERMARK_POLICY =
+ buildConf("spark.sql.streaming.multipleWatermarkPolicy")
+ .doc("Policy to calculate the global watermark value when there are multiple watermark " +
+ "operators in a streaming query. The default value is 'min' which chooses " +
+ "the minimum watermark reported across multiple operators. Other alternative value is " +
+ "'max' which chooses the maximum across multiple operators. " +
+ "Note: This configuration cannot be changed between query restarts from the same " +
+ "checkpoint location.")
+ .version("2.4.0")
+ .stringConf
+ .transform(_.toLowerCase(Locale.ROOT))
+ .checkValue(
+ str => Set("min", "max").contains(str),
+ "Invalid value for 'spark.sql.streaming.multipleWatermarkPolicy'. " +
+ "Valid values are 'min' and 'max'")
+ .createWithDefault("min") // must be same as MultipleWatermarkPolicy.DEFAULT_POLICY_NAME
+
+ val OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD =
+ buildConf("spark.sql.objectHashAggregate.sortBased.fallbackThreshold")
+ .internal()
+ .doc("In the case of ObjectHashAggregateExec, when the size of the in-memory hash map " +
+ "grows too large, we will fall back to sort-based aggregation. This option sets a row " +
+ "count threshold for the size of the hash map.")
+ .version("2.2.0")
+ .intConf
+ // We are trying to be conservative and use a relatively small default count threshold here
+ // since the state object of some TypedImperativeAggregate function can be quite large (e.g.
+ // percentile_approx).
+ .createWithDefault(128)
+
+ val USE_OBJECT_HASH_AGG = buildConf("spark.sql.execution.useObjectHashAggregateExec")
+ .internal()
+ .doc("Decides if we use ObjectHashAggregateExec")
+ .version("2.2.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val JSON_GENERATOR_IGNORE_NULL_FIELDS =
+ buildConf("spark.sql.jsonGenerator.ignoreNullFields")
+ .doc("Whether to ignore null fields when generating JSON objects in JSON data source and " +
+ "JSON functions such as to_json. " +
+ "If false, it generates null for null fields in JSON objects.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val JSON_EXPRESSION_OPTIMIZATION =
+ buildConf("spark.sql.optimizer.enableJsonExpressionOptimization")
+ .doc("Whether to optimize JSON expressions in SQL optimizer. It includes pruning " +
+ "unnecessary columns from from_json, simplifying from_json + to_json, to_json + " +
+ "named_struct(from_json.col1, from_json.col2, ....).")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val CSV_EXPRESSION_OPTIMIZATION =
+ buildConf("spark.sql.optimizer.enableCsvExpressionOptimization")
+ .doc("Whether to optimize CSV expressions in SQL optimizer. It includes pruning " +
+ "unnecessary columns from from_csv.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val FILE_SINK_LOG_DELETION = buildConf("spark.sql.streaming.fileSink.log.deletion")
+ .internal()
+ .doc("Whether to delete the expired log files in file stream sink.")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val FILE_SINK_LOG_COMPACT_INTERVAL =
+ buildConf("spark.sql.streaming.fileSink.log.compactInterval")
+ .internal()
+ .doc("Number of log files after which all the previous files " +
+ "are compacted into the next log file.")
+ .version("2.0.0")
+ .intConf
+ .createWithDefault(10)
+
+ val FILE_SINK_LOG_CLEANUP_DELAY =
+ buildConf("spark.sql.streaming.fileSink.log.cleanupDelay")
+ .internal()
+ .doc("How long that a file is guaranteed to be visible for all readers.")
+ .version("2.0.0")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefault(TimeUnit.MINUTES.toMillis(10)) // 10 minutes
+
+ val FILE_SOURCE_LOG_DELETION = buildConf("spark.sql.streaming.fileSource.log.deletion")
+ .internal()
+ .doc("Whether to delete the expired log files in file stream source.")
+ .version("2.0.1")
+ .booleanConf
+ .createWithDefault(true)
+
+ val FILE_SOURCE_LOG_COMPACT_INTERVAL =
+ buildConf("spark.sql.streaming.fileSource.log.compactInterval")
+ .internal()
+ .doc("Number of log files after which all the previous files " +
+ "are compacted into the next log file.")
+ .version("2.0.1")
+ .intConf
+ .createWithDefault(10)
+
+ val FILE_SOURCE_LOG_CLEANUP_DELAY =
+ buildConf("spark.sql.streaming.fileSource.log.cleanupDelay")
+ .internal()
+ .doc("How long in milliseconds a file is guaranteed to be visible for all readers.")
+ .version("2.0.1")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefault(TimeUnit.MINUTES.toMillis(10)) // 10 minutes
+
+ val FILE_SOURCE_SCHEMA_FORCE_NULLABLE =
+ buildConf("spark.sql.streaming.fileSource.schema.forceNullable")
+ .internal()
+ .doc("When true, force the schema of streaming file source to be nullable (including all " +
+ "the fields). Otherwise, the schema might not be compatible with actual data, which " +
+ "leads to corruptions.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val FILE_SOURCE_CLEANER_NUM_THREADS =
+ buildConf("spark.sql.streaming.fileSource.cleaner.numThreads")
+ .doc("Number of threads used in the file source completed file cleaner.")
+ .version("3.0.0")
+ .intConf
+ .createWithDefault(1)
+
+ val STREAMING_SCHEMA_INFERENCE =
+ buildConf("spark.sql.streaming.schemaInference")
+ .internal()
+ .doc("Whether file-based streaming sources will infer its own schema")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val STREAMING_POLLING_DELAY =
+ buildConf("spark.sql.streaming.pollingDelay")
+ .internal()
+ .doc("How long to delay polling new data when no data is available")
+ .version("2.0.0")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefault(10L)
+
+ val STREAMING_STOP_TIMEOUT =
+ buildConf("spark.sql.streaming.stopTimeout")
+ .doc("How long to wait in milliseconds for the streaming execution thread to stop when " +
+ "calling the streaming query's stop() method. 0 or negative values wait indefinitely.")
+ .version("3.0.0")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefaultString("0")
+
+ val STREAMING_NO_DATA_PROGRESS_EVENT_INTERVAL =
+ buildConf("spark.sql.streaming.noDataProgressEventInterval")
+ .internal()
+ .doc("How long to wait between two progress events when there is no data")
+ .version("2.1.1")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefault(10000L)
+
+ val STREAMING_NO_DATA_MICRO_BATCHES_ENABLED =
+ buildConf("spark.sql.streaming.noDataMicroBatches.enabled")
+ .doc(
+ "Whether streaming micro-batch engine will execute batches without data " +
+ "for eager state management for stateful streaming queries.")
+ .version("2.4.1")
+ .booleanConf
+ .createWithDefault(true)
+
+ val STREAMING_METRICS_ENABLED =
+ buildConf("spark.sql.streaming.metricsEnabled")
+ .doc("Whether Dropwizard/Codahale metrics will be reported for active streaming queries.")
+ .version("2.0.2")
+ .booleanConf
+ .createWithDefault(false)
+
+ val STREAMING_PROGRESS_RETENTION =
+ buildConf("spark.sql.streaming.numRecentProgressUpdates")
+ .doc("The number of progress updates to retain for a streaming query")
+ .version("2.1.1")
+ .intConf
+ .createWithDefault(100)
+
+ val STREAMING_CHECKPOINT_FILE_MANAGER_CLASS =
+ buildConf("spark.sql.streaming.checkpointFileManagerClass")
+ .internal()
+ .doc("The class used to write checkpoint files atomically. This class must be a subclass " +
+ "of the interface CheckpointFileManager.")
+ .version("2.4.0")
+ .stringConf
+
+ val STREAMING_CHECKPOINT_ESCAPED_PATH_CHECK_ENABLED =
+ buildConf("spark.sql.streaming.checkpoint.escapedPathCheck.enabled")
+ .internal()
+ .doc("Whether to detect a streaming query may pick up an incorrect checkpoint path due " +
+ "to SPARK-26824.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val PARALLEL_FILE_LISTING_IN_STATS_COMPUTATION =
+ buildConf("spark.sql.statistics.parallelFileListingInStatsComputation.enabled")
+ .internal()
+ .doc("When true, SQL commands use parallel file listing, " +
+ "as opposed to single thread listing. " +
+ "This usually speeds up commands that need to list many directories.")
+ .version("2.4.1")
+ .booleanConf
+ .createWithDefault(true)
+
+ val DEFAULT_SIZE_IN_BYTES = buildConf("spark.sql.defaultSizeInBytes")
+ .internal()
+ .doc("The default table size used in query planning. By default, it is set to Long.MaxValue " +
+ s"which is larger than `${AUTO_BROADCASTJOIN_THRESHOLD.key}` to be more conservative. " +
+ "That is to say by default the optimizer will not choose to broadcast a table unless it " +
+ "knows for sure its size is small enough.")
+ .version("1.1.0")
+ .bytesConf(ByteUnit.BYTE)
+ .createWithDefault(Long.MaxValue)
+
+ val ENABLE_FALL_BACK_TO_HDFS_FOR_STATS = buildConf("spark.sql.statistics.fallBackToHdfs")
+ .doc("When true, it will fall back to HDFS if the table statistics are not available from " +
+ "table metadata. This is useful in determining if a table is small enough to use " +
+ "broadcast joins. This flag is effective only for non-partitioned Hive tables. " +
+ "For non-partitioned data source tables, it will be automatically recalculated if table " +
+ "statistics are not available. For partitioned data source and partitioned Hive tables, " +
+ s"It is '${DEFAULT_SIZE_IN_BYTES.key}' if table statistics are not available.")
+ .version("2.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val NDV_MAX_ERROR =
+ buildConf("spark.sql.statistics.ndv.maxError")
+ .internal()
+ .doc("The maximum relative standard deviation allowed in HyperLogLog++ algorithm " +
+ "when generating column level statistics.")
+ .version("2.1.1")
+ .doubleConf
+ .createWithDefault(0.05)
+
+ val HISTOGRAM_ENABLED =
+ buildConf("spark.sql.statistics.histogram.enabled")
+ .doc("Generates histograms when computing column statistics if enabled. Histograms can " +
+ "provide better estimation accuracy. Currently, Spark only supports equi-height " +
+ "histogram. Note that collecting histograms takes extra cost. For example, collecting " +
+ "column statistics usually takes only one table scan, but generating equi-height " +
+ "histogram will cause an extra table scan.")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val HISTOGRAM_NUM_BINS =
+ buildConf("spark.sql.statistics.histogram.numBins")
+ .internal()
+ .doc("The number of bins when generating histograms.")
+ .version("2.3.0")
+ .intConf
+ .checkValue(num => num > 1, "The number of bins must be greater than 1.")
+ .createWithDefault(254)
+
+ val PERCENTILE_ACCURACY =
+ buildConf("spark.sql.statistics.percentile.accuracy")
+ .internal()
+ .doc("Accuracy of percentile approximation when generating equi-height histograms. " +
+ "Larger value means better accuracy. The relative error can be deduced by " +
+ "1.0 / PERCENTILE_ACCURACY.")
+ .version("2.3.0")
+ .intConf
+ .createWithDefault(10000)
+
+ val AUTO_SIZE_UPDATE_ENABLED =
+ buildConf("spark.sql.statistics.size.autoUpdate.enabled")
+ .doc("Enables automatic update for table size once table's data is changed. Note that if " +
+ "the total number of files of the table is very large, this can be expensive and slow " +
+ "down data change commands.")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val CBO_ENABLED =
+ buildConf("spark.sql.cbo.enabled")
+ .doc("Enables CBO for estimation of plan statistics when set true.")
+ .version("2.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val PLAN_STATS_ENABLED =
+ buildConf("spark.sql.cbo.planStats.enabled")
+ .doc("When true, the logical plan will fetch row counts and column statistics from catalog.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val JOIN_REORDER_ENABLED =
+ buildConf("spark.sql.cbo.joinReorder.enabled")
+ .doc("Enables join reorder in CBO.")
+ .version("2.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val JOIN_REORDER_DP_THRESHOLD =
+ buildConf("spark.sql.cbo.joinReorder.dp.threshold")
+ .doc("The maximum number of joined nodes allowed in the dynamic programming algorithm.")
+ .version("2.2.0")
+ .intConf
+ .checkValue(number => number > 0, "The maximum number must be a positive integer.")
+ .createWithDefault(12)
+
+ val JOIN_REORDER_CARD_WEIGHT =
+ buildConf("spark.sql.cbo.joinReorder.card.weight")
+ .internal()
+ .doc("The weight of the ratio of cardinalities (number of rows) " +
+ "in the cost comparison function. The ratio of sizes in bytes has weight " +
+ "1 - this value. The weighted geometric mean of these ratios is used to decide " +
+ "which of the candidate plans will be chosen by the CBO.")
+ .version("2.2.0")
+ .doubleConf
+ .checkValue(weight => weight >= 0 && weight <= 1, "The weight value must be in [0, 1].")
+ .createWithDefault(0.7)
+
+ val JOIN_REORDER_DP_STAR_FILTER =
+ buildConf("spark.sql.cbo.joinReorder.dp.star.filter")
+ .doc("Applies star-join filter heuristics to cost based join enumeration.")
+ .version("2.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val STARSCHEMA_DETECTION = buildConf("spark.sql.cbo.starSchemaDetection")
+ .doc("When true, it enables join reordering based on star schema detection. ")
+ .version("2.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val STARSCHEMA_FACT_TABLE_RATIO = buildConf("spark.sql.cbo.starJoinFTRatio")
+ .internal()
+ .doc("Specifies the upper limit of the ratio between the largest fact tables" +
+ " for a star join to be considered. ")
+ .version("2.2.0")
+ .doubleConf
+ .createWithDefault(0.9)
+
+ private def isValidTimezone(zone: String): Boolean = {
+ Try { DateTimeUtils.getZoneId(zone) }.isSuccess
+ }
+
+ val SESSION_LOCAL_TIMEZONE = buildConf("spark.sql.session.timeZone")
+ .doc("The ID of session local timezone in the format of either region-based zone IDs or " +
+ "zone offsets. Region IDs must have the form 'area/city', such as 'America/Los_Angeles'. " +
+ "Zone offsets must be in the format '(+|-)HH', '(+|-)HH:mm' or '(+|-)HH:mm:ss', e.g '-08', " +
+ "'+01:00' or '-13:33:33'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'. Other " +
+ "short names are not recommended to use because they can be ambiguous.")
+ .version("2.2.0")
+ .stringConf
+ .checkValue(isValidTimezone, s"Cannot resolve the given timezone with" +
+ " ZoneId.of(_, ZoneId.SHORT_IDS)")
+ .createWithDefaultFunction(() => TimeZone.getDefault.getID)
+
+ val WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD =
+ buildConf("spark.sql.windowExec.buffer.in.memory.threshold")
+ .internal()
+ .doc("Threshold for number of rows guaranteed to be held in memory by the window operator")
+ .version("2.2.1")
+ .intConf
+ .createWithDefault(4096)
+
+ val WINDOW_EXEC_BUFFER_SPILL_THRESHOLD =
+ buildConf("spark.sql.windowExec.buffer.spill.threshold")
+ .internal()
+ .doc("Threshold for number of rows to be spilled by window operator")
+ .version("2.2.0")
+ .intConf
+ .createWithDefault(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD.defaultValue.get)
+
+ val SESSION_WINDOW_BUFFER_IN_MEMORY_THRESHOLD =
+ buildConf("spark.sql.sessionWindow.buffer.in.memory.threshold")
+ .internal()
+ .doc("Threshold for number of windows guaranteed to be held in memory by the " +
+ "session window operator. Note that the buffer is used only for the query Spark " +
+ "cannot apply aggregations on determining session window.")
+ .version("3.2.0")
+ .intConf
+ .createWithDefault(4096)
+
+ val SESSION_WINDOW_BUFFER_SPILL_THRESHOLD =
+ buildConf("spark.sql.sessionWindow.buffer.spill.threshold")
+ .internal()
+ .doc("Threshold for number of rows to be spilled by window operator. Note that " +
+ "the buffer is used only for the query Spark cannot apply aggregations on determining " +
+ "session window.")
+ .version("3.2.0")
+ .intConf
+ .createWithDefault(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD.defaultValue.get)
+
+ val SORT_MERGE_JOIN_EXEC_BUFFER_IN_MEMORY_THRESHOLD =
+ buildConf("spark.sql.sortMergeJoinExec.buffer.in.memory.threshold")
+ .internal()
+ .doc("Threshold for number of rows guaranteed to be held in memory by the sort merge " +
+ "join operator")
+ .version("2.2.1")
+ .intConf
+ .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH)
+
+ val SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD =
+ buildConf("spark.sql.sortMergeJoinExec.buffer.spill.threshold")
+ .internal()
+ .doc("Threshold for number of rows to be spilled by sort merge join operator")
+ .version("2.2.0")
+ .intConf
+ .createWithDefault(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD.defaultValue.get)
+
+ val CARTESIAN_PRODUCT_EXEC_BUFFER_IN_MEMORY_THRESHOLD =
+ buildConf("spark.sql.cartesianProductExec.buffer.in.memory.threshold")
+ .internal()
+ .doc("Threshold for number of rows guaranteed to be held in memory by the cartesian " +
+ "product operator")
+ .version("2.2.1")
+ .intConf
+ .createWithDefault(4096)
+
+ val CARTESIAN_PRODUCT_EXEC_BUFFER_SPILL_THRESHOLD =
+ buildConf("spark.sql.cartesianProductExec.buffer.spill.threshold")
+ .internal()
+ .doc("Threshold for number of rows to be spilled by cartesian product operator")
+ .version("2.2.0")
+ .intConf
+ .createWithDefault(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD.defaultValue.get)
+
+ val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames")
+ .doc("When true, quoted Identifiers (using backticks) in SELECT statement are interpreted" +
+ " as regular expressions.")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val RANGE_EXCHANGE_SAMPLE_SIZE_PER_PARTITION =
+ buildConf("spark.sql.execution.rangeExchange.sampleSizePerPartition")
+ .internal()
+ .doc("Number of points to sample per partition in order to determine the range boundaries" +
+ " for range partitioning, typically used in global sorting (without limit).")
+ .version("2.3.0")
+ .intConf
+ .createWithDefault(100)
+
+ val ARROW_EXECUTION_ENABLED =
+ buildConf("spark.sql.execution.arrow.enabled")
+ .doc("(Deprecated since Spark 3.0, please set 'spark.sql.execution.arrow.pyspark.enabled'.)")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val ARROW_PYSPARK_EXECUTION_ENABLED =
+ buildConf("spark.sql.execution.arrow.pyspark.enabled")
+ .doc("When true, make use of Apache Arrow for columnar data transfers in PySpark. " +
+ "This optimization applies to: " +
+ "1. pyspark.sql.DataFrame.toPandas " +
+ "2. pyspark.sql.SparkSession.createDataFrame when its input is a Pandas DataFrame " +
+ "The following data types are unsupported: " +
+ "ArrayType of TimestampType, and nested StructType.")
+ .version("3.0.0")
+ .fallbackConf(ARROW_EXECUTION_ENABLED)
+
+ val ARROW_PYSPARK_SELF_DESTRUCT_ENABLED =
+ buildConf("spark.sql.execution.arrow.pyspark.selfDestruct.enabled")
+ .doc("(Experimental) When true, make use of Apache Arrow's self-destruct and split-blocks " +
+ "options for columnar data transfers in PySpark, when converting from Arrow to Pandas. " +
+ "This reduces memory usage at the cost of some CPU time. " +
+ "This optimization applies to: pyspark.sql.DataFrame.toPandas " +
+ "when 'spark.sql.execution.arrow.pyspark.enabled' is set.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val PYSPARK_JVM_STACKTRACE_ENABLED =
+ buildConf("spark.sql.pyspark.jvmStacktrace.enabled")
+ .doc("When true, it shows the JVM stacktrace in the user-facing PySpark exception " +
+ "together with Python stacktrace. By default, it is disabled and hides JVM stacktrace " +
+ "and shows a Python-friendly exception only.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val ARROW_SPARKR_EXECUTION_ENABLED =
+ buildConf("spark.sql.execution.arrow.sparkr.enabled")
+ .doc("When true, make use of Apache Arrow for columnar data transfers in SparkR. " +
+ "This optimization applies to: " +
+ "1. createDataFrame when its input is an R DataFrame " +
+ "2. collect " +
+ "3. dapply " +
+ "4. gapply " +
+ "The following data types are unsupported: " +
+ "FloatType, BinaryType, ArrayType, StructType and MapType.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val ARROW_FALLBACK_ENABLED =
+ buildConf("spark.sql.execution.arrow.fallback.enabled")
+ .doc("(Deprecated since Spark 3.0, please set " +
+ "'spark.sql.execution.arrow.pyspark.fallback.enabled'.)")
+ .version("2.4.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val ARROW_PYSPARK_FALLBACK_ENABLED =
+ buildConf("spark.sql.execution.arrow.pyspark.fallback.enabled")
+ .doc(s"When true, optimizations enabled by '${ARROW_PYSPARK_EXECUTION_ENABLED.key}' will " +
+ "fallback automatically to non-optimized implementations if an error occurs.")
+ .version("3.0.0")
+ .fallbackConf(ARROW_FALLBACK_ENABLED)
+
+ val ARROW_EXECUTION_MAX_RECORDS_PER_BATCH =
+ buildConf("spark.sql.execution.arrow.maxRecordsPerBatch")
+ .doc("When using Apache Arrow, limit the maximum number of records that can be written " +
+ "to a single ArrowRecordBatch in memory. If set to zero or negative there is no limit.")
+ .version("2.3.0")
+ .intConf
+ .createWithDefault(10000)
+
+ val PANDAS_UDF_BUFFER_SIZE =
+ buildConf("spark.sql.execution.pandas.udf.buffer.size")
+ .doc(
+ s"Same as `${BUFFER_SIZE.key}` but only applies to Pandas UDF executions. If it is not " +
+ s"set, the fallback is `${BUFFER_SIZE.key}`. Note that Pandas execution requires more " +
+ "than 4 bytes. Lowering this value could make small Pandas UDF batch iterated and " +
+ "pipelined; however, it might degrade performance. See SPARK-27870.")
+ .version("3.0.0")
+ .fallbackConf(BUFFER_SIZE)
+
+ val PYSPARK_SIMPLIFIEID_TRACEBACK =
+ buildConf("spark.sql.execution.pyspark.udf.simplifiedTraceback.enabled")
+ .doc(
+ "When true, the traceback from Python UDFs is simplified. It hides " +
+ "the Python worker, (de)serialization, etc from PySpark in tracebacks, and only " +
+ "shows the exception messages from UDFs. Note that this works only with CPython 3.7+.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME =
+ buildConf("spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName")
+ .internal()
+ .doc("When true, columns will be looked up by name if labeled with a string and fallback " +
+ "to use position if not. When false, a grouped map Pandas UDF will assign columns from " +
+ "the returned Pandas DataFrame based on position, regardless of column label type. " +
+ "This configuration will be deprecated in future releases.")
+ .version("2.4.1")
+ .booleanConf
+ .createWithDefault(true)
+
+ val PANDAS_ARROW_SAFE_TYPE_CONVERSION =
+ buildConf("spark.sql.execution.pandas.convertToArrowArraySafely")
+ .internal()
+ .doc("When true, Arrow will perform safe type conversion when converting " +
+ "Pandas.Series to Arrow array during serialization. Arrow will raise errors " +
+ "when detecting unsafe type conversion like overflow. When false, disabling Arrow's type " +
+ "check and do type conversions anyway. This config only works for Arrow 0.11.0+.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val REPLACE_EXCEPT_WITH_FILTER = buildConf("spark.sql.optimizer.replaceExceptWithFilter")
+ .internal()
+ .doc("When true, the apply function of the rule verifies whether the right node of the" +
+ " except operation is of type Filter or Project followed by Filter. If yes, the rule" +
+ " further verifies 1) Excluding the filter operations from the right (as well as the" +
+ " left node, if any) on the top, whether both the nodes evaluates to a same result." +
+ " 2) The left and right nodes don't contain any SubqueryExpressions. 3) The output" +
+ " column names of the left node are distinct. If all the conditions are met, the" +
+ " rule will replace the except operation with a Filter by flipping the filter" +
+ " condition(s) of the right node.")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val DECIMAL_OPERATIONS_ALLOW_PREC_LOSS =
+ buildConf("spark.sql.decimalOperations.allowPrecisionLoss")
+ .internal()
+ .doc("When true (default), establishing the result type of an arithmetic operation " +
+ "happens according to Hive behavior and SQL ANSI 2011 specification, i.e. rounding the " +
+ "decimal part of the result if an exact representation is not possible. Otherwise, NULL " +
+ "is returned in those cases, as previously.")
+ .version("2.3.1")
+ .booleanConf
+ .createWithDefault(true)
+
+ val LITERAL_PICK_MINIMUM_PRECISION =
+ buildConf("spark.sql.legacy.literal.pickMinimumPrecision")
+ .internal()
+ .doc("When integral literal is used in decimal operations, pick a minimum precision " +
+ "required by the literal if this config is true, to make the resulting precision and/or " +
+ "scale smaller. This can reduce the possibility of precision lose and/or overflow.")
+ .version("2.3.3")
+ .booleanConf
+ .createWithDefault(true)
+
+ val SQL_OPTIONS_REDACTION_PATTERN = buildConf("spark.sql.redaction.options.regex")
+ .doc("Regex to decide which keys in a Spark SQL command's options map contain sensitive " +
+ "information. The values of options whose names that match this regex will be redacted " +
+ "in the explain output. This redaction is applied on top of the global redaction " +
+ s"configuration defined by ${SECRET_REDACTION_PATTERN.key}.")
+ .version("2.2.2")
+ .regexConf
+ .createWithDefault("(?i)url".r)
+
+ val SQL_STRING_REDACTION_PATTERN =
+ buildConf("spark.sql.redaction.string.regex")
+ .doc("Regex to decide which parts of strings produced by Spark contain sensitive " +
+ "information. When this regex matches a string part, that string part is replaced by a " +
+ "dummy value. This is currently used to redact the output of SQL explain commands. " +
+ "When this conf is not set, the value from `spark.redaction.string.regex` is used.")
+ .version("2.3.0")
+ .fallbackConf(org.apache.spark.internal.config.STRING_REDACTION_PATTERN)
+
+ val CONCAT_BINARY_AS_STRING = buildConf("spark.sql.function.concatBinaryAsString")
+ .doc("When this option is set to false and all inputs are binary, `functions.concat` returns " +
+ "an output as binary. Otherwise, it returns as a string.")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val ELT_OUTPUT_AS_STRING = buildConf("spark.sql.function.eltOutputAsString")
+ .doc("When this option is set to false and all inputs are binary, `elt` returns " +
+ "an output as binary. Otherwise, it returns as a string.")
+ .version("2.3.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val VALIDATE_PARTITION_COLUMNS =
+ buildConf("spark.sql.sources.validatePartitionColumns")
+ .internal()
+ .doc("When this option is set to true, partition column values will be validated with " +
+ "user-specified schema. If the validation fails, a runtime exception is thrown. " +
+ "When this option is set to false, the partition column value will be converted to null " +
+ "if it can not be casted to corresponding user-specified schema.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE =
+ buildConf("spark.sql.streaming.continuous.epochBacklogQueueSize")
+ .doc("The max number of entries to be stored in queue to wait for late epochs. " +
+ "If this parameter is exceeded by the size of the queue, stream will stop with an error.")
+ .version("3.0.0")
+ .intConf
+ .createWithDefault(10000)
+
+ val CONTINUOUS_STREAMING_EXECUTOR_QUEUE_SIZE =
+ buildConf("spark.sql.streaming.continuous.executorQueueSize")
+ .internal()
+ .doc("The size (measured in number of rows) of the queue used in continuous execution to" +
+ " buffer the results of a ContinuousDataReader.")
+ .version("2.3.0")
+ .intConf
+ .createWithDefault(1024)
+
+ val CONTINUOUS_STREAMING_EXECUTOR_POLL_INTERVAL_MS =
+ buildConf("spark.sql.streaming.continuous.executorPollIntervalMs")
+ .internal()
+ .doc("The interval at which continuous execution readers will poll to check whether" +
+ " the epoch has advanced on the driver.")
+ .version("2.3.0")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefault(100)
+
+ val USE_V1_SOURCE_LIST = buildConf("spark.sql.sources.useV1SourceList")
+ .internal()
+ .doc("A comma-separated list of data source short names or fully qualified data source " +
+ "implementation class names for which Data Source V2 code path is disabled. These data " +
+ "sources will fallback to Data Source V1 code path.")
+ .version("3.0.0")
+ .stringConf
+ .createWithDefault("avro,csv,json,kafka,orc,parquet,text")
+
+ val DISABLED_V2_STREAMING_WRITERS = buildConf("spark.sql.streaming.disabledV2Writers")
+ .doc("A comma-separated list of fully qualified data source register class names for which" +
+ " StreamWriteSupport is disabled. Writes to these sources will fall back to the V1 Sinks.")
+ .version("2.3.1")
+ .stringConf
+ .createWithDefault("")
+
+ val DISABLED_V2_STREAMING_MICROBATCH_READERS =
+ buildConf("spark.sql.streaming.disabledV2MicroBatchReaders")
+ .internal()
+ .doc(
+ "A comma-separated list of fully qualified data source register class names for which " +
+ "MicroBatchReadSupport is disabled. Reads from these sources will fall back to the " +
+ "V1 Sources.")
+ .version("2.4.0")
+ .stringConf
+ .createWithDefault("")
+
+ val FASTFAIL_ON_FILEFORMAT_OUTPUT =
+ buildConf("spark.sql.execution.fastFailOnFileFormatOutput")
+ .internal()
+ .doc("Whether to fast fail task execution when writing output to FileFormat datasource. " +
+ "If this is enabled, in `FileFormatWriter` we will catch `FileAlreadyExistsException` " +
+ "and fast fail output task without further task retry. Only enabling this if you know " +
+ "the `FileAlreadyExistsException` of the output task is unrecoverable, i.e., further " +
+ "task attempts won't be able to success. If the `FileAlreadyExistsException` might be " +
+ "recoverable, you should keep this as disabled and let Spark to retry output tasks. " +
+ "This is disabled by default.")
+ .version("3.0.2")
+ .booleanConf
+ .createWithDefault(false)
+
+ object PartitionOverwriteMode extends Enumeration {
+ val STATIC, DYNAMIC = Value
+ }
+
+ val PARTITION_OVERWRITE_MODE =
+ buildConf("spark.sql.sources.partitionOverwriteMode")
+ .doc("When INSERT OVERWRITE a partitioned data source table, we currently support 2 modes: " +
+ "static and dynamic. In static mode, Spark deletes all the partitions that match the " +
+ "partition specification(e.g. PARTITION(a=1,b)) in the INSERT statement, before " +
+ "overwriting. In dynamic mode, Spark doesn't delete partitions ahead, and only overwrite " +
+ "those partitions that have data written into it at runtime. By default we use static " +
+ "mode to keep the same behavior of Spark prior to 2.3. Note that this config doesn't " +
+ "affect Hive serde tables, as they are always overwritten with dynamic mode. This can " +
+ "also be set as an output option for a data source using key partitionOverwriteMode " +
+ "(which takes precedence over this setting), e.g. " +
+ "dataframe.write.option(\"partitionOverwriteMode\", \"dynamic\").save(path)."
+ )
+ .version("2.3.0")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(PartitionOverwriteMode.values.map(_.toString))
+ .createWithDefault(PartitionOverwriteMode.STATIC.toString)
+
+ object StoreAssignmentPolicy extends Enumeration {
+ val ANSI, LEGACY, STRICT = Value
+ }
+
+ val STORE_ASSIGNMENT_POLICY =
+ buildConf("spark.sql.storeAssignmentPolicy")
+ .doc("When inserting a value into a column with different data type, Spark will perform " +
+ "type coercion. Currently, we support 3 policies for the type coercion rules: ANSI, " +
+ "legacy and strict. With ANSI policy, Spark performs the type coercion as per ANSI SQL. " +
+ "In practice, the behavior is mostly the same as PostgreSQL. " +
+ "It disallows certain unreasonable type conversions such as converting " +
+ "`string` to `int` or `double` to `boolean`. " +
+ "With legacy policy, Spark allows the type coercion as long as it is a valid `Cast`, " +
+ "which is very loose. e.g. converting `string` to `int` or `double` to `boolean` is " +
+ "allowed. It is also the only behavior in Spark 2.x and it is compatible with Hive. " +
+ "With strict policy, Spark doesn't allow any possible precision loss or data truncation " +
+ "in type coercion, e.g. converting `double` to `int` or `decimal` to `double` is " +
+ "not allowed."
+ )
+ .version("3.0.0")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(StoreAssignmentPolicy.values.map(_.toString))
+ .createWithDefault(StoreAssignmentPolicy.ANSI.toString)
+
+ val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled")
+ .doc("When true, Spark SQL uses an ANSI compliant dialect instead of being Hive compliant. " +
+ "For example, Spark will throw an exception at runtime instead of returning null results " +
+ "when the inputs to a SQL operator/function are invalid." +
+ "For full details of this dialect, you can find them in the section \"ANSI Compliance\" of " +
+ "Spark's documentation. Some ANSI dialect features may be not from the ANSI SQL " +
+ "standard directly, but their behaviors align with ANSI SQL's style")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val SORT_BEFORE_REPARTITION =
+ buildConf("spark.sql.execution.sortBeforeRepartition")
+ .internal()
+ .doc("When perform a repartition following a shuffle, the output row ordering would be " +
+ "nondeterministic. If some downstream stages fail and some tasks of the repartition " +
+ "stage retry, these tasks may generate different data, and that can lead to correctness " +
+ "issues. Turn on this config to insert a local sort before actually doing repartition " +
+ "to generate consistent repartition results. The performance of repartition() may go " +
+ "down since we insert extra local sort before it.")
+ .version("2.1.4")
+ .booleanConf
+ .createWithDefault(true)
+
+ val NESTED_SCHEMA_PRUNING_ENABLED =
+ buildConf("spark.sql.optimizer.nestedSchemaPruning.enabled")
+ .internal()
+ .doc("Prune nested fields from a logical relation's output which are unnecessary in " +
+ "satisfying a query. This optimization allows columnar file format readers to avoid " +
+ "reading unnecessary nested column data. Currently Parquet and ORC are the " +
+ "data sources that implement this optimization.")
+ .version("2.4.1")
+ .booleanConf
+ .createWithDefault(true)
+
+ val DISABLE_HINTS =
+ buildConf("spark.sql.optimizer.disableHints")
+ .internal()
+ .doc("When true, the optimizer will disable user-specified hints that are additional " +
+ "directives for better planning of a query.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val NESTED_PREDICATE_PUSHDOWN_FILE_SOURCE_LIST =
+ buildConf("spark.sql.optimizer.nestedPredicatePushdown.supportedFileSources")
+ .internal()
+ .doc("A comma-separated list of data source short names or fully qualified data source " +
+ "implementation class names for which Spark tries to push down predicates for nested " +
+ "columns and/or names containing `dots` to data sources. This configuration is only " +
+ "effective with file-based data sources in DSv1. Currently, Parquet and ORC implement " +
+ "both optimizations. The other data sources don't support this feature yet. So the " +
+ "default value is 'parquet,orc'.")
+ .version("3.0.0")
+ .stringConf
+ .createWithDefault("parquet,orc")
+
+ val SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED =
+ buildConf("spark.sql.optimizer.serializer.nestedSchemaPruning.enabled")
+ .internal()
+ .doc("Prune nested fields from object serialization operator which are unnecessary in " +
+ "satisfying a query. This optimization allows object serializers to avoid " +
+ "executing unnecessary nested expressions.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val NESTED_PRUNING_ON_EXPRESSIONS =
+ buildConf("spark.sql.optimizer.expression.nestedPruning.enabled")
+ .internal()
+ .doc("Prune nested fields from expressions in an operator which are unnecessary in " +
+ "satisfying a query. Note that this optimization doesn't prune nested fields from " +
+ "physical data source scanning. For pruning nested fields from scanning, please use " +
+ "`spark.sql.optimizer.nestedSchemaPruning.enabled` config.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val DECORRELATE_INNER_QUERY_ENABLED =
+ buildConf("spark.sql.optimizer.decorrelateInnerQuery.enabled")
+ .internal()
+ .doc("Decorrelate inner query by eliminating correlated references and build domain joins.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val OPTIMIZE_ONE_ROW_RELATION_SUBQUERY =
+ buildConf("spark.sql.optimizer.optimizeOneRowRelationSubquery")
+ .internal()
+ .doc("When true, the optimizer will inline subqueries with OneRowRelation as leaf nodes.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val TOP_K_SORT_FALLBACK_THRESHOLD =
+ buildConf("spark.sql.execution.topKSortFallbackThreshold")
+ .doc("In SQL queries with a SORT followed by a LIMIT like " +
+ "'SELECT x FROM t ORDER BY y LIMIT m', if m is under this threshold, do a top-K sort" +
+ " in memory, otherwise do a global sort which spills to disk if necessary.")
+ .version("2.4.0")
+ .intConf
+ .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH)
+
+ object Deprecated {
+ val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
+ }
+
+ object Replaced {
+ val MAPREDUCE_JOB_REDUCES = "mapreduce.job.reduces"
+ }
+
+ val CSV_PARSER_COLUMN_PRUNING = buildConf("spark.sql.csv.parser.columnPruning.enabled")
+ .internal()
+ .doc("If it is set to true, column names of the requested schema are passed to CSV parser. " +
+ "Other column values can be ignored during parsing even if they are malformed.")
+ .version("2.4.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val CSV_INPUT_BUFFER_SIZE = buildConf("spark.sql.csv.parser.inputBufferSize")
+ .internal()
+ .doc("If it is set, it configures the buffer size of CSV input during parsing. " +
+ "It is the same as inputBufferSize option in CSV which has a higher priority. " +
+ "Note that this is a workaround for the parsing library's regression, and this " +
+ "configuration is internal and supposed to be removed in the near future.")
+ .version("3.0.3")
+ .intConf
+ .createOptional
+
+ val REPL_EAGER_EVAL_ENABLED = buildConf("spark.sql.repl.eagerEval.enabled")
+ .doc("Enables eager evaluation or not. When true, the top K rows of Dataset will be " +
+ "displayed if and only if the REPL supports the eager evaluation. Currently, the " +
+ "eager evaluation is supported in PySpark and SparkR. In PySpark, for the notebooks like " +
+ "Jupyter, the HTML table (generated by _repr_html_) will be returned. For plain Python " +
+ "REPL, the returned outputs are formatted like dataframe.show(). In SparkR, the returned " +
+ "outputs are showed similar to R data.frame would.")
+ .version("2.4.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val REPL_EAGER_EVAL_MAX_NUM_ROWS = buildConf("spark.sql.repl.eagerEval.maxNumRows")
+ .doc("The max number of rows that are returned by eager evaluation. This only takes " +
+ s"effect when ${REPL_EAGER_EVAL_ENABLED.key} is set to true. The valid range of this " +
+ "config is from 0 to (Int.MaxValue - 1), so the invalid config like negative and " +
+ "greater than (Int.MaxValue - 1) will be normalized to 0 and (Int.MaxValue - 1).")
+ .version("2.4.0")
+ .intConf
+ .createWithDefault(20)
+
+ val REPL_EAGER_EVAL_TRUNCATE = buildConf("spark.sql.repl.eagerEval.truncate")
+ .doc("The max number of characters for each cell that is returned by eager evaluation. " +
+ s"This only takes effect when ${REPL_EAGER_EVAL_ENABLED.key} is set to true.")
+ .version("2.4.0")
+ .intConf
+ .createWithDefault(20)
+
+ val FAST_HASH_AGGREGATE_MAX_ROWS_CAPACITY_BIT =
+ buildConf("spark.sql.codegen.aggregate.fastHashMap.capacityBit")
+ .internal()
+ .doc("Capacity for the max number of rows to be held in memory " +
+ "by the fast hash aggregate product operator. The bit is not for actual value, " +
+ "but the actual numBuckets is determined by loadFactor " +
+ "(e.g: default bit value 16 , the actual numBuckets is ((1 << 16) / 0.5).")
+ .version("2.4.0")
+ .intConf
+ .checkValue(bit => bit >= 10 && bit <= 30, "The bit value must be in [10, 30].")
+ .createWithDefault(16)
+
+ val AVRO_COMPRESSION_CODEC = buildConf("spark.sql.avro.compression.codec")
+ .doc("Compression codec used in writing of AVRO files. Supported codecs: " +
+ "uncompressed, deflate, snappy, bzip2, xz and zstandard. Default codec is snappy.")
+ .version("2.4.0")
+ .stringConf
+ .checkValues(Set("uncompressed", "deflate", "snappy", "bzip2", "xz", "zstandard"))
+ .createWithDefault("snappy")
+
+ val AVRO_DEFLATE_LEVEL = buildConf("spark.sql.avro.deflate.level")
+ .doc("Compression level for the deflate codec used in writing of AVRO files. " +
+ "Valid value must be in the range of from 1 to 9 inclusive or -1. " +
+ "The default value is -1 which corresponds to 6 level in the current implementation.")
+ .version("2.4.0")
+ .intConf
+ .checkValues((1 to 9).toSet + Deflater.DEFAULT_COMPRESSION)
+ .createWithDefault(Deflater.DEFAULT_COMPRESSION)
+
+ val LEGACY_SIZE_OF_NULL = buildConf("spark.sql.legacy.sizeOfNull")
+ .internal()
+ .doc(s"If it is set to false, or ${ANSI_ENABLED.key} is true, then size of null returns " +
+ "null. Otherwise, it returns -1, which was inherited from Hive.")
+ .version("2.4.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val LEGACY_PARSE_NULL_PARTITION_SPEC_AS_STRING_LITERAL =
+ buildConf("spark.sql.legacy.parseNullPartitionSpecAsStringLiteral")
+ .internal()
+ .doc("If it is set to true, `PARTITION(col=null)` is parsed as a string literal of its " +
+ "text representation, e.g., string 'null', when the partition column is string type. " +
+ "Otherwise, it is always parsed as a null literal in the partition spec.")
+ .version("3.0.2")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_REPLACE_DATABRICKS_SPARK_AVRO_ENABLED =
+ buildConf("spark.sql.legacy.replaceDatabricksSparkAvro.enabled")
+ .internal()
+ .doc("If it is set to true, the data source provider com.databricks.spark.avro is mapped " +
+ "to the built-in but external Avro data source module for backward compatibility.")
+ .version("2.4.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val LEGACY_SETOPS_PRECEDENCE_ENABLED =
+ buildConf("spark.sql.legacy.setopsPrecedence.enabled")
+ .internal()
+ .doc("When set to true and the order of evaluation is not specified by parentheses, the " +
+ "set operations are performed from left to right as they appear in the query. When set " +
+ "to false and order of evaluation is not specified by parentheses, INTERSECT operations " +
+ "are performed before any UNION, EXCEPT and MINUS operations.")
+ .version("2.4.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_EXPONENT_LITERAL_AS_DECIMAL_ENABLED =
+ buildConf("spark.sql.legacy.exponentLiteralAsDecimal.enabled")
+ .internal()
+ .doc("When set to true, a literal with an exponent (e.g. 1E-30) would be parsed " +
+ "as Decimal rather than Double.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED =
+ buildConf("spark.sql.legacy.allowNegativeScaleOfDecimal")
+ .internal()
+ .doc("When set to true, negative scale of Decimal type is allowed. For example, " +
+ "the type of number 1E10BD under legacy mode is DecimalType(2, -9), but is " +
+ "Decimal(11, 0) in non legacy mode.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_BUCKETED_TABLE_SCAN_OUTPUT_ORDERING =
+ buildConf("spark.sql.legacy.bucketedTableScan.outputOrdering")
+ .internal()
+ .doc("When true, the bucketed table scan will list files during planning to figure out the " +
+ "output ordering, which is expensive and may make the planning quite slow.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE =
+ buildConf("spark.sql.legacy.parser.havingWithoutGroupByAsWhere")
+ .internal()
+ .doc("If it is set to true, the parser will treat HAVING without GROUP BY as a normal " +
+ "WHERE, which does not follow SQL standard.")
+ .version("2.4.1")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_ALLOW_EMPTY_STRING_IN_JSON =
+ buildConf("spark.sql.legacy.json.allowEmptyString.enabled")
+ .internal()
+ .doc("When set to true, the parser of JSON data source treats empty strings as null for " +
+ "some data types such as `IntegerType`.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_CREATE_EMPTY_COLLECTION_USING_STRING_TYPE =
+ buildConf("spark.sql.legacy.createEmptyCollectionUsingStringType")
+ .internal()
+ .doc("When set to true, Spark returns an empty collection with `StringType` as element " +
+ "type if the `array`/`map` function is called without any parameters. Otherwise, Spark " +
+ "returns an empty collection with `NullType` as element type.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_ALLOW_UNTYPED_SCALA_UDF =
+ buildConf("spark.sql.legacy.allowUntypedScalaUDF")
+ .internal()
+ .doc("When set to true, user is allowed to use org.apache.spark.sql.functions." +
+ "udf(f: AnyRef, dataType: DataType). Otherwise, an exception will be thrown at runtime.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_STATISTICAL_AGGREGATE =
+ buildConf("spark.sql.legacy.statisticalAggregate")
+ .internal()
+ .doc("When set to true, statistical aggregate function returns Double.NaN " +
+ "if divide by zero occurred during expression evaluation, otherwise, it returns null. " +
+ "Before version 3.1.0, it returns NaN in divideByZero case by default.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val TRUNCATE_TABLE_IGNORE_PERMISSION_ACL =
+ buildConf("spark.sql.truncateTable.ignorePermissionAcl.enabled")
+ .internal()
+ .doc("When set to true, TRUNCATE TABLE command will not try to set back original " +
+ "permission and ACLs when re-creating the table/partition paths.")
+ .version("2.4.6")
+ .booleanConf
+ .createWithDefault(false)
+
+ val NAME_NON_STRUCT_GROUPING_KEY_AS_VALUE =
+ buildConf("spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue")
+ .internal()
+ .doc("When set to true, the key attribute resulted from running `Dataset.groupByKey` " +
+ "for non-struct key type, will be named as `value`, following the behavior of Spark " +
+ "version 2.4 and earlier.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val MAX_TO_STRING_FIELDS = buildConf("spark.sql.debug.maxToStringFields")
+ .doc("Maximum number of fields of sequence-like entries can be converted to strings " +
+ "in debug output. Any elements beyond the limit will be dropped and replaced by a" +
+ """ "... N more fields" placeholder.""")
+ .version("3.0.0")
+ .intConf
+ .createWithDefault(25)
+
+ val MAX_PLAN_STRING_LENGTH = buildConf("spark.sql.maxPlanStringLength")
+ .doc("Maximum number of characters to output for a plan string. If the plan is " +
+ "longer, further output will be truncated. The default setting always generates a full " +
+ "plan. Set this to a lower value such as 8k if plan strings are taking up too much " +
+ "memory or are causing OutOfMemory errors in the driver or UI processes.")
+ .version("3.0.0")
+ .bytesConf(ByteUnit.BYTE)
+ .checkValue(i => i >= 0 && i <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, "Invalid " +
+ "value for 'spark.sql.maxPlanStringLength'. Length must be a valid string length " +
+ "(nonnegative and shorter than the maximum size).")
+ .createWithDefaultString(s"${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}")
+
+ val MAX_METADATA_STRING_LENGTH = buildConf("spark.sql.maxMetadataStringLength")
+ .doc("Maximum number of characters to output for a metadata string. e.g. " +
+ "file location in `DataSourceScanExec`, every value will be abbreviated if exceed length.")
+ .version("3.1.0")
+ .intConf
+ .checkValue(_ > 3, "This value must be bigger than 3.")
+ .createWithDefault(100)
+
+ val SET_COMMAND_REJECTS_SPARK_CORE_CONFS =
+ buildConf("spark.sql.legacy.setCommandRejectsSparkCoreConfs")
+ .internal()
+ .doc("If it is set to true, SET command will fail when the key is registered as " +
+ "a SparkConf entry.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ object TimestampTypes extends Enumeration {
+ val TIMESTAMP_NTZ, TIMESTAMP_LTZ = Value
+ }
+
+ val TIMESTAMP_TYPE =
+ buildConf("spark.sql.timestampType")
+ .doc("Configures the default timestamp type of Spark SQL, including SQL DDL, Cast clause " +
+ s"and type literal. Setting the configuration as ${TimestampTypes.TIMESTAMP_NTZ} will " +
+ "use TIMESTAMP WITHOUT TIME ZONE as the default type while putting it as " +
+ s"${TimestampTypes.TIMESTAMP_LTZ} will use TIMESTAMP WITH LOCAL TIME ZONE. " +
+ "Before the 3.3.0 release, Spark only supports the TIMESTAMP WITH " +
+ "LOCAL TIME ZONE type.")
+ .version("3.3.0")
+ .internal()
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(TimestampTypes.values.map(_.toString))
+ .createWithDefault(TimestampTypes.TIMESTAMP_LTZ.toString)
+
+ val DATETIME_JAVA8API_ENABLED = buildConf("spark.sql.datetime.java8API.enabled")
+ .doc("If the configuration property is set to true, java.time.Instant and " +
+ "java.time.LocalDate classes of Java 8 API are used as external types for " +
+ "Catalyst's TimestampType and DateType. If it is set to false, java.sql.Timestamp " +
+ "and java.sql.Date are used for the same purpose.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val UI_EXPLAIN_MODE = buildConf("spark.sql.ui.explainMode")
+ .doc("Configures the query explain mode used in the Spark SQL UI. The value can be 'simple', " +
+ "'extended', 'codegen', 'cost', or 'formatted'. The default value is 'formatted'.")
+ .version("3.1.0")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValue(mode => Set("SIMPLE", "EXTENDED", "CODEGEN", "COST", "FORMATTED").contains(mode),
+ "Invalid value for 'spark.sql.ui.explainMode'. Valid values are 'simple', 'extended', " +
+ "'codegen', 'cost' and 'formatted'.")
+ .createWithDefault("formatted")
+
+ val SOURCES_BINARY_FILE_MAX_LENGTH = buildConf("spark.sql.sources.binaryFile.maxLength")
+ .doc("The max length of a file that can be read by the binary file data source. " +
+ "Spark will fail fast and not attempt to read the file if its length exceeds this value. " +
+ "The theoretical max is Int.MaxValue, though VMs might implement a smaller max.")
+ .version("3.0.0")
+ .internal()
+ .intConf
+ .createWithDefault(Int.MaxValue)
+
+ val LEGACY_CAST_DATETIME_TO_STRING =
+ buildConf("spark.sql.legacy.typeCoercion.datetimeToString.enabled")
+ .internal()
+ .doc("If it is set to true, date/timestamp will cast to string in binary comparisons " +
+ s"with String when ${ANSI_ENABLED.key} is false.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val DEFAULT_CATALOG = buildConf("spark.sql.defaultCatalog")
+ .doc("Name of the default catalog. This will be the current catalog if users have not " +
+ "explicitly set the current catalog yet.")
+ .version("3.0.0")
+ .stringConf
+ .createWithDefault(SESSION_CATALOG_NAME)
+
+ val V2_SESSION_CATALOG_IMPLEMENTATION =
+ buildConf(s"spark.sql.catalog.$SESSION_CATALOG_NAME")
+ .doc("A catalog implementation that will be used as the v2 interface to Spark's built-in " +
+ s"v1 catalog: $SESSION_CATALOG_NAME. This catalog shares its identifier namespace with " +
+ s"the $SESSION_CATALOG_NAME and must be consistent with it; for example, if a table can " +
+ s"be loaded by the $SESSION_CATALOG_NAME, this catalog must also return the table " +
+ s"metadata. To delegate operations to the $SESSION_CATALOG_NAME, implementations can " +
+ "extend 'CatalogExtension'.")
+ .version("3.0.0")
+ .stringConf
+ .createOptional
+
+ object MapKeyDedupPolicy extends Enumeration {
+ val EXCEPTION, LAST_WIN = Value
+ }
+
+ val MAP_KEY_DEDUP_POLICY = buildConf("spark.sql.mapKeyDedupPolicy")
+ .doc("The policy to deduplicate map keys in builtin function: CreateMap, MapFromArrays, " +
+ "MapFromEntries, StringToMap, MapConcat and TransformKeys. When EXCEPTION, the query " +
+ "fails if duplicated map keys are detected. When LAST_WIN, the map key that is inserted " +
+ "at last takes precedence.")
+ .version("3.0.0")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(MapKeyDedupPolicy.values.map(_.toString))
+ .createWithDefault(MapKeyDedupPolicy.EXCEPTION.toString)
+
+ val LEGACY_LOOSE_UPCAST = buildConf("spark.sql.legacy.doLooseUpcast")
+ .internal()
+ .doc("When true, the upcast will be loose and allows string to atomic types.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ object LegacyBehaviorPolicy extends Enumeration {
+ val EXCEPTION, LEGACY, CORRECTED = Value
+ }
+
+ val LEGACY_CTE_PRECEDENCE_POLICY = buildConf("spark.sql.legacy.ctePrecedencePolicy")
+ .internal()
+ .doc("When LEGACY, outer CTE definitions takes precedence over inner definitions. If set to " +
+ "CORRECTED, inner CTE definitions take precedence. The default value is EXCEPTION, " +
+ "AnalysisException is thrown while name conflict is detected in nested CTE. This config " +
+ "will be removed in future versions and CORRECTED will be the only behavior.")
+ .version("3.0.0")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(LegacyBehaviorPolicy.values.map(_.toString))
+ .createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
+
+ val LEGACY_TIME_PARSER_POLICY = buildConf("spark.sql.legacy.timeParserPolicy")
+ .internal()
+ .doc("When LEGACY, java.text.SimpleDateFormat is used for formatting and parsing " +
+ "dates/timestamps in a locale-sensitive manner, which is the approach before Spark 3.0. " +
+ "When set to CORRECTED, classes from java.time.* packages are used for the same purpose. " +
+ "The default value is EXCEPTION, RuntimeException is thrown when we will get different " +
+ "results.")
+ .version("3.0.0")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(LegacyBehaviorPolicy.values.map(_.toString))
+ .createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
+
+ val LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC =
+ buildConf("spark.sql.legacy.followThreeValuedLogicInArrayExists")
+ .internal()
+ .doc("When true, the ArrayExists will follow the three-valued boolean logic.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val ADDITIONAL_REMOTE_REPOSITORIES =
+ buildConf("spark.sql.maven.additionalRemoteRepositories")
+ .doc("A comma-delimited string config of the optional additional remote Maven mirror " +
+ "repositories. This is only used for downloading Hive jars in IsolatedClientLoader " +
+ "if the default Maven Central repo is unreachable.")
+ .version("3.0.0")
+ .stringConf
+ .createWithDefault(
+ sys.env.getOrElse("DEFAULT_ARTIFACT_REPOSITORY",
+ "https://maven-central.storage-download.googleapis.com/maven2/"))
+
+ val LEGACY_FROM_DAYTIME_STRING =
+ buildConf("spark.sql.legacy.fromDayTimeString.enabled")
+ .internal()
+ .doc("When true, the `from` bound is not taken into account in conversion of " +
+ "a day-time string to an interval, and the `to` bound is used to skip " +
+ "all interval units out of the specified range. If it is set to `false`, " +
+ "`ParseException` is thrown if the input does not match to the pattern " +
+ "defined by `from` and `to`.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_PROPERTY_NON_RESERVED =
+ buildConf("spark.sql.legacy.notReserveProperties")
+ .internal()
+ .doc("When true, all database and table properties are not reserved and available for " +
+ "create/alter syntaxes. But please be aware that the reserved properties will be " +
+ "silently removed.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_ADD_SINGLE_FILE_IN_ADD_FILE =
+ buildConf("spark.sql.legacy.addSingleFileInAddFile")
+ .internal()
+ .doc("When true, only a single file can be added using ADD FILE. If false, then users " +
+ "can add directory by passing directory path to ADD FILE.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_MSSQLSERVER_NUMERIC_MAPPING_ENABLED =
+ buildConf("spark.sql.legacy.mssqlserver.numericMapping.enabled")
+ .internal()
+ .doc("When true, use legacy MySqlServer SMALLINT and REAL type mapping.")
+ .version("2.4.5")
+ .booleanConf
+ .createWithDefault(false)
+
+ val CSV_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.csv.filterPushdown.enabled")
+ .doc("When true, enable filter pushdown to CSV datasource.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val JSON_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.json.filterPushdown.enabled")
+ .doc("When true, enable filter pushdown to JSON datasource.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val AVRO_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.avro.filterPushdown.enabled")
+ .doc("When true, enable filter pushdown to Avro datasource.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val ADD_PARTITION_BATCH_SIZE =
+ buildConf("spark.sql.addPartitionInBatch.size")
+ .internal()
+ .doc("The number of partitions to be handled in one turn when use " +
+ "`AlterTableAddPartitionCommand` or `RepairTableCommand` to add partitions into table. " +
+ "The smaller batch size is, the less memory is required for the real handler, e.g. " +
+ "Hive Metastore.")
+ .version("3.0.0")
+ .intConf
+ .checkValue(_ > 0, "The value of spark.sql.addPartitionInBatch.size must be positive")
+ .createWithDefault(100)
+
+ val LEGACY_ALLOW_HASH_ON_MAPTYPE = buildConf("spark.sql.legacy.allowHashOnMapType")
+ .internal()
+ .doc("When set to true, hash expressions can be applied on elements of MapType. Otherwise, " +
+ "an analysis exception will be thrown.")
+ .version("3.0.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_INTEGER_GROUPING_ID =
+ buildConf("spark.sql.legacy.integerGroupingId")
+ .internal()
+ .doc("When true, grouping_id() returns int values instead of long values.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val PARQUET_INT96_REBASE_MODE_IN_WRITE =
+ buildConf("spark.sql.parquet.int96RebaseModeInWrite")
+ .internal()
+ .doc("When LEGACY, Spark will rebase INT96 timestamps from Proleptic Gregorian calendar to " +
+ "the legacy hybrid (Julian + Gregorian) calendar when writing Parquet files. " +
+ "When CORRECTED, Spark will not do rebase and write the timestamps as it is. " +
+ "When EXCEPTION, which is the default, Spark will fail the writing if it sees ancient " +
+ "timestamps that are ambiguous between the two calendars.")
+ .version("3.1.0")
+ .withAlternative("spark.sql.legacy.parquet.int96RebaseModeInWrite")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(LegacyBehaviorPolicy.values.map(_.toString))
+ .createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
+
+ val PARQUET_REBASE_MODE_IN_WRITE =
+ buildConf("spark.sql.parquet.datetimeRebaseModeInWrite")
+ .internal()
+ .doc("When LEGACY, Spark will rebase dates/timestamps from Proleptic Gregorian calendar " +
+ "to the legacy hybrid (Julian + Gregorian) calendar when writing Parquet files. " +
+ "When CORRECTED, Spark will not do rebase and write the dates/timestamps as it is. " +
+ "When EXCEPTION, which is the default, Spark will fail the writing if it sees " +
+ "ancient dates/timestamps that are ambiguous between the two calendars. " +
+ "This config influences on writes of the following parquet logical types: DATE, " +
+ "TIMESTAMP_MILLIS, TIMESTAMP_MICROS. The INT96 type has the separate config: " +
+ s"${PARQUET_INT96_REBASE_MODE_IN_WRITE.key}.")
+ .version("3.0.0")
+ .withAlternative("spark.sql.legacy.parquet.datetimeRebaseModeInWrite")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(LegacyBehaviorPolicy.values.map(_.toString))
+ .createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
+
+ val PARQUET_INT96_REBASE_MODE_IN_READ =
+ buildConf("spark.sql.parquet.int96RebaseModeInRead")
+ .internal()
+ .doc("When LEGACY, Spark will rebase INT96 timestamps from the legacy hybrid (Julian + " +
+ "Gregorian) calendar to Proleptic Gregorian calendar when reading Parquet files. " +
+ "When CORRECTED, Spark will not do rebase and read the timestamps as it is. " +
+ "When EXCEPTION, which is the default, Spark will fail the reading if it sees ancient " +
+ "timestamps that are ambiguous between the two calendars. This config is only effective " +
+ "if the writer info (like Spark, Hive) of the Parquet files is unknown.")
+ .version("3.1.0")
+ .withAlternative("spark.sql.legacy.parquet.int96RebaseModeInRead")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(LegacyBehaviorPolicy.values.map(_.toString))
+ .createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
+
+ val PARQUET_REBASE_MODE_IN_READ =
+ buildConf("spark.sql.parquet.datetimeRebaseModeInRead")
+ .internal()
+ .doc("When LEGACY, Spark will rebase dates/timestamps from the legacy hybrid (Julian + " +
+ "Gregorian) calendar to Proleptic Gregorian calendar when reading Parquet files. " +
+ "When CORRECTED, Spark will not do rebase and read the dates/timestamps as it is. " +
+ "When EXCEPTION, which is the default, Spark will fail the reading if it sees " +
+ "ancient dates/timestamps that are ambiguous between the two calendars. This config is " +
+ "only effective if the writer info (like Spark, Hive) of the Parquet files is unknown. " +
+ "This config influences on reads of the following parquet logical types: DATE, " +
+ "TIMESTAMP_MILLIS, TIMESTAMP_MICROS. The INT96 type has the separate config: " +
+ s"${PARQUET_INT96_REBASE_MODE_IN_READ.key}.")
+ .version("3.0.0")
+ .withAlternative("spark.sql.legacy.parquet.datetimeRebaseModeInRead")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(LegacyBehaviorPolicy.values.map(_.toString))
+ .createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
+
+ val AVRO_REBASE_MODE_IN_WRITE =
+ buildConf("spark.sql.avro.datetimeRebaseModeInWrite")
+ .internal()
+ .doc("When LEGACY, Spark will rebase dates/timestamps from Proleptic Gregorian calendar " +
+ "to the legacy hybrid (Julian + Gregorian) calendar when writing Avro files. " +
+ "When CORRECTED, Spark will not do rebase and write the dates/timestamps as it is. " +
+ "When EXCEPTION, which is the default, Spark will fail the writing if it sees " +
+ "ancient dates/timestamps that are ambiguous between the two calendars.")
+ .version("3.0.0")
+ .withAlternative("spark.sql.legacy.avro.datetimeRebaseModeInWrite")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(LegacyBehaviorPolicy.values.map(_.toString))
+ .createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
+
+ val AVRO_REBASE_MODE_IN_READ =
+ buildConf("spark.sql.avro.datetimeRebaseModeInRead")
+ .internal()
+ .doc("When LEGACY, Spark will rebase dates/timestamps from the legacy hybrid (Julian + " +
+ "Gregorian) calendar to Proleptic Gregorian calendar when reading Avro files. " +
+ "When CORRECTED, Spark will not do rebase and read the dates/timestamps as it is. " +
+ "When EXCEPTION, which is the default, Spark will fail the reading if it sees " +
+ "ancient dates/timestamps that are ambiguous between the two calendars. This config is " +
+ "only effective if the writer info (like Spark, Hive) of the Avro files is unknown.")
+ .version("3.0.0")
+ .withAlternative("spark.sql.legacy.avro.datetimeRebaseModeInRead")
+ .stringConf
+ .transform(_.toUpperCase(Locale.ROOT))
+ .checkValues(LegacyBehaviorPolicy.values.map(_.toString))
+ .createWithDefault(LegacyBehaviorPolicy.EXCEPTION.toString)
+
+ val SCRIPT_TRANSFORMATION_EXIT_TIMEOUT =
+ buildConf("spark.sql.scriptTransformation.exitTimeoutInSeconds")
+ .internal()
+ .doc("Timeout for executor to wait for the termination of transformation script when EOF.")
+ .version("3.0.0")
+ .timeConf(TimeUnit.SECONDS)
+ .checkValue(_ > 0, "The timeout value must be positive")
+ .createWithDefault(10L)
+
+ val COALESCE_BUCKETS_IN_JOIN_ENABLED =
+ buildConf("spark.sql.bucketing.coalesceBucketsInJoin.enabled")
+ .doc("When true, if two bucketed tables with the different number of buckets are joined, " +
+ "the side with a bigger number of buckets will be coalesced to have the same number " +
+ "of buckets as the other side. Bigger number of buckets is divisible by the smaller " +
+ "number of buckets. Bucket coalescing is applied to sort-merge joins and " +
+ "shuffled hash join. Note: Coalescing bucketed table can avoid unnecessary shuffling " +
+ "in join, but it also reduces parallelism and could possibly cause OOM for " +
+ "shuffled hash join.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val COALESCE_BUCKETS_IN_JOIN_MAX_BUCKET_RATIO =
+ buildConf("spark.sql.bucketing.coalesceBucketsInJoin.maxBucketRatio")
+ .doc("The ratio of the number of two buckets being coalesced should be less than or " +
+ "equal to this value for bucket coalescing to be applied. This configuration only " +
+ s"has an effect when '${COALESCE_BUCKETS_IN_JOIN_ENABLED.key}' is set to true.")
+ .version("3.1.0")
+ .intConf
+ .checkValue(_ > 0, "The difference must be positive.")
+ .createWithDefault(4)
+
+ val BROADCAST_HASH_JOIN_OUTPUT_PARTITIONING_EXPAND_LIMIT =
+ buildConf("spark.sql.execution.broadcastHashJoin.outputPartitioningExpandLimit")
+ .internal()
+ .doc("The maximum number of partitionings that a HashPartitioning can be expanded to. " +
+ "This configuration is applicable only for BroadcastHashJoin inner joins and can be " +
+ "set to '0' to disable this feature.")
+ .version("3.1.0")
+ .intConf
+ .checkValue(_ >= 0, "The value must be non-negative.")
+ .createWithDefault(8)
+
+ val OPTIMIZE_NULL_AWARE_ANTI_JOIN =
+ buildConf("spark.sql.optimizeNullAwareAntiJoin")
+ .internal()
+ .doc("When true, NULL-aware anti join execution will be planed into " +
+ "BroadcastHashJoinExec with flag isNullAwareAntiJoin enabled, " +
+ "optimized from O(M*N) calculation into O(M) calculation " +
+ "using Hash lookup instead of Looping lookup." +
+ "Only support for singleColumn NAAJ for now.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val LEGACY_COMPLEX_TYPES_TO_STRING =
+ buildConf("spark.sql.legacy.castComplexTypesToString.enabled")
+ .internal()
+ .doc("When true, maps and structs are wrapped by [] in casting to strings, and " +
+ "NULL elements of structs/maps/arrays will be omitted while converting to strings. " +
+ "Otherwise, if this is false, which is the default, maps and structs are wrapped by {}, " +
+ "and NULL elements will be converted to \"null\".")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_PATH_OPTION_BEHAVIOR =
+ buildConf("spark.sql.legacy.pathOptionBehavior.enabled")
+ .internal()
+ .doc("When true, \"path\" option is overwritten if one path parameter is passed to " +
+ "DataFrameReader.load(), DataFrameWriter.save(), DataStreamReader.load(), or " +
+ "DataStreamWriter.start(). Also, \"path\" option is added to the overall paths if " +
+ "multiple path parameters are passed to DataFrameReader.load()")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_EXTRA_OPTIONS_BEHAVIOR =
+ buildConf("spark.sql.legacy.extraOptionsBehavior.enabled")
+ .internal()
+ .doc("When true, the extra options will be ignored for DataFrameReader.table(). If set it " +
+ "to false, which is the default, Spark will check if the extra options have the same " +
+ "key, but the value is different with the table serde properties. If the check passes, " +
+ "the extra options will be merged with the serde properties as the scan options. " +
+ "Otherwise, an exception will be thrown.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val DISABLED_JDBC_CONN_PROVIDER_LIST =
+ buildConf("spark.sql.sources.disabledJdbcConnProviderList")
+ .internal()
+ .doc("Configures a list of JDBC connection providers, which are disabled. " +
+ "The list contains the name of the JDBC connection providers separated by comma.")
+ .version("3.1.0")
+ .stringConf
+ .createWithDefault("")
+
+ val LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT =
+ buildConf("spark.sql.legacy.createHiveTableByDefault")
+ .internal()
+ .doc("When set to true, CREATE TABLE syntax without USING or STORED AS will use Hive " +
+ s"instead of the value of ${DEFAULT_DATA_SOURCE_NAME.key} as the table provider.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(true)
+
+ val LEGACY_CHAR_VARCHAR_AS_STRING =
+ buildConf("spark.sql.legacy.charVarcharAsString")
+ .internal()
+ .doc("When true, Spark will not fail if user uses char and varchar type directly in those" +
+ " APIs that accept or parse data types as parameters, e.g." +
+ " `SparkSession.read.schema(...)`, `SparkSession.udf.register(...)` but treat them as" +
+ " string type as Spark 3.0 and earlier.")
+ .version("3.1.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val CLI_PRINT_HEADER =
+ buildConf("spark.sql.cli.print.header")
+ .doc("When set to true, spark-sql CLI prints the names of the columns in query output.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_KEEP_COMMAND_OUTPUT_SCHEMA =
+ buildConf("spark.sql.legacy.keepCommandOutputSchema")
+ .internal()
+ .doc("When true, Spark will keep the output schema of commands such as SHOW DATABASES " +
+ "unchanged, for v1 catalog and/or table.")
+ .version("3.0.2")
+ .booleanConf
+ .createWithDefault(false)
+
+ val LEGACY_INTERVAL_ENABLED = buildConf("spark.sql.legacy.interval.enabled")
+ .internal()
+ .doc("When set to true, Spark SQL uses the mixed legacy interval type `CalendarIntervalType` " +
+ "instead of the ANSI compliant interval types `YearMonthIntervalType` and " +
+ "`DayTimeIntervalType`. For instance, the date subtraction expression returns " +
+ "`CalendarIntervalType` when the SQL config is set to `true` otherwise an ANSI interval.")
+ .version("3.2.0")
+ .booleanConf
+ .createWithDefault(false)
+
+ val MAX_CONCURRENT_OUTPUT_FILE_WRITERS = buildConf("spark.sql.maxConcurrentOutputFileWriters")
+ .internal()
+ .doc("Maximum number of output file writers to use concurrently. If number of writers " +
+ "needed reaches this limit, task will sort rest of output then writing them.")
+ .version("3.2.0")
+ .intConf
+ .createWithDefault(0)
+
+ /**
+ * Holds information about keys that have been deprecated.
+ *
+ * @param key The deprecated key.
+ * @param version Version of Spark where key was deprecated.
+ * @param comment Additional info regarding to the removed config. For example,
+ * reasons of config deprecation, what users should use instead of it.
+ */
+ case class DeprecatedConfig(key: String, version: String, comment: String)
+
+ /**
+ * Maps deprecated SQL config keys to information about the deprecation.
+ *
+ * The extra information is logged as a warning when the SQL config is present
+ * in the user's configuration.
+ */
+ val deprecatedSQLConfigs: Map[String, DeprecatedConfig] = {
+ val configs = Seq(
+ DeprecatedConfig(
+ PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME.key, "2.4",
+ "The config allows to switch to the behaviour before Spark 2.4 " +
+ "and will be removed in the future releases."),
+ DeprecatedConfig(HIVE_VERIFY_PARTITION_PATH.key, "3.0",
+ s"This config is replaced by '${SPARK_IGNORE_MISSING_FILES.key}'."),
+ DeprecatedConfig(ARROW_EXECUTION_ENABLED.key, "3.0",
+ s"Use '${ARROW_PYSPARK_EXECUTION_ENABLED.key}' instead of it."),
+ DeprecatedConfig(ARROW_FALLBACK_ENABLED.key, "3.0",
+ s"Use '${ARROW_PYSPARK_FALLBACK_ENABLED.key}' instead of it."),
+ DeprecatedConfig(SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "3.0",
+ s"Use '${ADVISORY_PARTITION_SIZE_IN_BYTES.key}' instead of it."),
+ DeprecatedConfig(OPTIMIZER_METADATA_ONLY.key, "3.0",
+ "Avoid to depend on this optimization to prevent a potential correctness issue. " +
+ "If you must use, use 'SparkSessionExtensions' instead to inject it as a custom rule."),
+ DeprecatedConfig(CONVERT_CTAS.key, "3.1",
+ s"Set '${LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT.key}' to false instead."),
+ DeprecatedConfig("spark.sql.sources.schemaStringLengthThreshold", "3.2",
+ s"Use '${HIVE_TABLE_PROPERTY_LENGTH_THRESHOLD.key}' instead."),
+ DeprecatedConfig(PARQUET_INT96_REBASE_MODE_IN_WRITE.alternatives.head, "3.2",
+ s"Use '${PARQUET_INT96_REBASE_MODE_IN_WRITE.key}' instead."),
+ DeprecatedConfig(PARQUET_INT96_REBASE_MODE_IN_READ.alternatives.head, "3.2",
+ s"Use '${PARQUET_INT96_REBASE_MODE_IN_READ.key}' instead."),
+ DeprecatedConfig(PARQUET_REBASE_MODE_IN_WRITE.alternatives.head, "3.2",
+ s"Use '${PARQUET_REBASE_MODE_IN_WRITE.key}' instead."),
+ DeprecatedConfig(PARQUET_REBASE_MODE_IN_READ.alternatives.head, "3.2",
+ s"Use '${PARQUET_REBASE_MODE_IN_READ.key}' instead."),
+ DeprecatedConfig(AVRO_REBASE_MODE_IN_WRITE.alternatives.head, "3.2",
+ s"Use '${AVRO_REBASE_MODE_IN_WRITE.key}' instead."),
+ DeprecatedConfig(AVRO_REBASE_MODE_IN_READ.alternatives.head, "3.2",
+ s"Use '${AVRO_REBASE_MODE_IN_READ.key}' instead."),
+ DeprecatedConfig(LEGACY_REPLACE_DATABRICKS_SPARK_AVRO_ENABLED.key, "3.2",
+ """Use `.format("avro")` in `DataFrameWriter` or `DataFrameReader` instead."""),
+ DeprecatedConfig(COALESCE_PARTITIONS_MIN_PARTITION_NUM.key, "3.2",
+ s"Use '${COALESCE_PARTITIONS_MIN_PARTITION_SIZE.key}' instead.")
+ )
+
+ Map(configs.map { cfg => cfg.key -> cfg } : _*)
+ }
+
+ /**
+ * Holds information about keys that have been removed.
+ *
+ * @param key The removed config key.
+ * @param version Version of Spark where key was removed.
+ * @param defaultValue The default config value. It can be used to notice
+ * users that they set non-default value to an already removed config.
+ * @param comment Additional info regarding to the removed config.
+ */
+ case class RemovedConfig(key: String, version: String, defaultValue: String, comment: String)
+
+ /**
+ * The map contains info about removed SQL configs. Keys are SQL config names,
+ * map values contain extra information like the version in which the config was removed,
+ * config's default value and a comment.
+ *
+ * Please, add a removed SQL configuration property here only when it affects behaviours.
+ * For example, `spark.sql.variable.substitute.depth` was not added as it virtually
+ * became no-op later. By this, it makes migrations to new Spark versions painless.
+ */
+ val removedSQLConfigs: Map[String, RemovedConfig] = {
+ val configs = Seq(
+ RemovedConfig("spark.sql.fromJsonForceNullableSchema", "3.0.0", "true",
+ "It was removed to prevent errors like SPARK-23173 for non-default value."),
+ RemovedConfig(
+ "spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation", "3.0.0", "false",
+ "It was removed to prevent loosing of users data for non-default value."),
+ RemovedConfig("spark.sql.legacy.compareDateTimestampInTimestamp", "3.0.0", "true",
+ "It was removed to prevent errors like SPARK-23549 for non-default value."),
+ RemovedConfig("spark.sql.parquet.int64AsTimestampMillis", "3.0.0", "false",
+ "The config was deprecated since Spark 2.3." +
+ s"Use '${PARQUET_OUTPUT_TIMESTAMP_TYPE.key}' instead of it."),
+ RemovedConfig("spark.sql.execution.pandas.respectSessionTimeZone", "3.0.0", "true",
+ "The non-default behavior is considered as a bug, see SPARK-22395. " +
+ "The config was deprecated since Spark 2.3."),
+ RemovedConfig("spark.sql.optimizer.planChangeLog.level", "3.1.0", "trace",
+ s"Please use `${PLAN_CHANGE_LOG_LEVEL.key}` instead."),
+ RemovedConfig("spark.sql.optimizer.planChangeLog.rules", "3.1.0", "",
+ s"Please use `${PLAN_CHANGE_LOG_RULES.key}` instead."),
+ RemovedConfig("spark.sql.optimizer.planChangeLog.batches", "3.1.0", "",
+ s"Please use `${PLAN_CHANGE_LOG_BATCHES.key}` instead.")
+ )
+
+ Map(configs.map { cfg => cfg.key -> cfg } : _*)
+ }
+}
+
+/**
+ * A class that enables the setting and getting of mutable config parameters/hints.
+ *
+ * In the presence of a SQLContext, these can be set and queried by passing SET commands
+ * into Spark SQL's query functions (i.e. sql()). Otherwise, users of this class can
+ * modify the hints by programmatically calling the setters and getters of this class.
+ *
+ * SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads).
+ */
+class SQLConf extends Serializable with Logging {
+ import SQLConf._
+
+ /** Only low degree of contention is expected for conf, thus NOT using ConcurrentHashMap. */
+ @transient protected[spark] val settings = java.util.Collections.synchronizedMap(
+ new java.util.HashMap[String, String]())
+
+ @transient protected val reader = new ConfigReader(settings)
+
+ /** ************************ Spark SQL Params/Hints ******************* */
+
+ def analyzerMaxIterations: Int = getConf(ANALYZER_MAX_ITERATIONS)
+
+ def optimizerExcludedRules: Option[String] = getConf(OPTIMIZER_EXCLUDED_RULES)
+
+ def optimizerMaxIterations: Int = getConf(OPTIMIZER_MAX_ITERATIONS)
+
+ def optimizerInSetConversionThreshold: Int = getConf(OPTIMIZER_INSET_CONVERSION_THRESHOLD)
+
+ def optimizerInSetSwitchThreshold: Int = getConf(OPTIMIZER_INSET_SWITCH_THRESHOLD)
+
+ def planChangeLogLevel: String = getConf(PLAN_CHANGE_LOG_LEVEL)
+
+ def planChangeRules: Option[String] = getConf(PLAN_CHANGE_LOG_RULES)
+
+ def planChangeBatches: Option[String] = getConf(PLAN_CHANGE_LOG_BATCHES)
+
+ def dynamicPartitionPruningEnabled: Boolean = getConf(DYNAMIC_PARTITION_PRUNING_ENABLED)
+
+ def dynamicPartitionPruningUseStats: Boolean = getConf(DYNAMIC_PARTITION_PRUNING_USE_STATS)
+
+ def dynamicPartitionPruningFallbackFilterRatio: Double =
+ getConf(DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO)
+
+ def dynamicPartitionPruningReuseBroadcastOnly: Boolean =
+ getConf(DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY)
+
+ def runtimeFilterSemiJoinReductionEnabled: Boolean =
+ getConf(RUNTIME_FILTER_SEMI_JOIN_REDUCTION_ENABLED)
+
+ def runtimeFilterBloomFilterEnabled: Boolean =
+ getConf(RUNTIME_BLOOM_FILTER_ENABLED)
+
+ def runtimeFilterBloomFilterThreshold: Long =
+ getConf(RUNTIME_BLOOM_FILTER_THRESHOLD)
+
+ def stateStoreProviderClass: String = getConf(STATE_STORE_PROVIDER_CLASS)
+
+ def isStateSchemaCheckEnabled: Boolean = getConf(STATE_SCHEMA_CHECK_ENABLED)
+
+ def stateStoreMinDeltasForSnapshot: Int = getConf(STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT)
+
+ def stateStoreFormatValidationEnabled: Boolean = getConf(STATE_STORE_FORMAT_VALIDATION_ENABLED)
+
+ def checkpointLocation: Option[String] = getConf(CHECKPOINT_LOCATION)
+
+ def isUnsupportedOperationCheckEnabled: Boolean = getConf(UNSUPPORTED_OPERATION_CHECK_ENABLED)
+
+ def useDeprecatedKafkaOffsetFetching: Boolean = getConf(USE_DEPRECATED_KAFKA_OFFSET_FETCHING)
+
+ def statefulOperatorCorrectnessCheckEnabled: Boolean =
+ getConf(STATEFUL_OPERATOR_CHECK_CORRECTNESS_ENABLED)
+
+ def fileStreamSinkMetadataIgnored: Boolean = getConf(FILESTREAM_SINK_METADATA_IGNORED)
+
+ def streamingFileCommitProtocolClass: String = getConf(STREAMING_FILE_COMMIT_PROTOCOL_CLASS)
+
+ def fileSinkLogDeletion: Boolean = getConf(FILE_SINK_LOG_DELETION)
+
+ def fileSinkLogCompactInterval: Int = getConf(FILE_SINK_LOG_COMPACT_INTERVAL)
+
+ def fileSinkLogCleanupDelay: Long = getConf(FILE_SINK_LOG_CLEANUP_DELAY)
+
+ def fileSourceLogDeletion: Boolean = getConf(FILE_SOURCE_LOG_DELETION)
+
+ def fileSourceLogCompactInterval: Int = getConf(FILE_SOURCE_LOG_COMPACT_INTERVAL)
+
+ def fileSourceLogCleanupDelay: Long = getConf(FILE_SOURCE_LOG_CLEANUP_DELAY)
+
+ def streamingSchemaInference: Boolean = getConf(STREAMING_SCHEMA_INFERENCE)
+
+ def streamingPollingDelay: Long = getConf(STREAMING_POLLING_DELAY)
+
+ def streamingNoDataProgressEventInterval: Long =
+ getConf(STREAMING_NO_DATA_PROGRESS_EVENT_INTERVAL)
+
+ def streamingNoDataMicroBatchesEnabled: Boolean =
+ getConf(STREAMING_NO_DATA_MICRO_BATCHES_ENABLED)
+
+ def streamingMetricsEnabled: Boolean = getConf(STREAMING_METRICS_ENABLED)
+
+ def streamingProgressRetention: Int = getConf(STREAMING_PROGRESS_RETENTION)
+
+ def filesMaxPartitionBytes: Long = getConf(FILES_MAX_PARTITION_BYTES)
+
+ def filesOpenCostInBytes: Long = getConf(FILES_OPEN_COST_IN_BYTES)
+
+ def filesMinPartitionNum: Option[Int] = getConf(FILES_MIN_PARTITION_NUM)
+
+ def ignoreCorruptFiles: Boolean = getConf(IGNORE_CORRUPT_FILES)
+
+ def ignoreMissingFiles: Boolean = getConf(IGNORE_MISSING_FILES)
+
+ def maxRecordsPerFile: Long = getConf(MAX_RECORDS_PER_FILE)
+
+ def useCompression: Boolean = getConf(COMPRESS_CACHED)
+
+ def orcCompressionCodec: String = getConf(ORC_COMPRESSION)
+
+ def orcVectorizedReaderEnabled: Boolean = getConf(ORC_VECTORIZED_READER_ENABLED)
+
+ def orcVectorizedReaderBatchSize: Int = getConf(ORC_VECTORIZED_READER_BATCH_SIZE)
+
+ def orcVectorizedReaderNestedColumnEnabled: Boolean =
+ getConf(ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED)
+
+ def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION)
+
+ def parquetVectorizedReaderEnabled: Boolean = getConf(PARQUET_VECTORIZED_READER_ENABLED)
+
+ def parquetVectorizedReaderBatchSize: Int = getConf(PARQUET_VECTORIZED_READER_BATCH_SIZE)
+
+ def columnBatchSize: Int = getConf(COLUMN_BATCH_SIZE)
+
+ def cacheVectorizedReaderEnabled: Boolean = getConf(CACHE_VECTORIZED_READER_ENABLED)
+
+ def defaultNumShufflePartitions: Int = getConf(SHUFFLE_PARTITIONS)
+
+ def numShufflePartitions: Int = {
+ if (adaptiveExecutionEnabled && coalesceShufflePartitionsEnabled) {
+ getConf(COALESCE_PARTITIONS_INITIAL_PARTITION_NUM).getOrElse(defaultNumShufflePartitions)
+ } else {
+ defaultNumShufflePartitions
+ }
+ }
+
+ def adaptiveExecutionEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_ENABLED)
+
+ def adaptiveExecutionLogLevel: String = getConf(ADAPTIVE_EXECUTION_LOG_LEVEL)
+
+ def fetchShuffleBlocksInBatch: Boolean = getConf(FETCH_SHUFFLE_BLOCKS_IN_BATCH)
+
+ def nonEmptyPartitionRatioForBroadcastJoin: Double =
+ getConf(NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN)
+
+ def coalesceShufflePartitionsEnabled: Boolean = getConf(COALESCE_PARTITIONS_ENABLED)
+
+ def minBatchesToRetain: Int = getConf(MIN_BATCHES_TO_RETAIN)
+
+ def maxBatchesToRetainInMemory: Int = getConf(MAX_BATCHES_TO_RETAIN_IN_MEMORY)
+
+ def streamingMaintenanceInterval: Long = getConf(STREAMING_MAINTENANCE_INTERVAL)
+
+ def stateStoreCompressionCodec: String = getConf(STATE_STORE_COMPRESSION_CODEC)
+
+ def parquetFilterPushDown: Boolean = getConf(PARQUET_FILTER_PUSHDOWN_ENABLED)
+
+ def parquetFilterPushDownDate: Boolean = getConf(PARQUET_FILTER_PUSHDOWN_DATE_ENABLED)
+
+ def parquetFilterPushDownTimestamp: Boolean = getConf(PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED)
+
+ def parquetFilterPushDownDecimal: Boolean = getConf(PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED)
+
+ def parquetFilterPushDownStringStartWith: Boolean =
+ getConf(PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED)
+
+ def parquetFilterPushDownInFilterThreshold: Int =
+ getConf(PARQUET_FILTER_PUSHDOWN_INFILTERTHRESHOLD)
+
+ def orcFilterPushDown: Boolean = getConf(ORC_FILTER_PUSHDOWN_ENABLED)
+
+ def isOrcSchemaMergingEnabled: Boolean = getConf(ORC_SCHEMA_MERGING_ENABLED)
+
+ def verifyPartitionPath: Boolean = getConf(HIVE_VERIFY_PARTITION_PATH)
+
+ def metastorePartitionPruning: Boolean = getConf(HIVE_METASTORE_PARTITION_PRUNING)
+
+ def metastorePartitionPruningInSetThreshold: Int =
+ getConf(HIVE_METASTORE_PARTITION_PRUNING_INSET_THRESHOLD)
+
+ def manageFilesourcePartitions: Boolean = getConf(HIVE_MANAGE_FILESOURCE_PARTITIONS)
+
+ def filesourcePartitionFileCacheSize: Long = getConf(HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE)
+
+ def caseSensitiveInferenceMode: HiveCaseSensitiveInferenceMode.Value =
+ HiveCaseSensitiveInferenceMode.withName(getConf(HIVE_CASE_SENSITIVE_INFERENCE))
+
+ def gatherFastStats: Boolean = getConf(GATHER_FASTSTAT)
+
+ def optimizerMetadataOnly: Boolean = getConf(OPTIMIZER_METADATA_ONLY)
+
+ def wholeStageEnabled: Boolean = getConf(WHOLESTAGE_CODEGEN_ENABLED)
+
+ def wholeStageUseIdInClassName: Boolean = getConf(WHOLESTAGE_CODEGEN_USE_ID_IN_CLASS_NAME)
+
+ def wholeStageMaxNumFields: Int = getConf(WHOLESTAGE_MAX_NUM_FIELDS)
+
+ def codegenFallback: Boolean = getConf(CODEGEN_FALLBACK)
+
+ def codegenComments: Boolean = getConf(StaticSQLConf.CODEGEN_COMMENTS)
+
+ def loggingMaxLinesForCodegen: Int = getConf(CODEGEN_LOGGING_MAX_LINES)
+
+ def hugeMethodLimit: Int = getConf(WHOLESTAGE_HUGE_METHOD_LIMIT)
+
+ def methodSplitThreshold: Int = getConf(CODEGEN_METHOD_SPLIT_THRESHOLD)
+
+ def wholeStageSplitConsumeFuncByOperator: Boolean =
+ getConf(WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR)
+
+ def tableRelationCacheSize: Int =
+ getConf(StaticSQLConf.FILESOURCE_TABLE_RELATION_CACHE_SIZE)
+
+ def codegenCacheMaxEntries: Int = getConf(StaticSQLConf.CODEGEN_CACHE_MAX_ENTRIES)
+
+ def exchangeReuseEnabled: Boolean = getConf(EXCHANGE_REUSE_ENABLED)
+
+ def subqueryReuseEnabled: Boolean = getConf(SUBQUERY_REUSE_ENABLED)
+
+ def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE)
+
+ def constraintPropagationEnabled: Boolean = getConf(CONSTRAINT_PROPAGATION_ENABLED)
+
+ def escapedStringLiterals: Boolean = getConf(ESCAPED_STRING_LITERALS)
+
+ def fileCompressionFactor: Double = getConf(FILE_COMPRESSION_FACTOR)
+
+ def stringRedactionPattern: Option[Regex] = getConf(SQL_STRING_REDACTION_PATTERN)
+
+ def sortBeforeRepartition: Boolean = getConf(SORT_BEFORE_REPARTITION)
+
+ def topKSortFallbackThreshold: Int = getConf(TOP_K_SORT_FALLBACK_THRESHOLD)
+
+ def fastHashAggregateRowMaxCapacityBit: Int = getConf(FAST_HASH_AGGREGATE_MAX_ROWS_CAPACITY_BIT)
+
+ def streamingSessionWindowMergeSessionInLocalPartition: Boolean =
+ getConf(STREAMING_SESSION_WINDOW_MERGE_SESSIONS_IN_LOCAL_PARTITION)
+
+ def datetimeJava8ApiEnabled: Boolean = getConf(DATETIME_JAVA8API_ENABLED)
+
+ def uiExplainMode: String = getConf(UI_EXPLAIN_MODE)
+
+ def addSingleFileInAddFile: Boolean = getConf(LEGACY_ADD_SINGLE_FILE_IN_ADD_FILE)
+
+ def legacyMsSqlServerNumericMappingEnabled: Boolean =
+ getConf(LEGACY_MSSQLSERVER_NUMERIC_MAPPING_ENABLED)
+
+ def legacyTimeParserPolicy: LegacyBehaviorPolicy.Value = {
+ LegacyBehaviorPolicy.withName(getConf(SQLConf.LEGACY_TIME_PARSER_POLICY))
+ }
+
+ def broadcastHashJoinOutputPartitioningExpandLimit: Int =
+ getConf(BROADCAST_HASH_JOIN_OUTPUT_PARTITIONING_EXPAND_LIMIT)
+
+ /**
+ * Returns the [[Resolver]] for the current configuration, which can be used to determine if two
+ * identifiers are equal.
+ */
+ def resolver: Resolver = {
+ if (caseSensitiveAnalysis) {
+ org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution
+ } else {
+ org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution
+ }
+ }
+
+ /**
+ * Returns the error handler for handling hint errors.
+ */
+ def hintErrorHandler: HintErrorHandler = HintErrorLogger
+
+ def subexpressionEliminationEnabled: Boolean =
+ getConf(SUBEXPRESSION_ELIMINATION_ENABLED)
+
+ def subexpressionEliminationCacheMaxEntries: Int =
+ getConf(SUBEXPRESSION_ELIMINATION_CACHE_MAX_ENTRIES)
+
+ def autoBroadcastJoinThreshold: Long = getConf(AUTO_BROADCASTJOIN_THRESHOLD)
+
+ def limitScaleUpFactor: Int = getConf(LIMIT_SCALE_UP_FACTOR)
+
+ def advancedPartitionPredicatePushdownEnabled: Boolean =
+ getConf(ADVANCED_PARTITION_PREDICATE_PUSHDOWN)
+
+ def preferSortMergeJoin: Boolean = getConf(PREFER_SORTMERGEJOIN)
+
+ def enableRadixSort: Boolean = getConf(RADIX_SORT_ENABLED)
+
+ def isParquetSchemaMergingEnabled: Boolean = getConf(PARQUET_SCHEMA_MERGING_ENABLED)
+
+ def isParquetSchemaRespectSummaries: Boolean = getConf(PARQUET_SCHEMA_RESPECT_SUMMARIES)
+
+ def parquetOutputCommitterClass: String = getConf(PARQUET_OUTPUT_COMMITTER_CLASS)
+
+ def isParquetBinaryAsString: Boolean = getConf(PARQUET_BINARY_AS_STRING)
+
+ def isParquetINT96AsTimestamp: Boolean = getConf(PARQUET_INT96_AS_TIMESTAMP)
+
+ def isParquetINT96TimestampConversion: Boolean = getConf(PARQUET_INT96_TIMESTAMP_CONVERSION)
+
+ def parquetOutputTimestampType: ParquetOutputTimestampType.Value = {
+ ParquetOutputTimestampType.withName(getConf(PARQUET_OUTPUT_TIMESTAMP_TYPE))
+ }
+
+ def writeLegacyParquetFormat: Boolean = getConf(PARQUET_WRITE_LEGACY_FORMAT)
+
+ def parquetRecordFilterEnabled: Boolean = getConf(PARQUET_RECORD_FILTER_ENABLED)
+
+ def inMemoryPartitionPruning: Boolean = getConf(IN_MEMORY_PARTITION_PRUNING)
+
+ def inMemoryTableScanStatisticsEnabled: Boolean = getConf(IN_MEMORY_TABLE_SCAN_STATISTICS_ENABLED)
+
+ def offHeapColumnVectorEnabled: Boolean = getConf(COLUMN_VECTOR_OFFHEAP_ENABLED)
+
+ def columnNameOfCorruptRecord: String = getConf(COLUMN_NAME_OF_CORRUPT_RECORD)
+
+ def broadcastTimeout: Long = {
+ val timeoutValue = getConf(BROADCAST_TIMEOUT)
+ if (timeoutValue < 0) Long.MaxValue else timeoutValue
+ }
+
+ def defaultDataSourceName: String = getConf(DEFAULT_DATA_SOURCE_NAME)
+
+ def convertCTAS: Boolean = getConf(CONVERT_CTAS)
+
+ def partitionColumnTypeInferenceEnabled: Boolean =
+ getConf(SQLConf.PARTITION_COLUMN_TYPE_INFERENCE)
+
+ def fileCommitProtocolClass: String = getConf(SQLConf.FILE_COMMIT_PROTOCOL_CLASS)
+
+ def parallelPartitionDiscoveryThreshold: Int =
+ getConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD)
+
+ def parallelPartitionDiscoveryParallelism: Int =
+ getConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_PARALLELISM)
+
+ def bucketingEnabled: Boolean = getConf(SQLConf.BUCKETING_ENABLED)
+
+ def bucketingMaxBuckets: Int = getConf(SQLConf.BUCKETING_MAX_BUCKETS)
+
+ def autoBucketedScanEnabled: Boolean = getConf(SQLConf.AUTO_BUCKETED_SCAN_ENABLED)
+
+ def dataFrameSelfJoinAutoResolveAmbiguity: Boolean =
+ getConf(DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY)
+
+ def dataFrameRetainGroupColumns: Boolean = getConf(DATAFRAME_RETAIN_GROUP_COLUMNS)
+
+ def dataFramePivotMaxValues: Int = getConf(DATAFRAME_PIVOT_MAX_VALUES)
+
+ def runSQLonFile: Boolean = getConf(RUN_SQL_ON_FILES)
+
+ def enableTwoLevelAggMap: Boolean = getConf(ENABLE_TWOLEVEL_AGG_MAP)
+
+ def enableVectorizedHashMap: Boolean = getConf(ENABLE_VECTORIZED_HASH_MAP)
+
+ def useObjectHashAggregation: Boolean = getConf(USE_OBJECT_HASH_AGG)
+
+ def objectAggSortBasedFallbackThreshold: Int = getConf(OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD)
+
+ def variableSubstituteEnabled: Boolean = getConf(VARIABLE_SUBSTITUTE_ENABLED)
+
+ def warehousePath: String = new Path(getConf(StaticSQLConf.WAREHOUSE_PATH)).toString
+
+ def hiveThriftServerSingleSession: Boolean =
+ getConf(StaticSQLConf.HIVE_THRIFT_SERVER_SINGLESESSION)
+
+ def orderByOrdinal: Boolean = getConf(ORDER_BY_ORDINAL)
+
+ def groupByOrdinal: Boolean = getConf(GROUP_BY_ORDINAL)
+
+ def groupByAliases: Boolean = getConf(GROUP_BY_ALIASES)
+
+ def crossJoinEnabled: Boolean = getConf(SQLConf.CROSS_JOINS_ENABLED)
+
+ def sessionLocalTimeZone: String = getConf(SQLConf.SESSION_LOCAL_TIMEZONE)
+
+ def jsonGeneratorIgnoreNullFields: Boolean = getConf(SQLConf.JSON_GENERATOR_IGNORE_NULL_FIELDS)
+
+ def jsonExpressionOptimization: Boolean = getConf(SQLConf.JSON_EXPRESSION_OPTIMIZATION)
+
+ def csvExpressionOptimization: Boolean = getConf(SQLConf.CSV_EXPRESSION_OPTIMIZATION)
+
+ def parallelFileListingInStatsComputation: Boolean =
+ getConf(SQLConf.PARALLEL_FILE_LISTING_IN_STATS_COMPUTATION)
+
+ def fallBackToHdfsForStatsEnabled: Boolean = getConf(ENABLE_FALL_BACK_TO_HDFS_FOR_STATS)
+
+ def defaultSizeInBytes: Long = getConf(DEFAULT_SIZE_IN_BYTES)
+
+ def ndvMaxError: Double = getConf(NDV_MAX_ERROR)
+
+ def histogramEnabled: Boolean = getConf(HISTOGRAM_ENABLED)
+
+ def histogramNumBins: Int = getConf(HISTOGRAM_NUM_BINS)
+
+ def percentileAccuracy: Int = getConf(PERCENTILE_ACCURACY)
+
+ def cboEnabled: Boolean = getConf(SQLConf.CBO_ENABLED)
+
+ def planStatsEnabled: Boolean = getConf(SQLConf.PLAN_STATS_ENABLED)
+
+ def autoSizeUpdateEnabled: Boolean = getConf(SQLConf.AUTO_SIZE_UPDATE_ENABLED)
+
+ def joinReorderEnabled: Boolean = getConf(SQLConf.JOIN_REORDER_ENABLED)
+
+ def joinReorderDPThreshold: Int = getConf(SQLConf.JOIN_REORDER_DP_THRESHOLD)
+
+ def joinReorderCardWeight: Double = getConf(SQLConf.JOIN_REORDER_CARD_WEIGHT)
+
+ def joinReorderDPStarFilter: Boolean = getConf(SQLConf.JOIN_REORDER_DP_STAR_FILTER)
+
+ def windowExecBufferInMemoryThreshold: Int = getConf(WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD)
+
+ def windowExecBufferSpillThreshold: Int = getConf(WINDOW_EXEC_BUFFER_SPILL_THRESHOLD)
+
+ def sessionWindowBufferInMemoryThreshold: Int = getConf(SESSION_WINDOW_BUFFER_IN_MEMORY_THRESHOLD)
+
+ def sessionWindowBufferSpillThreshold: Int = getConf(SESSION_WINDOW_BUFFER_SPILL_THRESHOLD)
+
+ def sortMergeJoinExecBufferInMemoryThreshold: Int =
+ getConf(SORT_MERGE_JOIN_EXEC_BUFFER_IN_MEMORY_THRESHOLD)
+
+ def sortMergeJoinExecBufferSpillThreshold: Int =
+ getConf(SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD)
+
+ def cartesianProductExecBufferInMemoryThreshold: Int =
+ getConf(CARTESIAN_PRODUCT_EXEC_BUFFER_IN_MEMORY_THRESHOLD)
+
+ def cartesianProductExecBufferSpillThreshold: Int =
+ getConf(CARTESIAN_PRODUCT_EXEC_BUFFER_SPILL_THRESHOLD)
+
+ def codegenSplitAggregateFunc: Boolean = getConf(SQLConf.CODEGEN_SPLIT_AGGREGATE_FUNC)
+
+ def maxNestedViewDepth: Int = getConf(SQLConf.MAX_NESTED_VIEW_DEPTH)
+
+ def useCurrentSQLConfigsForView: Boolean = getConf(SQLConf.USE_CURRENT_SQL_CONFIGS_FOR_VIEW)
+
+ def storeAnalyzedPlanForView: Boolean = getConf(SQLConf.STORE_ANALYZED_PLAN_FOR_VIEW)
+
+ def allowAutoGeneratedAliasForView: Boolean = getConf(SQLConf.ALLOW_AUTO_GENERATED_ALIAS_FOR_VEW)
+
+ def allowStarWithSingleTableIdentifierInCount: Boolean =
+ getConf(SQLConf.ALLOW_STAR_WITH_SINGLE_TABLE_IDENTIFIER_IN_COUNT)
+
+ def allowNonEmptyLocationInCTAS: Boolean =
+ getConf(SQLConf.ALLOW_NON_EMPTY_LOCATION_IN_CTAS)
+
+ def starSchemaDetection: Boolean = getConf(STARSCHEMA_DETECTION)
+
+ def starSchemaFTRatio: Double = getConf(STARSCHEMA_FACT_TABLE_RATIO)
+
+ def supportQuotedRegexColumnName: Boolean = getConf(SUPPORT_QUOTED_REGEX_COLUMN_NAME)
+
+ def rangeExchangeSampleSizePerPartition: Int = getConf(RANGE_EXCHANGE_SAMPLE_SIZE_PER_PARTITION)
+
+ def arrowPySparkEnabled: Boolean = getConf(ARROW_PYSPARK_EXECUTION_ENABLED)
+
+ def arrowPySparkSelfDestructEnabled: Boolean = getConf(ARROW_PYSPARK_SELF_DESTRUCT_ENABLED)
+
+ def pysparkJVMStacktraceEnabled: Boolean = getConf(PYSPARK_JVM_STACKTRACE_ENABLED)
+
+ def arrowSparkREnabled: Boolean = getConf(ARROW_SPARKR_EXECUTION_ENABLED)
+
+ def arrowPySparkFallbackEnabled: Boolean = getConf(ARROW_PYSPARK_FALLBACK_ENABLED)
+
+ def arrowMaxRecordsPerBatch: Int = getConf(ARROW_EXECUTION_MAX_RECORDS_PER_BATCH)
+
+ def pandasUDFBufferSize: Int = getConf(PANDAS_UDF_BUFFER_SIZE)
+
+ def pysparkSimplifiedTraceback: Boolean = getConf(PYSPARK_SIMPLIFIEID_TRACEBACK)
+
+ def pandasGroupedMapAssignColumnsByName: Boolean =
+ getConf(SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME)
+
+ def arrowSafeTypeConversion: Boolean = getConf(SQLConf.PANDAS_ARROW_SAFE_TYPE_CONVERSION)
+
+ def replaceExceptWithFilter: Boolean = getConf(REPLACE_EXCEPT_WITH_FILTER)
+
+ def decimalOperationsAllowPrecisionLoss: Boolean = getConf(DECIMAL_OPERATIONS_ALLOW_PREC_LOSS)
+
+ def literalPickMinimumPrecision: Boolean = getConf(LITERAL_PICK_MINIMUM_PRECISION)
+
+ def continuousStreamingEpochBacklogQueueSize: Int =
+ getConf(CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE)
+
+ def continuousStreamingExecutorQueueSize: Int = getConf(CONTINUOUS_STREAMING_EXECUTOR_QUEUE_SIZE)
+
+ def continuousStreamingExecutorPollIntervalMs: Long =
+ getConf(CONTINUOUS_STREAMING_EXECUTOR_POLL_INTERVAL_MS)
+
+ def disabledV2StreamingWriters: String = getConf(DISABLED_V2_STREAMING_WRITERS)
+
+ def disabledV2StreamingMicroBatchReaders: String =
+ getConf(DISABLED_V2_STREAMING_MICROBATCH_READERS)
+
+ def fastFailFileFormatOutput: Boolean = getConf(FASTFAIL_ON_FILEFORMAT_OUTPUT)
+
+ def concatBinaryAsString: Boolean = getConf(CONCAT_BINARY_AS_STRING)
+
+ def eltOutputAsString: Boolean = getConf(ELT_OUTPUT_AS_STRING)
+
+ def validatePartitionColumns: Boolean = getConf(VALIDATE_PARTITION_COLUMNS)
+
+ def partitionOverwriteMode: PartitionOverwriteMode.Value =
+ PartitionOverwriteMode.withName(getConf(PARTITION_OVERWRITE_MODE))
+
+ def storeAssignmentPolicy: StoreAssignmentPolicy.Value =
+ StoreAssignmentPolicy.withName(getConf(STORE_ASSIGNMENT_POLICY))
+
+ def ansiEnabled: Boolean = getConf(ANSI_ENABLED)
+
+ def timestampType: AtomicType = getConf(TIMESTAMP_TYPE) match {
+ // SPARK-36227: Remove TimestampNTZ type support in Spark 3.2 with minimal code changes.
+ // The configuration `TIMESTAMP_TYPE` is only effective for testing in Spark 3.2.
+ case "TIMESTAMP_NTZ" if Utils.isTesting =>
+ TimestampNTZType
+
+ case _ =>
+ // For historical reason, the TimestampType maps to TIMESTAMP WITH LOCAL TIME ZONE
+ TimestampType
+ }
+
+ def nestedSchemaPruningEnabled: Boolean = getConf(NESTED_SCHEMA_PRUNING_ENABLED)
+
+ def serializerNestedSchemaPruningEnabled: Boolean =
+ getConf(SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED)
+
+ def nestedPruningOnExpressions: Boolean = getConf(NESTED_PRUNING_ON_EXPRESSIONS)
+
+ def csvColumnPruning: Boolean = getConf(SQLConf.CSV_PARSER_COLUMN_PRUNING)
+
+ def legacySizeOfNull: Boolean = {
+ // size(null) should return null under ansi mode.
+ getConf(SQLConf.LEGACY_SIZE_OF_NULL) && !getConf(ANSI_ENABLED)
+ }
+
+ def isReplEagerEvalEnabled: Boolean = getConf(SQLConf.REPL_EAGER_EVAL_ENABLED)
+
+ def replEagerEvalMaxNumRows: Int = getConf(SQLConf.REPL_EAGER_EVAL_MAX_NUM_ROWS)
+
+ def replEagerEvalTruncate: Int = getConf(SQLConf.REPL_EAGER_EVAL_TRUNCATE)
+
+ def avroCompressionCodec: String = getConf(SQLConf.AVRO_COMPRESSION_CODEC)
+
+ def avroDeflateLevel: Int = getConf(SQLConf.AVRO_DEFLATE_LEVEL)
+
+ def replaceDatabricksSparkAvroEnabled: Boolean =
+ getConf(SQLConf.LEGACY_REPLACE_DATABRICKS_SPARK_AVRO_ENABLED)
+
+ def setOpsPrecedenceEnforced: Boolean = getConf(SQLConf.LEGACY_SETOPS_PRECEDENCE_ENABLED)
+
+ def exponentLiteralAsDecimalEnabled: Boolean =
+ getConf(SQLConf.LEGACY_EXPONENT_LITERAL_AS_DECIMAL_ENABLED)
+
+ def allowNegativeScaleOfDecimalEnabled: Boolean =
+ getConf(SQLConf.LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED)
+
+ def legacyStatisticalAggregate: Boolean = getConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE)
+
+ def truncateTableIgnorePermissionAcl: Boolean =
+ getConf(SQLConf.TRUNCATE_TABLE_IGNORE_PERMISSION_ACL)
+
+ def nameNonStructGroupingKeyAsValue: Boolean =
+ getConf(SQLConf.NAME_NON_STRUCT_GROUPING_KEY_AS_VALUE)
+
+ def maxToStringFields: Int = getConf(SQLConf.MAX_TO_STRING_FIELDS)
+
+ def maxPlanStringLength: Int = getConf(SQLConf.MAX_PLAN_STRING_LENGTH).toInt
+
+ def maxMetadataStringLength: Int = getConf(SQLConf.MAX_METADATA_STRING_LENGTH)
+
+ def setCommandRejectsSparkCoreConfs: Boolean =
+ getConf(SQLConf.SET_COMMAND_REJECTS_SPARK_CORE_CONFS)
+
+ def castDatetimeToString: Boolean = getConf(SQLConf.LEGACY_CAST_DATETIME_TO_STRING)
+
+ def ignoreDataLocality: Boolean = getConf(SQLConf.IGNORE_DATA_LOCALITY)
+
+ def csvFilterPushDown: Boolean = getConf(CSV_FILTER_PUSHDOWN_ENABLED)
+
+ def jsonFilterPushDown: Boolean = getConf(JSON_FILTER_PUSHDOWN_ENABLED)
+
+ def avroFilterPushDown: Boolean = getConf(AVRO_FILTER_PUSHDOWN_ENABLED)
+
+ def integerGroupingIdEnabled: Boolean = getConf(SQLConf.LEGACY_INTEGER_GROUPING_ID)
+
+ def metadataCacheTTL: Long = getConf(StaticSQLConf.METADATA_CACHE_TTL_SECONDS)
+
+ def coalesceBucketsInJoinEnabled: Boolean = getConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED)
+
+ def coalesceBucketsInJoinMaxBucketRatio: Int =
+ getConf(SQLConf.COALESCE_BUCKETS_IN_JOIN_MAX_BUCKET_RATIO)
+
+ def optimizeNullAwareAntiJoin: Boolean =
+ getConf(SQLConf.OPTIMIZE_NULL_AWARE_ANTI_JOIN)
+
+ def legacyPathOptionBehavior: Boolean = getConf(SQLConf.LEGACY_PATH_OPTION_BEHAVIOR)
+
+ def disabledJdbcConnectionProviders: String = getConf(SQLConf.DISABLED_JDBC_CONN_PROVIDER_LIST)
+
+ def charVarcharAsString: Boolean = getConf(SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING)
+
+ def cliPrintHeader: Boolean = getConf(SQLConf.CLI_PRINT_HEADER)
+
+ def legacyIntervalEnabled: Boolean = getConf(LEGACY_INTERVAL_ENABLED)
+
+ def decorrelateInnerQueryEnabled: Boolean = getConf(SQLConf.DECORRELATE_INNER_QUERY_ENABLED)
+
+ def maxConcurrentOutputFileWriters: Int = getConf(SQLConf.MAX_CONCURRENT_OUTPUT_FILE_WRITERS)
+
+ /** ********************** SQLConf functionality methods ************ */
+
+ /** Set Spark SQL configuration properties. */
+ def setConf(props: Properties): Unit = settings.synchronized {
+ props.asScala.foreach { case (k, v) => setConfString(k, v) }
+ }
+
+ /** Set the given Spark SQL configuration property using a `string` value. */
+ def setConfString(key: String, value: String): Unit = {
+ require(key != null, "key cannot be null")
+ require(value != null, s"value cannot be null for key: $key")
+ val entry = getConfigEntry(key)
+ if (entry != null) {
+ // Only verify configs in the SQLConf object
+ entry.valueConverter(value)
+ }
+ setConfWithCheck(key, value)
+ }
+
+ /** Set the given Spark SQL configuration property. */
+ def setConf[T](entry: ConfigEntry[T], value: T): Unit = {
+ require(entry != null, "entry cannot be null")
+ require(value != null, s"value cannot be null for key: ${entry.key}")
+ require(containsConfigEntry(entry), s"$entry is not registered")
+ setConfWithCheck(entry.key, entry.stringConverter(value))
+ }
+
+ /** Return the value of Spark SQL configuration property for the given key. */
+ @throws[NoSuchElementException]("if key is not set")
+ def getConfString(key: String): String = {
+ Option(settings.get(key)).
+ orElse {
+ // Try to use the default value
+ Option(getConfigEntry(key)).map { e => e.stringConverter(e.readFrom(reader)) }
+ }.
+ getOrElse(throw QueryExecutionErrors.noSuchElementExceptionError(key))
+ }
+
+ /**
+ * Return the value of Spark SQL configuration property for the given key. If the key is not set
+ * yet, return `defaultValue`. This is useful when `defaultValue` in ConfigEntry is not the
+ * desired one.
+ */
+ def getConf[T](entry: ConfigEntry[T], defaultValue: T): T = {
+ require(containsConfigEntry(entry), s"$entry is not registered")
+ Option(settings.get(entry.key)).map(entry.valueConverter).getOrElse(defaultValue)
+ }
+
+ /**
+ * Return the value of Spark SQL configuration property for the given key. If the key is not set
+ * yet, return `defaultValue` in [[ConfigEntry]].
+ */
+ def getConf[T](entry: ConfigEntry[T]): T = {
+ require(containsConfigEntry(entry), s"$entry is not registered")
+ entry.readFrom(reader)
+ }
+
+ /**
+ * Return the value of an optional Spark SQL configuration property for the given key. If the key
+ * is not set yet, returns None.
+ */
+ def getConf[T](entry: OptionalConfigEntry[T]): Option[T] = {
+ require(containsConfigEntry(entry), s"$entry is not registered")
+ entry.readFrom(reader)
+ }
+
+ /**
+ * Return the `string` value of Spark SQL configuration property for the given key. If the key is
+ * not set yet, return `defaultValue`.
+ */
+ def getConfString(key: String, defaultValue: String): String = {
+ Option(settings.get(key)).getOrElse {
+ // If the key is not set, need to check whether the config entry is registered and is
+ // a fallback conf, so that we can check its parent.
+ getConfigEntry(key) match {
+ case e: FallbackConfigEntry[_] =>
+ getConfString(e.fallback.key, defaultValue)
+ case e: ConfigEntry[_] if defaultValue != null && defaultValue != ConfigEntry.UNDEFINED =>
+ // Only verify configs in the SQLConf object
+ e.valueConverter(defaultValue)
+ defaultValue
+ case _ =>
+ defaultValue
+ }
+ }
+ }
+
+ private var definedConfsLoaded = false
+ /**
+ * Init [[StaticSQLConf]] and [[org.apache.spark.sql.hive.HiveUtils]] so that all the defined
+ * SQL Configurations will be registered to SQLConf
+ */
+ private def loadDefinedConfs(): Unit = {
+ if (!definedConfsLoaded) {
+ definedConfsLoaded = true
+ // Force to register static SQL configurations
+ StaticSQLConf
+ try {
+ // Force to register SQL configurations from Hive module
+ val symbol = ScalaReflection.mirror.staticModule("org.apache.spark.sql.hive.HiveUtils")
+ ScalaReflection.mirror.reflectModule(symbol).instance
+ } catch {
+ case NonFatal(e) =>
+ logWarning("SQL configurations from Hive module is not loaded", e)
+ }
+ }
+ }
+
+ /**
+ * Return all the configuration properties that have been set (i.e. not the default).
+ * This creates a new copy of the config properties in the form of a Map.
+ */
+ def getAllConfs: immutable.Map[String, String] =
+ settings.synchronized { settings.asScala.toMap }
+
+ /**
+ * Return all the configuration definitions that have been defined in [[SQLConf]]. Each
+ * definition contains key, defaultValue and doc.
+ */
+ def getAllDefinedConfs: Seq[(String, String, String, String)] = {
+ loadDefinedConfs()
+ getConfigEntries().asScala.filter(_.isPublic).map { entry =>
+ val displayValue = Option(getConfString(entry.key, null)).getOrElse(entry.defaultValueString)
+ (entry.key, displayValue, entry.doc, entry.version)
+ }.toSeq
+ }
+
+ /**
+ * Redacts the given option map according to the description of SQL_OPTIONS_REDACTION_PATTERN.
+ */
+ def redactOptions[K, V](options: Map[K, V]): Map[K, V] = {
+ redactOptions(options.toSeq).toMap
+ }
+
+ /**
+ * Redacts the given option map according to the description of SQL_OPTIONS_REDACTION_PATTERN.
+ */
+ def redactOptions[K, V](options: Seq[(K, V)]): Seq[(K, V)] = {
+ val regexes = Seq(
+ getConf(SQL_OPTIONS_REDACTION_PATTERN),
+ SECRET_REDACTION_PATTERN.readFrom(reader))
+
+ regexes.foldLeft(options) { case (opts, r) => Utils.redact(Some(r), opts) }
+ }
+
+ /**
+ * Return whether a given key is set in this [[SQLConf]].
+ */
+ def contains(key: String): Boolean = {
+ settings.containsKey(key)
+ }
+
+ /**
+ * Logs a warning message if the given config key is deprecated.
+ */
+ private def logDeprecationWarning(key: String): Unit = {
+ SQLConf.deprecatedSQLConfigs.get(key).foreach {
+ case DeprecatedConfig(configName, version, comment) =>
+ logWarning(
+ s"The SQL config '$configName' has been deprecated in Spark v$version " +
+ s"and may be removed in the future. $comment")
+ }
+ }
+
+ private def requireDefaultValueOfRemovedConf(key: String, value: String): Unit = {
+ SQLConf.removedSQLConfigs.get(key).foreach {
+ case RemovedConfig(configName, version, defaultValue, comment) =>
+ if (value != defaultValue) {
+ throw QueryCompilationErrors.configRemovedInVersionError(configName, version, comment)
+ }
+ }
+ }
+
+ protected def setConfWithCheck(key: String, value: String): Unit = {
+ logDeprecationWarning(key)
+ requireDefaultValueOfRemovedConf(key, value)
+ settings.put(key, value)
+ }
+
+ def unsetConf(key: String): Unit = {
+ logDeprecationWarning(key)
+ settings.remove(key)
+ }
+
+ def unsetConf(entry: ConfigEntry[_]): Unit = {
+ unsetConf(entry.key)
+ }
+
+ def clear(): Unit = {
+ settings.clear()
+ }
+
+ override def clone(): SQLConf = {
+ val result = new SQLConf
+ getAllConfs.foreach {
+ case(k, v) => if (v ne null) result.setConfString(k, v)
+ }
+ result
+ }
+
+ // For test only
+ def copy(entries: (ConfigEntry[_], Any)*): SQLConf = {
+ val cloned = clone()
+ entries.foreach {
+ case (entry, value) => cloned.setConfString(entry.key, value.toString)
+ }
+ cloned
+ }
+
+ def isModifiable(key: String): Boolean = {
+ containsConfigKey(key) && !isStaticConfigKey(key)
+ }
+}