Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22952
BTW: [HADOOP-15748](https://issues.apache.org/jira/browse/HADOOP-15748),
*S3 listing inconsistency can raise NPE in globber*
Could be backported to 2.8+; low risk
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22952
> Looks like we can leverage GlobPattern but it is marked as @Private.
I will happily switch this from private to public/evolving if you submit a
patch against hadoop-trunk; backp
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22952
bq. GlobExpander is private
that's correctable.
1. Make sure there are standalone tests (if none around)
1. Make sure that off filesystem.md there's something declaring
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22952
> HDFS does not support it yet, though on the way, see
https://issues.apache.org/jira/browse/HADOOP-10019
That's an old patch; I don't know of any active
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22952
Hadoop FS glob filtering is pathologically bad on object stores.
I have tried in the past to do an ~O(1) impl for S3
[HADOOP-13371](https://issues.apache.org/jira/browse/HADOOP-13371
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22598#discussion_r237206264
--- Diff:
core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala ---
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22598#discussion_r237203744
--- Diff:
core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala ---
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21066
+1
one thing to consider here is to be ruthless about when there are things in
bits of the HDFS APIs/libraries which don't suit, and rather than think "how do
we work a
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21066
The main barrier to this is the what-do-we-do-about-hive problem, as
without it ASF Spark doesn't run against Hadoop 3.x
It looks like "support Hive 2" is the plan the
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22598#discussion_r233871529
--- Diff:
core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala ---
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22598#discussion_r233870838
--- Diff: core/pom.xml ---
@@ -408,6 +408,19 @@
provided
+
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22598#discussion_r233870430
--- Diff:
core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala
---
@@ -0,0 +1,65 @@
+/*
+ * Licensed
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22952#discussion_r232825455
--- Diff: docs/structured-streaming-programming-guide.md ---
@@ -530,6 +530,8 @@ Here are the details of all the sources in Spark.
&qu
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22952#discussion_r232824087
--- Diff: docs/structured-streaming-programming-guide.md ---
@@ -530,6 +530,8 @@ Here are the details of all the sources in Spark.
&qu
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22752#discussion_r226243409
--- Diff:
core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
@@ -449,7 +450,7 @@ private[history] class
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22598#discussion_r226235800
--- Diff:
external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelperSuite.scala
---
@@ -0,0 +1,129
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22598#discussion_r226235157
--- Diff:
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/TokenUtil.scala
---
@@ -0,0 +1,111 @@
+/*
+ * Licensed
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22598#discussion_r226234330
--- Diff:
core/src/main/scala/org/apache/spark/deploy/security/KafkaDelegationTokenProvider.scala
---
@@ -0,0 +1,65 @@
+/*
+ * Licensed
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22752#discussion_r225908701
--- Diff:
core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
@@ -449,7 +450,7 @@ private[history] class
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22598#discussion_r223625059
--- Diff:
external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/TokenUtilSuite.scala
---
@@ -0,0 +1,114 @@
+/*
+ * Licensed
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22598#discussion_r223623927
--- Diff:
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala
---
@@ -0,0 +1,91
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22598#discussion_r222061609
--- Diff:
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala
---
@@ -0,0 +1,86
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22598#discussion_r221586243
--- Diff:
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSecurityHelper.scala
---
@@ -0,0 +1,86
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22339
no, no cost penalties. Slightly lower namenode load too. If you had many,
many spark streaming clients scanning directories, HDFS ops teams would
eventually get upset. This will postpone
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22339#discussion_r221326673
--- Diff:
streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
---
@@ -196,29 +191,29 @@ class FileInputDStream[K, V
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22339
Why the speedups? Comes from that glob filter calling getFileStatus() on
every entry, which is is 1-3 HTTP requests and a few hundred millis per call,
when instead that can be handled later
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22339#discussion_r22134
--- Diff:
streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
---
@@ -196,29 +191,29 @@ class FileInputDStream[K, V
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21048
if this is being looked at again, it'd be nice to have a reference back end
which just did the write straight to the destination: this is exactly what all
the public cloud stores (s3, azure
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22444
I see the reasoning here
* @jianjianjiao has a very large cluster with many thousands of history
files of past (successful) jobs.
* history server startup needs to go through all
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22327
The goal for the 2.7.x line should be "nothing breaks", which is precisely
why it's only getting critical patches. Reverting might make the problem go
away, but you can assume tha
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22327
The 2.7.x branches updates are generally all security plus some fixes for
JVM/JDK regressions.
without looking at the details, you can assume that the regression will be
related
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/17745
Patch is in the spark cloud integration module, you can take it and try to
get into ASF spark provided you also add some credit to me in the patch
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22186
thanks
---
-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22213
code LGTM. Clearly its a tangible problem, especially for some one-char
option like "myapp.line.sepa
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22213
This actually makes sense. We always forget this, but java properties file
format is [more complex than any of us
remember](https://docs.oracle.com/javase/10/docs/api/java/util
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22186
This will eliminate a race condition between FS shutdown (in the hadoop
shutdown manager) and the hive callback. Theres a risk today that the
filesystems will be closed before that event log
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22186
The latest patch builds locally
Maven test outcome
* lots of json missing method errors, clearly jackson version problems of
some kind
* I don't see log messages of hive
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22186
My local build wasn't including that module; it now does and the link works
with a subclass of `AbstractFunction0`.
The local tests are failing under maven with hive/jackson mismatch
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22186
my local maven build *did* work, so maybe its a javac/JVM version thing.
Will move back to a java class callback
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/17745#discussion_r212822877
--- Diff:
streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
---
@@ -196,29 +191,29 @@ class FileInputDStream[K, V
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/17745#discussion_r212391371
--- Diff:
streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
---
@@ -196,29 +191,29 @@ class FileInputDStream[K, V
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22186
Not sure what is up with the build here; worked with mvn locally. Possibly
my use of a java 8 lamda-expression as the hook
GitHub user steveloughran opened a pull request:
https://github.com/apache/spark/pull/22186
[SPARK-25183][SQL] Spark HiveServer2 to use Spark ShutdownHookManager
## What changes were proposed in this pull request?
Switch `org.apache.hive.service.server.HiveServer2
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/17342
Well, no obvious answer there I'm afraid, except "don't put HDFS JARs on
the classpath"; if you serve them up via HTTP all s
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/17342
Hmmm.[SPARK-21697](https://issues.apache.org/jira/browse/SPARK-21697) has a
lot of the CP, but the problem in that one is some recursive loading of
artifacts off HDFS, the can for commons
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/17342
At a guess, there's possibly a mix here between hadoop hdfs JARs on your
classpath. You sure everything on the classpath is in sync? What JARs with
hadoop-hdfs
Github user steveloughran closed the pull request at:
https://github.com/apache/spark/pull/22117
---
-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22081
Thanks. Two less JARs on the CP to keep up to date âwhat more can anyone
want?
---
-
To unsubscribe, e-mail: reviews
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22117
Test failure in `
org.apache.spark.sql.hive.client.HiveClientSuites.(It is not a test it is a
sbt.testing.SuiteSelector)`:
```
Caused by: sbt.ForkMain$ForkError
GitHub user steveloughran opened a pull request:
https://github.com/apache/spark/pull/22117
[SPARK-23654][BUILD] remove jets3t as a dependency of spark
# What changes were proposed in this pull request?
Remove jets3t dependency, and bouncy castle which it brings in; update
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22099
thanks
---
-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h
Github user steveloughran closed the pull request at:
https://github.com/apache/spark/pull/21146
---
-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21146
Closing now we have a test run with the combination of: No jets3t, no
bouncy castle, upgraded kinesis. *all the kinesis tests now run
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22099
> To be clear you think this passed because it still uses jets3t and that
still brings in BC?
correct
> Then we can maybe merge this and rebase the other change to fi
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22081
making a test-time option is a reasonable idea -getting the unlimited JCE
on the test machines (they don't right now) would remove the need
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22099
Local kinesis tests with both -Phadoop-3.1, -Phadoop-2.7 & `Phadoop-3.1
-Dhadoop.version=3.1.1` are all working here (with bouncycastle, unlimited JCE
in JVM).
I'm upda
GitHub user steveloughran reopened a pull request:
https://github.com/apache/spark/pull/21146
[SPARK-23654][BUILD] remove jets3t as a dependency of spark
## What changes were proposed in this pull request?
With the update of bouncy-castle JAR in Spark 2.3; jets3t doesn't
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22081
No, the SDKs dont pull in bouncy-castle. Checked via mvnrepo
* [core
sdk](http://mvnrepository.com/artifact/com.amazonaws/aws-java-sdk-core/1.11.271)
pulls in jackson & httpcl
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22099
@srowen @budde @ajfabbri
---
-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22099
As noted in #22146; stripping off bouncy castle and upgrading the SDK
worked. But a local test run of just this patch brought up the same error seen
in #22081
GitHub user steveloughran opened a pull request:
https://github.com/apache/spark/pull/22099
[SPARK-25111][BUILD] increment kinesis client/producer & aws-sdk versions
## What changes were proposed in this pull request?
Increment the kinesis client, producer and transient
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21146
And if you bump up the kinesis client and AWS SDK version to 1.11.271,
those failures go away.
```
Run completed in 15 minutes, 28 seconds.
Total number of tests run: 59
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21146
FYI, I just did a kinesis test run with this PR on a JVM with the unlimited
JCE installed (explicitly verified by shasum of the relevant JARs); failure
with cert errors
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/22081
I've just pushed up my PR which is ~ in sync with this one; I'll close that
one now and this can be the one to use.
Assume: kinesis uses bouncy castle somewhere. There's some hints
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21146
closing as #22081 supercedes it
---
-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands
Github user steveloughran closed the pull request at:
https://github.com/apache/spark/pull/21146
---
-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22081#discussion_r209707448
--- Diff: pom.xml ---
@@ -984,24 +987,15
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/22009#discussion_r208317741
--- Diff:
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
---
@@ -270,11 +269,11 @@ private[kafka010
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21146#discussion_r200596975
--- Diff: dev/deps/spark-deps-hadoop-2.6 ---
@@ -21,8 +21,6 @@ automaton-1.11-8.jar
avro-1.7.7.jar
avro-ipc-1.7.7.jar
avro-mapred-1.7.7
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21146#discussion_r200162894
--- Diff: pom.xml ---
@@ -141,7 +141,7 @@
3.1.5
1.7.7
hadoop2
-0.9.4
+1.1.1
--- End diff
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21146
OK, I've reinstated javax.activation 1.1.1 as an export from spark core
(over v 1.1), point to this JIRA in the comments, and updated the -deps lists
to remove the others.
Removing
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21146
BTW, the activation framework (primariy used for some mime type stuff) is
still being developed, now on github at @javaee
[https://github.com/javaee/activation](https://github.com/javaee
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21146
There's usually good reasons for upgrading crypto stuff like bouncy castle;
nothing to feel bad about.
How about I take this patch & add the explicit activation 1.1.1
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21146
The transitive dependencies are a separate issue. Jets3t pulls in 3 JARs
which nothing else seems to need, but which transitively go onto the spark CP
downgraded
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21146
It's not going to stop user code from working as the bouncy castle version
on the classpath means that Jets3t doesn't actually work.
The fact that nobody has complained about
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21588
There's a technical issue: trivial change to the case statement
and a ASF process one: the only ASF project which can release hive
artifacts is the hive team; it's that way due to ASF
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21257
some overall thought
* I think this is only happening on a successful job commit, not abort.
This is the desired action?
* if something goes wrong here, is failing the entire job
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21257#discussion_r197177156
--- Diff:
core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
---
@@ -235,4 +244,41 @@ class
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21257#discussion_r197176461
--- Diff:
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala
---
@@ -207,9 +210,23 @@ case
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21257#discussion_r197176292
--- Diff:
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala
---
@@ -207,9 +210,23 @@ case
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21257#discussion_r197174835
--- Diff:
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala
---
@@ -207,9 +210,23 @@ case
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21257#discussion_r197173180
--- Diff:
core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
---
@@ -235,4 +244,41 @@ class
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21257#discussion_r197172859
--- Diff:
core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
---
@@ -235,4 +244,41 @@ class
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21588
If jenkins is happy, this is good.
* Be interesting to see what happens in a build with the
hadoop-cloud-storage module, if it adds new dependencies
* regarding commons-config, know
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21286
@jinxing64 yes, with the detail that the way some bits of hadoop parse a
jobattempt, they like it to be an integer. Some random number used as the upper
digits of counter could work; it'd
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21286
@jinxing64 from my reading of the code, the original patch proposed
creating a temp dir for every query, which could then do its own work & cleanup
in parallel, with a new meta-commit on
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21286
> After the job is committed, `skip_dir/tab1/_temporary` will be deleted.
Then when other jobs attempt to commit, an error will be reported.
I see. Yes, tha
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21257#discussion_r188262174
--- Diff:
core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
---
@@ -163,6 +170,15 @@ class
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21286
...this makes me think that the FileOutputCommitter actually has an
assumption that nobody has called out before, specifically "only one
application will be writing data to the targ
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21286
that would work. Like you say, no need to worry about job attempt IDs, just
uniqueness. If you put the timestamp first, you could still sort the listing by
time, which might be good
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21257#discussion_r187954805
--- Diff:
core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
---
@@ -235,4 +247,23 @@ class
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21257#discussion_r187953870
--- Diff:
core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
---
@@ -163,6 +169,12 @@ class
Github user steveloughran commented on the issue:
https://github.com/apache/spark/pull/21286
> cc @steveloughran who I believe is the expert in this area.
I suppose "Stepped through the FileOutputCommit operations with a debugger
and a pen and paper"
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21066#discussion_r186484550
--- Diff:
hadoop-cloud/src/hadoop-3/main/scala/org/apache/spark/internal/io/cloud/PathOutputCommitProtocol.scala
---
@@ -0,0 +1,260
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21066#discussion_r186482587
--- Diff:
hadoop-cloud/src/hadoop-3/main/scala/org/apache/spark/internal/io/cloud/package.scala
---
@@ -0,0 +1,105 @@
+/*
+ * Licensed
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21066#discussion_r186482016
--- Diff:
hadoop-cloud/src/hadoop-3/main/scala/org/apache/spark/internal/io/cloud/PathOutputCommitProtocol.scala
---
@@ -0,0 +1,260
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21066#discussion_r186475370
--- Diff:
hadoop-cloud/src/hadoop-3/main/scala/org/apache/spark/internal/io/cloud/PathOutputCommitProtocol.scala
---
@@ -0,0 +1,260
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21066#discussion_r186474730
--- Diff:
hadoop-cloud/src/hadoop-3/main/scala/org/apache/spark/internal/io/cloud/PathOutputCommitProtocol.scala
---
@@ -0,0 +1,260
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21066#discussion_r186474501
--- Diff:
hadoop-cloud/src/hadoop-3/main/scala/org/apache/spark/internal/io/cloud/PathOutputCommitProtocol.scala
---
@@ -0,0 +1,260
Github user steveloughran commented on a diff in the pull request:
https://github.com/apache/spark/pull/21066#discussion_r186474366
--- Diff:
hadoop-cloud/src/hadoop-3/main/scala/org/apache/spark/internal/io/cloud/PathOutputCommitProtocol.scala
---
@@ -0,0 +1,260
1 - 100 of 1115 matches
Mail list logo