1616
1717package io .delta .sharing .client
1818
19+ import java .io .File
1920import java .net .{URI , URLDecoder , URLEncoder }
2021import java .util .concurrent .TimeUnit
2122
2223import org .apache .hadoop .fs ._
2324import org .apache .hadoop .fs .permission .FsPermission
2425import org .apache .hadoop .util .Progressable
2526import org .apache .http .{HttpClientConnection , HttpHost , HttpRequest , HttpResponse }
26- import org .apache .http .auth .{AuthScope , UsernamePasswordCredentials }
2727import org .apache .http .client .config .RequestConfig
2828import org .apache .http .client .utils .URIBuilder
2929import org .apache .http .conn .routing .HttpRoute
30- import org .apache .http .impl .client .{BasicCredentialsProvider , HttpClientBuilder , RequestWrapper }
30+ import org .apache .http .conn .ssl .NoopHostnameVerifier
31+ import org .apache .http .conn .ssl .TrustSelfSignedStrategy
32+ import org .apache .http .impl .client .{CloseableHttpClient , HttpClientBuilder , RequestWrapper }
3133import org .apache .http .impl .conn .{DefaultRoutePlanner , DefaultSchemePortResolver }
3234import org .apache .http .protocol .{HttpContext , HttpRequestExecutor }
35+ import org .apache .http .ssl .SSLContextBuilder
3336import org .apache .spark .SparkEnv
3437import org .apache .spark .delta .sharing .{PreSignedUrlCache , PreSignedUrlFetcher }
3538import org .apache .spark .internal .Logging
@@ -44,18 +47,20 @@ private[sharing] class DeltaSharingFileSystem extends FileSystem with Logging {
4447
4548 lazy private val numRetries = ConfUtils .numRetries(getConf)
4649 lazy private val maxRetryDurationMillis = ConfUtils .maxRetryDurationMillis(getConf)
47- lazy private val timeoutInSeconds = ConfUtils .timeoutInSeconds (getConf)
50+ lazy private val timeoutInMillis = ConfUtils .getTimeoutInMillis (getConf)
4851 lazy private val httpClient = createHttpClient()
4952
50- private [sharing] def createHttpClient () = {
53+ private [sharing] def createHttpClient (): CloseableHttpClient = {
5154 val proxyConfigOpt = ConfUtils .getProxyConfig(getConf)
5255 val maxConnections = ConfUtils .maxConnections(getConf)
56+ val customHeadersOpt = ConfUtils .getCustomHeaders(getConf)
57+
5358 val config = RequestConfig .custom()
54- .setConnectTimeout(timeoutInSeconds * 1000 )
55- .setConnectionRequestTimeout(timeoutInSeconds * 1000 )
56- .setSocketTimeout(timeoutInSeconds * 1000 ).build()
59+ .setConnectTimeout(timeoutInMillis )
60+ .setConnectionRequestTimeout(timeoutInMillis )
61+ .setSocketTimeout(timeoutInMillis ).build()
5762
58- logDebug(s " Creating delta sharing httpClient with timeoutInSeconds : $timeoutInSeconds . " )
63+ logDebug(s " Creating delta sharing httpClient with timeoutInMillis : $timeoutInMillis . " )
5964 val clientBuilder = HttpClientBuilder .create()
6065 .setMaxConnTotal(maxConnections)
6166 .setMaxConnPerRoute(maxConnections)
@@ -66,10 +71,15 @@ private[sharing] class DeltaSharingFileSystem extends FileSystem with Logging {
6671
6772 // Set proxy if provided.
6873 proxyConfigOpt.foreach { proxyConfig =>
69-
7074 val proxy = new HttpHost (proxyConfig.host, proxyConfig.port)
7175 clientBuilder.setProxy(proxy)
7276
77+ if (proxyConfig.authToken.nonEmpty) {
78+ clientBuilder.addInterceptorFirst((request : HttpRequest , _ : HttpContext ) => {
79+ request.addHeader(" Proxy-Authorization" , s " Bearer ${proxyConfig.authToken}" )
80+ })
81+ }
82+
7383 val neverUseHttps = ConfUtils .getNeverUseHttps(getConf)
7484 if (neverUseHttps) {
7585 val httpRequestDowngradeExecutor = new HttpRequestExecutor {
@@ -94,6 +104,7 @@ private[sharing] class DeltaSharingFileSystem extends FileSystem with Logging {
94104 }
95105 clientBuilder.setRequestExecutor(httpRequestDowngradeExecutor)
96106 }
107+
97108 if (proxyConfig.noProxyHosts.nonEmpty || neverUseHttps) {
98109 val routePlanner = new DefaultRoutePlanner (DefaultSchemePortResolver .INSTANCE ) {
99110 override def determineRoute (target : HttpHost ,
@@ -110,6 +121,28 @@ private[sharing] class DeltaSharingFileSystem extends FileSystem with Logging {
110121 }
111122 clientBuilder.setRoutePlanner(routePlanner)
112123 }
124+
125+ if (proxyConfig.sslTrustAll) {
126+ clientBuilder.setSSLHostnameVerifier(NoopHostnameVerifier .INSTANCE )
127+ clientBuilder.setSSLContext(
128+ new SSLContextBuilder ()
129+ .loadTrustMaterial(null , new TrustSelfSignedStrategy )
130+ .build()
131+ )
132+ } else if (proxyConfig.caCertPath.nonEmpty) {
133+ clientBuilder.setSSLContext(
134+ new SSLContextBuilder ()
135+ .loadTrustMaterial(new File (proxyConfig.caCertPath.getOrElse(" " )), null )
136+ .build()
137+ )
138+ }
139+
140+ customHeadersOpt.foreach { headers =>
141+ ConfUtils .validateCustomHeaders(headers)
142+ clientBuilder.addInterceptorFirst((request : HttpRequest , _ : HttpContext ) => {
143+ headers.foreach { case (key, value) => request.addHeader(key, value) }
144+ })
145+ }
113146 }
114147 clientBuilder.build()
115148 }
@@ -122,7 +155,7 @@ private[sharing] class DeltaSharingFileSystem extends FileSystem with Logging {
122155
123156 override def getScheme : String = SCHEME
124157
125- override def getUri () : URI = URI .create(s " $SCHEME:/// " )
158+ override def getUri : URI = URI .create(s " $SCHEME:/// " )
126159
127160 // open a file path with the format below:
128161 // ```
@@ -204,7 +237,7 @@ private[sharing] class DeltaSharingFileSystem extends FileSystem with Logging {
204237
205238private [sharing] object DeltaSharingFileSystem {
206239
207- val SCHEME = " delta-sharing"
240+ private val SCHEME = " delta-sharing"
208241
209242 case class DeltaSharingPath (tablePath : String , fileId : String , fileSize : Long ) {
210243
0 commit comments