Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ object ExternalCatalogUtils {
// The following string escaping code is mainly copied from Hive (o.a.h.h.common.FileUtils).
//////////////////////////////////////////////////////////////////////////////////////////////////

val charToEscape = {
final val charToEscape = {
val bitSet = new java.util.BitSet(128)

/**
Expand All @@ -63,29 +63,35 @@ object ExternalCatalogUtils {
bitSet
}

@inline private final def needsEscaping(c: Char): Boolean = {
private final val HEX_CHARS = "0123456789ABCDEF".toCharArray

@inline final def needsEscaping(c: Char): Boolean = {
c < charToEscape.size() && charToEscape.get(c)
}

def escapePathName(path: String): String = {
val firstIndex = path.indexWhere(needsEscaping)
if (firstIndex == -1) {
var firstIndex = 0
val length = path.length
while (firstIndex < length && !needsEscaping(path.charAt(firstIndex))) {
firstIndex += 1
}
if (firstIndex == 0) {
path
} else {
val builder = new StringBuilder(path.substring(0, firstIndex))
path.substring(firstIndex).foreach { c =>
val sb = new StringBuilder(path.substring(0, firstIndex))
while(firstIndex < length) {
val c = path.charAt(firstIndex)
if (needsEscaping(c)) {
builder.append('%')
builder.append(f"${c.asInstanceOf[Int]}%02X")
sb.append('%').append(HEX_CHARS((c & 0xF0) >> 4)).append(HEX_CHARS(c & 0x0F))
Copy link
Contributor

Choose a reason for hiding this comment

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

+1 for performing the hex conversion this way, since the old string formatting was really really expensive:

image

} else {
builder.append(c)
sb.append(c)
}
firstIndex += 1
}
builder.toString()
sb.toString()
}
}


def unescapePathName(path: String): String = {
val sb = new StringBuilder
var i = 0
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* 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

import org.apache.spark.benchmark.{Benchmark, BenchmarkBase}
import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils

object EscapePathBenchmark extends BenchmarkBase {
override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
val N = 1000000
runBenchmark("Escape") {
val benchmark = new Benchmark("Escape Tests", N, 10, output = output)
val paths = Seq(
"https://issues.apache.org/jira/browse/SPARK-48551",
"https...issues.apache.org/jira/browse/SPARK-48551",
"https...issues.apache.org.jira/browse/SPARK-48551",
"https...issues.apache.org.jira.browse/SPARK-48551",
"https...issues.apache.org.jira.browse.SPARK-48551")
benchmark.addCase("Legacy") { _ =>
(1 to N).foreach(_ => paths.foreach(escapePathNameLegacy))
}

benchmark.addCase("New") { _ =>
(1 to N).foreach(_ => {
paths.foreach(ExternalCatalogUtils.escapePathName)
})
}
benchmark.run()
}
}

/**
* Legacy implementation of escapePathName before Spark 4.0
*/
def escapePathNameLegacy(path: String): String = {
val builder = new StringBuilder()
path.foreach { c =>
if (ExternalCatalogUtils.needsEscaping(c)) {
builder.append('%')
builder.append(f"${c.asInstanceOf[Int]}%02X")
} else {
builder.append(c)
}
}

builder.toString()
}
}