steveloughran commented on code in PR #37474: URL: https://github.com/apache/spark/pull/37474#discussion_r943251576
########## hadoop-cloud/src/hadoop-3/test/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManagerSuite.scala: ########## @@ -0,0 +1,72 @@ +/* + * 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.internal.io.cloud + +import java.io.File + +import scala.util.Properties + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.scalatest.BeforeAndAfter + +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.cloud.abortable.AbortableFileSystem +import org.apache.spark.sql.execution.streaming.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.CheckpointFileManagerTests + +class AbortableStreamBasedCheckpointFileManagerSuite + extends CheckpointFileManagerTests with Logging { + + override def withTempHadoopPath(p: Path => Unit): Unit = { + withTempDir { f: File => + val basePath = new Path(AbortableFileSystem.ABORTABLE_FS_SCHEME, null, f.getAbsolutePath) + p(basePath) + } + } + + override def checkLeakingCrcFiles(path: Path): Unit = { } + + override def createManager(path: Path): CheckpointFileManager = { + val conf = new Configuration() + conf.set(s"fs.AbstractFileSystem.${AbortableFileSystem.ABORTABLE_FS_SCHEME}.impl", + "org.apache.spark.internal.io.cloud.abortable.AbstractAbortableFileSystem") + new AbortableStreamBasedCheckpointFileManager(path, conf) + } +} + +@IntegrationTestSuite +class AwsAbortableStreamBasedCheckpointFileManagerSuite + extends AbortableStreamBasedCheckpointFileManagerSuite with BeforeAndAfter { + + val s3aPath = Properties.envOrNone("S3A_PATH") + + override protected def beforeAll(): Unit = { + assert(s3aPath.isDefined, "S3A_PATH must be defined!") + } + + override def withTempHadoopPath(p: Path => Unit): Unit = { + p(new Path(s3aPath.get)) + } + + override def createManager(path: Path): CheckpointFileManager = { + val conf = new Configuration() + conf.set("fs.s3a.aws.credentials.provider", Review Comment: this is always set in the default chain. leaving that list alone will ensure that when run in ec2 it will also check the IAM role provider and so pick up the credentials the VM/container was granted. propose: cut this line ########## hadoop-cloud/src/hadoop-3/test/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManagerSuite.scala: ########## @@ -0,0 +1,72 @@ +/* + * 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.internal.io.cloud + +import java.io.File + +import scala.util.Properties + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.scalatest.BeforeAndAfter + +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.cloud.abortable.AbortableFileSystem +import org.apache.spark.sql.execution.streaming.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.CheckpointFileManagerTests + +class AbortableStreamBasedCheckpointFileManagerSuite + extends CheckpointFileManagerTests with Logging { + + override def withTempHadoopPath(p: Path => Unit): Unit = { + withTempDir { f: File => + val basePath = new Path(AbortableFileSystem.ABORTABLE_FS_SCHEME, null, f.getAbsolutePath) + p(basePath) + } + } + + override def checkLeakingCrcFiles(path: Path): Unit = { } + + override def createManager(path: Path): CheckpointFileManager = { + val conf = new Configuration() + conf.set(s"fs.AbstractFileSystem.${AbortableFileSystem.ABORTABLE_FS_SCHEME}.impl", + "org.apache.spark.internal.io.cloud.abortable.AbstractAbortableFileSystem") + new AbortableStreamBasedCheckpointFileManager(path, conf) + } +} + +@IntegrationTestSuite +class AwsAbortableStreamBasedCheckpointFileManagerSuite + extends AbortableStreamBasedCheckpointFileManagerSuite with BeforeAndAfter { Review Comment: propose an afterAll() which deletes the path if set, swallows any exceptions. keeps costs down as no data is retained on the bucket after the run ########## hadoop-cloud/src/hadoop-3/test/scala/org/apache/spark/internal/io/cloud/AbortableStreamBasedCheckpointFileManagerSuite.scala: ########## @@ -0,0 +1,72 @@ +/* + * 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.internal.io.cloud + +import java.io.File + +import scala.util.Properties + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.scalatest.BeforeAndAfter + +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.cloud.abortable.AbortableFileSystem +import org.apache.spark.sql.execution.streaming.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.CheckpointFileManagerTests + +class AbortableStreamBasedCheckpointFileManagerSuite + extends CheckpointFileManagerTests with Logging { + + override def withTempHadoopPath(p: Path => Unit): Unit = { + withTempDir { f: File => + val basePath = new Path(AbortableFileSystem.ABORTABLE_FS_SCHEME, null, f.getAbsolutePath) + p(basePath) + } + } + + override def checkLeakingCrcFiles(path: Path): Unit = { } + + override def createManager(path: Path): CheckpointFileManager = { + val conf = new Configuration() + conf.set(s"fs.AbstractFileSystem.${AbortableFileSystem.ABORTABLE_FS_SCHEME}.impl", + "org.apache.spark.internal.io.cloud.abortable.AbstractAbortableFileSystem") + new AbortableStreamBasedCheckpointFileManager(path, conf) + } +} + +@IntegrationTestSuite +class AwsAbortableStreamBasedCheckpointFileManagerSuite Review Comment: maybe use s3 in the title as it is s3 only ########## hadoop-cloud/README.md: ########## @@ -0,0 +1,20 @@ +--- +layout: global +title: Spark Hadoop3 Integration Tests +--- + +# Running the Integration Tests + +As mocking of an external systems (like AWS S3) is not always perfect the unit testing should be +extended with integration testing. This is why the build profile `integration-test` has been +introduced here. When it is given (`-pintegration-test`) for testing then only those tests are Review Comment: `-P` rather than `-p`? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
