From reviews-return-700915-archive-asf-public=cust-asf.ponee.io@spark.apache.org Mon Sep 10 20:38:10 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 314BB180656 for ; Mon, 10 Sep 2018 20:38:10 +0200 (CEST) Received: (qmail 40092 invoked by uid 500); 10 Sep 2018 18:38:09 -0000 Mailing-List: contact reviews-help@spark.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list reviews@spark.apache.org Received: (qmail 40066 invoked by uid 99); 10 Sep 2018 18:38:08 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 10 Sep 2018 18:38:08 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 8E5FBDFC7C; Mon, 10 Sep 2018 18:38:08 +0000 (UTC) From: srowen To: reviews@spark.apache.org Reply-To: reviews@spark.apache.org References: In-Reply-To: Subject: [GitHub] spark pull request #20611: [SPARK-23425][SQL]Support wildcard in HDFS path f... Content-Type: text/plain Message-Id: <20180910183808.8E5FBDFC7C@git1-us-west.apache.org> Date: Mon, 10 Sep 2018 18:38:08 +0000 (UTC) Github user srowen commented on a diff in the pull request: https://github.com/apache/spark/pull/20611#discussion_r216429821 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala --- @@ -303,94 +303,44 @@ case class LoadDataCommand( s"partitioned, but a partition spec was provided.") } } - - val loadPath = + val loadPath = { if (isLocal) { - val uri = Utils.resolveURI(path) - val file = new File(uri.getPath) - val exists = if (file.getAbsolutePath.contains("*")) { - val fileSystem = FileSystems.getDefault - val dir = file.getParentFile.getAbsolutePath - if (dir.contains("*")) { - throw new AnalysisException( - s"LOAD DATA input path allows only filename wildcard: $path") - } - - // Note that special characters such as "*" on Windows are not allowed as a path. - // Calling `WindowsFileSystem.getPath` throws an exception if there are in the path. - val dirPath = fileSystem.getPath(dir) - val pathPattern = new File(dirPath.toAbsolutePath.toString, file.getName).toURI.getPath - val safePathPattern = if (Utils.isWindows) { - // On Windows, the pattern should not start with slashes for absolute file paths. - pathPattern.stripPrefix("/") - } else { - pathPattern - } - val files = new File(dir).listFiles() - if (files == null) { - false - } else { - val matcher = fileSystem.getPathMatcher("glob:" + safePathPattern) - files.exists(f => matcher.matches(fileSystem.getPath(f.getAbsolutePath))) - } - } else { - new File(file.getAbsolutePath).exists() - } - if (!exists) { - throw new AnalysisException(s"LOAD DATA input path does not exist: $path") - } - uri + val localFS = FileContext.getLocalFSFileContext() + makeQualified(FsConstants.LOCAL_FS_URI, localFS.getWorkingDirectory(), new Path(path)) } else { - val uri = new URI(path) - val hdfsUri = if (uri.getScheme() != null && uri.getAuthority() != null) { - uri - } else { - // Follow Hive's behavior: - // If no schema or authority is provided with non-local inpath, - // we will use hadoop configuration "fs.defaultFS". - val defaultFSConf = sparkSession.sessionState.newHadoopConf().get("fs.defaultFS") - val defaultFS = if (defaultFSConf == null) { - new URI("") - } else { - new URI(defaultFSConf) - } - - val scheme = if (uri.getScheme() != null) { - uri.getScheme() - } else { - defaultFS.getScheme() - } - val authority = if (uri.getAuthority() != null) { - uri.getAuthority() - } else { - defaultFS.getAuthority() - } - - if (scheme == null) { - throw new AnalysisException( - s"LOAD DATA: URI scheme is required for non-local input paths: '$path'") - } - - // Follow Hive's behavior: - // If LOCAL is not specified, and the path is relative, - // then the path is interpreted relative to "/user/" - val uriPath = uri.getPath() - val absolutePath = if (uriPath != null && uriPath.startsWith("/")) { - uriPath - } else { - s"/user/${System.getProperty("user.name")}/$uriPath" - } - new URI(scheme, authority, absolutePath, uri.getQuery(), uri.getFragment()) - } - val hadoopConf = sparkSession.sessionState.newHadoopConf() - val srcPath = new Path(hdfsUri) - val fs = srcPath.getFileSystem(hadoopConf) - if (!fs.exists(srcPath)) { - throw new AnalysisException(s"LOAD DATA input path does not exist: $path") - } - hdfsUri + val loadPath = new Path(path) --- End diff -- Ah right. I think it's necessary to not parse it as a URI because it could contain "?" and that's a reserved char in URIs. We don't want any other behavior to change though; paths with a space in it should still work for example. @sujith71955 do you have maybe a quick test for a case or two like that? and/or @gatorsmile do you know of a case where user-visible behavior has changed (aside from wildcards)? --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org For additional commands, e-mail: reviews-help@spark.apache.org