[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-24 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/15382


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-19 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r84189944
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala ---
@@ -219,8 +219,8 @@ class SQLConfSuite extends QueryTest with 
SharedSQLContext {
 try {
   // to get the default value, always unset it
   spark.conf.unset(SQLConf.WAREHOUSE_PATH.key)
-  assert(spark.sessionState.conf.warehousePath
-=== new 
Path(s"${System.getProperty("user.dir")}/spark-warehouse").toString)
+  assert(new Path(Utils.resolveURI("spark-warehouse")).toString ===
+spark.sessionState.conf.warehousePath + "/")
--- End diff --

could "/" change on Windows?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-18 Thread mridulm
Github user mridulm commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r83928417
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -741,7 +741,7 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = 
Utils.resolveURI(getConf(WAREHOUSE_PATH)).toString
--- End diff --

@srowen Just for clarity, what is the proposed change in behavior expected 
(other than default being under cwd instead of $HOME) ? Thx


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-18 Thread koertkuipers
Github user koertkuipers commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r83921525
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -741,7 +741,7 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = 
Utils.resolveURI(getConf(WAREHOUSE_PATH)).toString
--- End diff --

i agree with @vanzin about dislike for resolveURI. i expect paths without 
schemes to on my default filesystem.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-18 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r83899709
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -741,7 +741,7 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = 
Utils.resolveURI(getConf(WAREHOUSE_PATH)).toString
--- End diff --

You are right, the difference was that the _default_ before was always a 
local path. Right now ends up defaulting to an HDFS path (when using with HDFS 
for example). But explicitly setting "/foo/bar" has been interpreted as a HDFS 
path in the past and that would change. Hm, OK I see your point about only 
applying this to the default value.

Looking at `hive.metastore.warehouse.dir`, that seems to accept a path 
without scheme and that's supposed to be read as an HDFS path, I suppose.

The catch remains Windows paths. It should always be possible to write 
these as URIs, but, then becomes mandatory to write them as a URI, if I make 
this new change. That's a downside. That's only if you need to specify a local, 
non-default path on Windows, which might be a fairly corner case.

I can make the change to have it available for consideration here which I 
assume @vanzin  would favor, and I can see the logic of. How about @avulanov ? 
I know this has dragged on but I want to get it right. It's complex.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-18 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r83897527
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -741,7 +741,7 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = 
Utils.resolveURI(getConf(WAREHOUSE_PATH)).toString
--- End diff --

> interpreted as an HDFS path by default because that's not how it worked 
historically

Except that is how it's worked historically for this piece of code at 
least. Taking a small detour, the reason I really dislike the use of 
`resolveURI` everywhere (and I commented on this when that code was added) is 
that it makes it awkward to use HDFS. Instead of relying on `fs.defaultFS` now 
you have to know the HDFS scheme (which can be "hdfs", "maprfs", "s3", 
"webhdfs", "viewfs", or others depending on your particular configuration) or 
things will break in mysterious ways.

Anyway, I don't really care that much for this particular setting, so if 
you think changing the behavior is better, it's fine with me.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-18 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r83823942
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -741,7 +741,7 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = 
Utils.resolveURI(getConf(WAREHOUSE_PATH)).toString
--- End diff --

Well, before SPARK-15899 it was always interpreted as a local file path. 
After that change (i.e. right now) it would be interpreted as an HDFS path if 
that's what `fs.defaultFS` says. My proposition is that:

- It should default to `spark-warehouse` in the local filesystem working 
dir, because that's what the docs say (therefore, we have a bug at the moment 
after SPARK-15899)
- It should be interpreted as a local path if no scheme is given, because 
other stuff in Spark works that way via `Utils.resolveURI`
- It should be possible to set this to a non-local path, because you can do 
this with the Hive warehouse dir, which this option sort of parallels

I think the current change matches those requirements. I think we do not 
want "/user/blah/warehouse" to be interpreted as an HDFS path by default 
because that's not how it worked historically, and that it works this way now 
is I believe an error.

Is that convincing?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-14 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r83485770
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -757,7 +758,10 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = {
+val path = new Path(getConf(WAREHOUSE_PATH))
+FileSystem.get(path.toUri, new 
Configuration()).makeQualified(path).toString
--- End diff --

Can we split the difference here and treat those two as different issues?

- for this bug, let's hardcode `.transform(new File(_).toURI().toString())` 
into the config constant definition; that's on par with the previous version of 
this code modified in 11a6844beb, which hardcoded the `file:` scheme into the 
default value.

- separately, if it's a real issue, let's make changes to make this support 
non-local directories.

That way we fix the immediate issue and can have more discussions about how 
to properly solve the other and maybe even fix `resolveURI`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-14 Thread avulanov
Github user avulanov commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r83478546
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -757,7 +758,10 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = {
+val path = new Path(getConf(WAREHOUSE_PATH))
+FileSystem.get(path.toUri, new 
Configuration()).makeQualified(path).toString
--- End diff --

I mean fixing the behavior because it is incorrect. We cannot use this 
function in its present state, it will introduce more issues. This seems to be 
a part of a bigger issue that paths in Spark are processed in many different 
ways that are not consistent. Here few examples: 
https://github.com/apache/spark/pull/12695#issuecomment-237824442, 
https://github.com/apache/spark/pull/13868#issuecomment-228932793



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-14 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r83410693
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -757,7 +758,10 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = {
+val path = new Path(getConf(WAREHOUSE_PATH))
+FileSystem.get(path.toUri, new 
Configuration()).makeQualified(path).toString
--- End diff --

Do you mean just updating the documentation or changing the behavior? 
Updating the docs are fine. Updating the behavior (and name I guess) would 
affect a bunch of other places in the code, and that seems like a bigger issue 
to tackle separately.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-13 Thread avulanov
Github user avulanov commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r83346125
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -757,7 +758,10 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = {
+val path = new Path(getConf(WAREHOUSE_PATH))
+FileSystem.get(path.toUri, new 
Configuration()).makeQualified(path).toString
--- End diff --

`resolveURI` function does not process correctly some of the inputs (even 
some Linux paths). It contains a bug and should be fixed. The name itself is a 
bit misleading, because it processes `String` rather than `URI`, and is capable 
of resolving Paths. I propose to rename it to `resolvePath`, and rewrite it 
according to its documentation (below). Then we can use it to fix the initial 
issue. Does it sound reasonable to you?
```
  /**
   * Return a well-formed URI for the file described by a user input string.
   *
   * If the supplied path does not contain a scheme, or is a relative path, 
it will be
   * converted into an absolute path with a file:// scheme.
   */
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-13 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r83183636
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -757,7 +758,10 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = {
+val path = new Path(getConf(WAREHOUSE_PATH))
+FileSystem.get(path.toUri, new 
Configuration()).makeQualified(path).toString
--- End diff --

Yeah I agree that's why this behaves this way, and it's how similar 
paths/URIs would be treated in a lot of places in Spark. Therefore I'm 
wondering if that's the right thing to do, locally, to accept the same behavior 
and quirks in parsing this path (and then consider changing that globally later 
if needed). If I'm right then there is at least a syntax available for hdfs: 
and file: URIs, and one that works for Linux and Windows, including paths with 
spaces. Not all possible strings work as expected, maybe, but the more common 
ones seem to.

The net change at this point is just to use `Utils.resolveURI` to resolve 
this thing like other places in Spark, and then default to "spark-warehouse" 
which will now go back to defaulting to a local working dir path as intended. 
The rest is just test/doc/example updates.

What's your opinion on this current state, compared to the problem being 
solved here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-12 Thread avulanov
Github user avulanov commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r83089960
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -757,7 +758,10 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = {
+val path = new Path(getConf(WAREHOUSE_PATH))
+FileSystem.get(path.toUri, new 
Configuration()).makeQualified(path).toString
--- End diff --

`resolveURI` tries to use `new URI`, encounters a `URISyntaxException` due 
to the white space symbol and falls back to: `new 
File(path).getAbsoluteFile().toURI()`. The latter does not deal with schema. 
This is why the first example looks weird:
```
scala> resolveURI("file:///C:/My Programs/path")
(Spark)res28: java.net.URI = 
file:/c:/dis/dev/spark-2.0.0-preview-bin-hadoop2.7/bin/fil
e:/C:/My%20Programs/path
(Scala) res1: java.net.URI = 
file:/C:/Users/ulanov/file:/C:/My%20Programs/path
-no space
scala> resolveURI("file:///C:/MyPrograms/path")
(Spark)java.net.URI = file:///C:/MyPrograms/path
(Scala) java.net.URI = file:///C:/MyPrograms/path
```
Second example is OK.
Third example works fine with white space on Windows (adds home on Linux) 
but breaks without it:

scala> resolveURI("C:/My Programs/path")
res41: java.net.URI = file:/C:/My%20Programs/path
no space---
scala> resolveURI("C:/MyPrograms/path")
res42: java.net.URI = C:/MyPrograms/path
```
Fourth works fine both with both. There is one more subtle thing, the 
letter drive becomes lower case in Spark.
```
scala>  resolveURI("/My Programs/path")
(Spark)res31: java.net.URI = file:/c:/My%20Programs/path
(Scala) res4: java.net.URI = file:/C:/My%20Programs/path
```
A character in the string should not be a reason to execute a particular 
branch of code. We should rather check the schema and work from this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-11 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r82878712
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -757,7 +758,10 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = {
+val path = new Path(getConf(WAREHOUSE_PATH))
+FileSystem.get(path.toUri, new 
Configuration()).makeQualified(path).toString
--- End diff --

There I'm looking at, for example, 
https://github.com/avulanov/spark/blob/ea24b59fe83c37dbab27579141b5c63cccee138d/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala#L141
 in the test code.  In non-test code I think it's the same source you copied, 
in SessionCatalog, line 154. 

Although these code locations can also deal with a scheme vs no scheme, it 
seemed to be easier to deal with it upfront, where it's returned to the rest of 
the code from the conf object. I think it'll be the same code, same complexity 
either place.

The fact that `resolveURI` doesn't quite do what we want here suggests, I 
suppose, that lots of things in Spark aren't going to play well with a Windows 
path with spaces. See:

```
scala> resolveURI("file:///C:/My Programs/path")
res14: java.net.URI = file:/Users/srowen/file:/C:/My%20Programs/path

scala> resolveURI("/C:/My Programs/path")
res15: java.net.URI = file:/C:/My%20Programs/path

scala> resolveURI("C:/My Programs/path")
res16: java.net.URI = file:/Users/srowen/C:/My%20Programs/path

scala> resolveURI("/My Programs/path")
res17: java.net.URI = file:/My%20Programs/path
```

The second (possibly alternative absolute Windows path with space) and 
fourth examples (Linux path with space) happen to come out right. If we're 
willing to also accept here that it's just what's going to work and not work, 
then, at least there is a working syntax for all scenarios when using this 
method.

I believe there's an argument for further changing resolveURI to work with 
more variants here, but I'd have to even figure out first what is supposed to 
work and not work!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-11 Thread avulanov
Github user avulanov commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r82869165
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -757,7 +758,10 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = {
+val path = new Path(getConf(WAREHOUSE_PATH))
+FileSystem.get(path.toUri, new 
Configuration()).makeQualified(path).toString
--- End diff --

You 
[mentioned](https://github.com/apache/spark/pull/13868#discussion_r82154809) 
that the original issue is as follows: _"...the usages of the new 
makeQualifiedPath are a bit wrong in that they explicitly resolve the path 
against the Hadoop file system, which can be HDFS."_ Should we rather look into 
the code that does `makeQualifiedPath` on `warehousePath` given Hadoop FS 
configuration? The fix would be to have a special case with paths that do not 
have a schema. Actually, could you give a link to this code, I could not find 
it right away?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-09 Thread avulanov
Github user avulanov commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r82529161
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -757,7 +758,10 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = {
+val path = new Path(getConf(WAREHOUSE_PATH))
+FileSystem.get(path.toUri, new 
Configuration()).makeQualified(path).toString
--- End diff --

URI standard might be too restrictive for something that a user has to 
manually specify as a string. According to [RFC 
3986](https://tools.ietf.org/html/rfc3986#section-2.3), it allows only numbers, 
digits, dash, underscore, dot and tilde. Anything else is required to be 
[percent-encoded](https://tools.ietf.org/html/rfc3986#section-2.1). This means 
that we have to require user to always do percent-encoding of her path if we 
stick with URI. There are other paths in Spark that user can specify and they 
are not required to be encoded. This might add unnecessary confusion. I would 
suggest to keep it as a path, document that one can optionally add a schema if 
the path is not local, and try to handle different options within Spark.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-09 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r82520916
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -757,7 +758,10 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = {
+val path = new Path(getConf(WAREHOUSE_PATH))
+FileSystem.get(path.toUri, new 
Configuration()).makeQualified(path).toString
--- End diff --

I should say I'm assuming the input must be a URI, because we have to allow 
users to disambiguate by specifying a scheme. And if we do that, then Windows 
paths can't be allowed. Of course, Windows file URIs should work, like 
`file:///C:/...` So yes I am certainly assuming this must be a URI, not a path, 
because I think we have to? If so, sure, documenting it sounds good.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-08 Thread avulanov
Github user avulanov commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r82507085
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -757,7 +758,10 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = {
+val path = new Path(getConf(WAREHOUSE_PATH))
+FileSystem.get(path.toUri, new 
Configuration()).makeQualified(path).toString
--- End diff --

There are two concerns here. Linux paths will work without adding 
`file://`, and Windows will not. That might lead to confusion. Also, I doubt 
that both Windows and Linux users typically add `file://` if they specify a 
local path. As a rule of thumb, if no schema is given, then by default it is a 
local path. The second concern is that `URI` requires percent encoded string, 
so it does not work with white spaces or special symbols in the path. As far as 
I understand, URI is for URIs and not for paths. If we want to use URI then we 
need to change the variable name to `warehouseURI` and add a proper 
documentation about this configuration parameter.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-07 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r82424398
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -757,7 +758,10 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH)
 
-  def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString
+  def warehousePath: String = {
+val path = new Path(getConf(WAREHOUSE_PATH))
+FileSystem.get(path.toUri, new 
Configuration()).makeQualified(path).toString
--- End diff --

Maybe it's better to use `Utils.resolveURI` here? That resolves to `file:` 
by default like most of the rest of Spark, and the previous versions of this 
code. And handles absolute vs. relative paths too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-07 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r82401682
  
--- Diff: 
examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java
 ---
@@ -56,7 +56,7 @@ public void setValue(String value) {
   public static void main(String[] args) {
 // $example on:spark_hive$
 // warehouseLocation points to the default location for managed 
databases and tables
-String warehouseLocation = "file:" + System.getProperty("user.dir") + 
"spark-warehouse";
+String warehouseLocation = "spark-warehouse";
--- End diff --

Here I've just updated the example's location to match the new default.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-06 Thread srowen
Github user srowen commented on a diff in the pull request:

https://github.com/apache/spark/pull/15382#discussion_r82269678
  
--- Diff: docs/sql-programming-guide.md ---
@@ -869,50 +869,27 @@ access data stored in Hive.
 Configuration of Hive is done by placing your `hive-site.xml`, 
`core-site.xml` (for security configuration),
 and `hdfs-site.xml` (for HDFS configuration) file in `conf/`.
 
-
-
-
-
 When working with Hive, one must instantiate `SparkSession` with Hive 
support, including
 connectivity to a persistent Hive metastore, support for Hive serdes, and 
Hive user-defined functions.
 Users who do not have an existing Hive deployment can still enable Hive 
support. When not configured
 by the `hive-site.xml`, the context automatically creates `metastore_db` 
in the current directory and
 creates a directory configured by `spark.sql.warehouse.dir`, which 
defaults to the directory
-`spark-warehouse` in the current directory that the spark application is 
started. Note that
+`spark-warehouse` in the current directory that the Spark application is 
started. Note that
 the `hive.metastore.warehouse.dir` property in `hive-site.xml` is 
deprecated since Spark 2.0.0.
 Instead, use `spark.sql.warehouse.dir` to specify the default location of 
database in warehouse.
-You may need to grant write privilege to the user who starts the spark 
application.
+You may need to grant write privilege to the user who starts the Spark 
application.
 
+
+
+
 {% include_example spark_hive 
scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala %}
 
 
 
-
--- End diff --

Here I am just fixing some duplication of docs that relate to this change. 
There were 3 identical stanzas


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #15382: [SPARK-17810] [SQL] Default spark.sql.warehouse.d...

2016-10-06 Thread srowen
GitHub user srowen opened a pull request:

https://github.com/apache/spark/pull/15382

[SPARK-17810] [SQL] Default spark.sql.warehouse.dir is relative to local FS 
but can resolve as HDFS path

## What changes were proposed in this pull request?

Always resolve spark.sql.warehouse.dir as a local path, and as relative to 
working dir not home dir

## How was this patch tested?

Existing tests.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/srowen/spark SPARK-17810

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/15382.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #15382


commit 4fb0e584aa9711780f89f6309d1c386fb501c336
Author: Sean Owen 
Date:   2016-10-06T19:49:41Z

Always resolve spark.sql.warehouse.dir as a local path, and as relative to 
working dir not home dir




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org